From c17fa1360df9c63099458de606808c32a9eb0558 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 6 Aug 2019 15:12:47 -0500 Subject: [PATCH 001/149] [SPARK-28514][ML] Remove the redundant transformImpl method in RF & GBT ## What changes were proposed in this pull request? Remove the redundant and confusing transformImpl method in RF & GBT; 1, In `GBTClassifier` & `RandomForestClassifier`, the real `transform` methods inherit from `ProbabilisticClassificationModel` which can deal with multi output columns. The `transformImpl` method, which deals with only one column - `predictionCol`, completely does nothing. This is quite confusing. 2, In `GBTRegressor` & `RandomForestRegressor`, the `transformImpl` do exactly what the superclass `PredictionModel` does (except model broadcasting), so can be removed. ## How was this patch tested? existing suites Closes #25256 from zhengruifeng/del_ensamble_transformImpl. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../apache/spark/ml/classification/Classifier.scala | 3 +++ .../spark/ml/classification/GBTClassifier.scala | 10 +--------- .../ml/classification/RandomForestClassifier.scala | 13 ++----------- 3 files changed, 6 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 568cdd11a12a8..b6b02e77909bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -210,6 +210,9 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur outputData.toDF } + final override def transformImpl(dataset: Dataset[_]): DataFrame = + throw new UnsupportedOperationException(s"transformImpl is not supported in $getClass") + /** * Predict label for the given features. * This method is used to implement `transform()` and output [[predictionCol]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index a5ed4a38a8863..5e0c66b3ab418 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -34,7 +34,7 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ /** @@ -286,14 +286,6 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) - val predictUDF = udf { (features: Any) => - bcastModel.value.predict(features.asInstanceOf[Vector]) - } - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) - } - override def predict(features: Vector): Double = { // If thresholds defined, use predictRaw to get probabilities, otherwise use optimization if (isDefined(thresholds)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 3500f2ad52a56..4424319a4c63c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ -import org.apache.spark.ml.tree.{RandomForestParams, TreeClassifierParams, TreeEnsembleModel} +import org.apache.spark.ml.tree.{TreeClassifierParams, TreeEnsembleModel} import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.{Identifiable, MetadataUtils} @@ -34,8 +34,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.Dataset /** * Random Forest learning algorithm for @@ -208,14 +207,6 @@ class RandomForestClassificationModel private[ml] ( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) - val predictUDF = udf { (features: Any) => - bcastModel.value.predict(features.asInstanceOf[Vector]) - } - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) - } - override protected def predictRaw(features: Vector): Vector = { // TODO: When we add a generic Bagging class, handle transform there: SPARK-7128 // Classifies using majority votes. From 9e3aab8b950cfb11268c2265506ab61594784b3a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 6 Aug 2019 13:39:49 -0700 Subject: [PATCH 002/149] [SPARK-28623][SQL] Support `dow`, `isodow` and `doy` by `extract()` ## What changes were proposed in this pull request? In the PR, I propose to use existing expressions `DayOfYear`, `WeekDay` and `DayOfWeek`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT): 1. `dow` - the day of the week as Sunday (0) to Saturday (6) 2. `isodow` - the day of the week as Monday (1) to Sunday (7) 3. `doy` - the day of the year (1 - 365/366) Here are examples: ```sql spark-sql> SELECT EXTRACT(DOW FROM TIMESTAMP '2001-02-16 20:38:40'); 5 spark-sql> SELECT EXTRACT(ISODOW FROM TIMESTAMP '2001-02-18 20:38:40'); 7 spark-sql> SELECT EXTRACT(DOY FROM TIMESTAMP '2001-02-16 20:38:40'); 47 ``` ## How was this patch tested? Updated `extract.sql`. Closes #25367 from MaxGekk/extract-ext. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/parser/AstBuilder.scala | 6 +++ .../resources/sql-tests/inputs/extract.sql | 6 +++ .../sql-tests/results/extract.sql.out | 48 ++++++++++++++----- 3 files changed, 48 insertions(+), 12 deletions(-) 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 e727abd503de0..ef813d4fd157c 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 @@ -1408,6 +1408,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging DayOfMonth(expression(ctx.source)) case "DAYOFWEEK" => DayOfWeek(expression(ctx.source)) + case "DOW" => + Subtract(DayOfWeek(expression(ctx.source)), Literal(1)) + case "ISODOW" => + Add(WeekDay(expression(ctx.source)), Literal(1)) + case "DOY" => + DayOfYear(expression(ctx.source)) case "HOUR" => Hour(expression(ctx.source)) case "MINUTE" => diff --git a/sql/core/src/test/resources/sql-tests/inputs/extract.sql b/sql/core/src/test/resources/sql-tests/inputs/extract.sql index 9adf5d70056e2..5dd3e6686f1cc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/extract.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/extract.sql @@ -12,6 +12,12 @@ select extract(day from c) from t; select extract(dayofweek from c) from t; +select extract(dow from c) from t; + +select extract(isodow from c) from t; + +select extract(doy from c) from t; + select extract(hour from c) from t; select extract(minute from c) from t; diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index 160e4c7d78455..0ca7bdc09b019 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 11 +-- Number of queries: 14 -- !query 0 @@ -59,34 +59,58 @@ struct -- !query 7 -select extract(hour from c) from t +select extract(dow from c) from t -- !query 7 schema -struct +struct<(dayofweek(CAST(c AS DATE)) - 1):int> -- !query 7 output -7 +5 -- !query 8 -select extract(minute from c) from t +select extract(isodow from c) from t -- !query 8 schema -struct +struct<(weekday(CAST(c AS DATE)) + 1):int> -- !query 8 output -8 +5 -- !query 9 -select extract(second from c) from t +select extract(doy from c) from t -- !query 9 schema -struct +struct -- !query 9 output -9 +126 -- !query 10 -select extract(not_supported from c) from t +select extract(hour from c) from t -- !query 10 schema -struct<> +struct -- !query 10 output +7 + + +-- !query 11 +select extract(minute from c) from t +-- !query 11 schema +struct +-- !query 11 output +8 + + +-- !query 12 +select extract(second from c) from t +-- !query 12 schema +struct +-- !query 12 output +9 + + +-- !query 13 +select extract(not_supported from c) from t +-- !query 13 schema +struct<> +-- !query 13 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'NOT_SUPPORTED' are currently not supported.(line 1, pos 7) From 9e931e787dde3dc586721280fe436ae88db52ed2 Mon Sep 17 00:00:00 2001 From: Nik Vanderhoof Date: Tue, 6 Aug 2019 14:25:53 -0700 Subject: [PATCH 003/149] [SPARK-27905][SQL] Add higher order function 'forall' ## What changes were proposed in this pull request? Add's the higher order function `forall`, which tests an array to see if a predicate holds for every element. The function is implemented in `org.apache.spark.sql.catalyst.expressions.ArrayForAll`. The function is added to the function registry under the pretty name `forall`. ## How was this patch tested? I've added appropriate unit tests for the new ArrayForAll expression in `sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala`. Also added tests for the function in `sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala`. Not sure who is best to ask about this PR so: HyukjinKwon rxin gatorsmile ueshin srowen hvanhovell gatorsmile Closes #24761 from nvander1/feature/for_all. Lead-authored-by: Nik Vanderhoof Co-authored-by: Nik Signed-off-by: Takuya UESHIN --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/higherOrderFunctions.scala | 67 ++++++++++- .../HigherOrderFunctionsSuite.scala | 50 ++++++++ .../spark/sql/DataFrameFunctionsSuite.scala | 110 ++++++++++++++++++ 4 files changed, 227 insertions(+), 1 deletion(-) 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 7404a27e1753f..5177f1e55829e 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 @@ -456,6 +456,7 @@ object FunctionRegistry { expression[MapFilter]("map_filter"), expression[ArrayFilter]("filter"), expression[ArrayExists]("exists"), + expression[ArrayForAll]("forall"), expression[ArrayAggregate]("aggregate"), expression[TransformValues]("transform_values"), expression[TransformKeys]("transform_keys"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index b326e1c4c6af4..87b35b10d29fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -445,8 +445,73 @@ case class ArrayExists( false } } +} + +/** + * Tests whether a predicate holds for all elements in the array. + */ +@ExpressionDescription(usage = + "_FUNC_(expr, pred) - Tests whether a predicate holds for all elements in the array.", + examples = """ + Examples: + > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 0); + false + > SELECT _FUNC_(array(2, 4, 8), x -> x % 2 == 0); + true + > SELECT _FUNC_(array(1, null, 3), x -> x % 2 == 0); + false + > SELECT _FUNC_(array(2, null, 8), x -> x % 2 == 0); + null + """, + since = "3.0.0") +case class ArrayForAll( + argument: Expression, + function: Expression) + extends ArrayBasedSimpleHigherOrderFunction with CodegenFallback { + + override def nullable: Boolean = + super.nullable || function.nullable + + override def dataType: DataType = BooleanType + + override def functionType: AbstractDataType = BooleanType - override def prettyName: String = "exists" + override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArrayForAll = { + val ArrayType(elementType, containsNull) = argument.dataType + copy(function = f(function, (elementType, containsNull) :: Nil)) + } + + @transient lazy val LambdaFunction(_, Seq(elementVar: NamedLambdaVariable), _) = function + + /* + * true for all non null elements foundNull result + * F F F + * F T F + * T F T + * T T N + */ + override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { + val arr = argumentValue.asInstanceOf[ArrayData] + val f = functionForEval + var forall = true + var foundNull = false + var i = 0 + while (i < arr.numElements && forall) { + elementVar.value.set(arr.get(i, elementVar.dataType)) + val ret = f.eval(inputRow) + if (ret == null) { + foundNull = true + } else if (!ret.asInstanceOf[Boolean]) { + forall = false + } + i += 1 + } + if (foundNull && forall) { + null + } else { + forall + } + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index 1411be8007deb..b83d03025d21c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -304,6 +304,56 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq(true, null, true)) } + test("ArrayForAll") { + def forall(expr: Expression, f: Expression => Expression): Expression = { + val ArrayType(et, cn) = expr.dataType + ArrayForAll(expr, createLambda(et, cn, f)).bind(validateBinding) + } + + val ai0 = Literal.create(Seq(2, 4, 8), ArrayType(IntegerType, containsNull = false)) + val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, containsNull = true)) + val ai2 = Literal.create(Seq[Integer](2, null, 8), ArrayType(IntegerType, containsNull = true)) + val ain = Literal.create(null, ArrayType(IntegerType, containsNull = false)) + + val isEven: Expression => Expression = x => x % 2 === 0 + val isNullOrOdd: Expression => Expression = x => x.isNull || x % 2 === 1 + val alwaysFalse: Expression => Expression = _ => Literal.FalseLiteral + val alwaysNull: Expression => Expression = _ => Literal(null, BooleanType) + + checkEvaluation(forall(ai0, isEven), true) + checkEvaluation(forall(ai0, isNullOrOdd), false) + checkEvaluation(forall(ai0, alwaysFalse), false) + checkEvaluation(forall(ai0, alwaysNull), null) + checkEvaluation(forall(ai1, isEven), false) + checkEvaluation(forall(ai1, isNullOrOdd), true) + checkEvaluation(forall(ai1, alwaysFalse), false) + checkEvaluation(forall(ai1, alwaysNull), null) + checkEvaluation(forall(ai2, isEven), null) + checkEvaluation(forall(ai2, isNullOrOdd), false) + checkEvaluation(forall(ai2, alwaysFalse), false) + checkEvaluation(forall(ai2, alwaysNull), null) + checkEvaluation(forall(ain, isEven), null) + checkEvaluation(forall(ain, isNullOrOdd), null) + checkEvaluation(forall(ain, alwaysFalse), null) + checkEvaluation(forall(ain, alwaysNull), null) + + val as0 = + Literal.create(Seq("a0", "a1", "a2", "a3"), ArrayType(StringType, containsNull = false)) + val as1 = Literal.create(Seq(null, "b", "c"), ArrayType(StringType, containsNull = true)) + val asn = Literal.create(null, ArrayType(StringType, containsNull = false)) + + val startsWithA: Expression => Expression = x => x.startsWith("a") + + checkEvaluation(forall(as0, startsWithA), true) + checkEvaluation(forall(as1, startsWithA), false) + checkEvaluation(forall(asn, startsWithA), null) + + val aai = Literal.create(Seq(Seq(1, 3, null), null, Seq(4, 5)), + ArrayType(ArrayType(IntegerType, containsNull = true), containsNull = true)) + checkEvaluation(transform(aai, ix => forall(ix, isNullOrOdd)), + Seq(true, null, false)) + } + test("ArrayAggregate") { val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, containsNull = true)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 3f16f64f4b900..69d442a86c29e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -2322,6 +2322,116 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { assert(ex4.getMessage.contains("cannot resolve '`a`'")) } + test("forall function - array for primitive type not containing null") { + val df = Seq( + Seq(1, 9, 8, 7), + Seq(2, 4, 6), + Seq.empty, + null + ).toDF("i") + + def testArrayOfPrimitiveTypeNotContainsNull(): Unit = { + checkAnswer(df.selectExpr("forall(i, x -> x % 2 == 0)"), + Seq( + Row(false), + Row(true), + Row(true), + Row(null))) + } + + // Test with local relation, the Project will be evaluated without codegen + testArrayOfPrimitiveTypeNotContainsNull() + // Test with cached relation, the Project will be evaluated with codegen + df.cache() + testArrayOfPrimitiveTypeNotContainsNull() + } + + test("forall function - array for primitive type containing null") { + val df = Seq[Seq[Integer]]( + Seq(1, 9, 8, null, 7), + Seq(2, null, null, 4, 6, null), + Seq(2, 4, 6, 8), + Seq.empty, + null + ).toDF("i") + + def testArrayOfPrimitiveTypeContainsNull(): Unit = { + checkAnswer(df.selectExpr("forall(i, x -> x % 2 == 0 or x is null)"), + Seq( + Row(false), + Row(true), + Row(true), + Row(true), + Row(null))) + checkAnswer(df.selectExpr("forall(i, x -> x % 2 == 0)"), + Seq( + Row(false), + Row(null), + Row(true), + Row(true), + Row(null))) + } + + // Test with local relation, the Project will be evaluated without codegen + testArrayOfPrimitiveTypeContainsNull() + // Test with cached relation, the Project will be evaluated with codegen + df.cache() + testArrayOfPrimitiveTypeContainsNull() + } + + test("forall function - array for non-primitive type") { + val df = Seq( + Seq("c", "a", "b"), + Seq[String](null, null, null, null), + Seq.empty, + null + ).toDF("s") + + def testNonPrimitiveType(): Unit = { + checkAnswer(df.selectExpr("forall(s, x -> x is null)"), + Seq( + Row(false), + Row(true), + Row(true), + Row(null))) + } + + // Test with local relation, the Project will be evaluated without codegen + testNonPrimitiveType() + // Test with cached relation, the Project will be evaluated with codegen + df.cache() + testNonPrimitiveType() + } + + test("forall function - invalid") { + val df = Seq( + (Seq("c", "a", "b"), 1), + (Seq("b", null, "c", null), 2), + (Seq.empty, 3), + (null, 4) + ).toDF("s", "i") + + val ex1 = intercept[AnalysisException] { + df.selectExpr("forall(s, (x, y) -> x + y)") + } + assert(ex1.getMessage.contains("The number of lambda function arguments '2' does not match")) + + val ex2 = intercept[AnalysisException] { + df.selectExpr("forall(i, x -> x)") + } + assert(ex2.getMessage.contains("data type mismatch: argument 1 requires array type")) + + val ex3 = intercept[AnalysisException] { + df.selectExpr("forall(s, x -> x)") + } + assert(ex3.getMessage.contains("data type mismatch: argument 2 requires boolean type")) + + val ex4 = intercept[AnalysisException] { + df.selectExpr("forall(a, x -> x)") + } + assert(ex4.getMessage.contains("cannot resolve '`a`'")) + } + test("aggregate function - array for primitive type not containing null") { val df = Seq( Seq(1, 9, 8, 7), From a133175ffaf8f45d0f2ca731fc0eda8d8f397726 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 7 Aug 2019 10:39:47 +0900 Subject: [PATCH 004/149] [SPARK-28615][SQL][DOCS] Add a guide line for dataframe functions to say column signature function is by default ## What changes were proposed in this pull request? Add a guide line for dataframe functions, say: ``` This function APIs usually have methods with Column signature only because it can support not only Column but also other types such as a native string. The other variants currently exist for historical reasons. ``` ## How was this patch tested? N/A Closes #25355 from WeichenXu123/update_functions_guide2. Authored-by: WeichenXu Signed-off-by: HyukjinKwon --- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index db095e0513d64..afafde114a3ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -55,6 +55,10 @@ import org.apache.spark.util.Utils * `regr_count` is an example of a function that is built-in but not defined here, because it is * less commonly used. To invoke it, use `expr("regr_count(yCol, xCol)")`. * + * This function APIs usually have methods with `Column` signature only because it can support not + * only `Column` but also other types such as a native string. The other variants currently exist + * for historical reasons. + * * @groupname udf_funcs UDF functions * @groupname agg_funcs Aggregate functions * @groupname datetime_funcs Date time functions From 44e607e9213bdceab970606fb15292db2fe157c2 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 7 Aug 2019 14:26:45 +0800 Subject: [PATCH 005/149] [SPARK-28238][SQL] Implement DESCRIBE TABLE for Data Source V2 Tables ## What changes were proposed in this pull request? Implements the `DESCRIBE TABLE` logical and physical plans for data source v2 tables. ## How was this patch tested? Added unit tests to `DataSourceV2SQLSuite`. Closes #25040 from mccheah/describe-table-v2. Authored-by: mcheah Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 18 +++- .../sql/catalyst/parser/AstBuilder.scala | 34 +++++++- .../catalyst/plans/DescribeTableSchema.scala | 31 +++++++ .../plans/logical/basicLogicalOperators.scala | 8 ++ .../logical/sql/DescribeColumnStatement.scala | 35 ++++++++ .../logical/sql/DescribeTableStatement.scala | 29 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 43 +++++++++- .../spark/sql/execution/SparkSqlParser.scala | 32 ------- .../spark/sql/execution/command/tables.scala | 11 +-- .../datasources/DataSourceResolution.scala | 16 +++- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../datasources/v2/DescribeTableExec.scala | 84 +++++++++++++++++++ .../sql-tests/results/describe.sql.out | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 6 +- .../sql/execution/SparkSqlParserSuite.scala | 40 +-------- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 52 +++++++++++- .../sources/v2/TestInMemoryTableCatalog.scala | 1 - .../hive/execution/HiveComparisonTest.scala | 3 + 19 files changed, 361 insertions(+), 91 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 68bd47046174d..aa44e677e577b 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -198,7 +198,7 @@ statement | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - tableIdentifier partitionSpec? describeColName? #describeTable + multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery | REFRESH TABLE tableIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource 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 4970727b9646d..f8eef0cf32361 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 @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, InsertIntoStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, DescribeTableStatement, InsertIntoStatement} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -171,6 +171,7 @@ class Analyzer( Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: ResolveAlterTable :: + ResolveDescribeTable :: ResolveInsertInto :: ResolveTables :: ResolveRelations :: @@ -972,6 +973,21 @@ class Analyzer( Seq(TableChange.setProperty("location", newLoc))) } } + /** + * Resolve DESCRIBE TABLE statements that use a DSv2 catalog. + * + * This rule converts unresolved DESCRIBE TABLE statements to v2 when a v2 catalog is responsible + * for the table identifier. A v2 catalog is responsible for an identifier when the identifier + * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and + * the table identifier does not include a catalog. + */ + object ResolveDescribeTable extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case describe @ DescribeTableStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), _, isExtended) => + DescribeTable(UnresolvedRelation(describe.tableName), isExtended) + } + } /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from 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 ef813d4fd157c..49ca09d9ef076 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2567,4 +2567,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.multipartIdentifier), visitLocationSpec(ctx.locationSpec)) } + + /** + * Create a [[DescribeColumnStatement]] or [[DescribeTableStatement]] commands. + */ + override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { + val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null + if (ctx.describeColName != null) { + if (ctx.partitionSpec != null) { + throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) + } else { + DescribeColumnStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.describeColName.nameParts.asScala.map(_.getText), + isExtended) + } + } else { + val partitionSpec = if (ctx.partitionSpec != null) { + // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. + visitPartitionSpec(ctx.partitionSpec).map { + case (key, Some(value)) => key -> value + case (key, _) => + throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) + } + } else { + Map.empty[String, String] + } + DescribeTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + partitionSpec, + isExtended) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala new file mode 100644 index 0000000000000..ff35972b901f9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} + +private[sql] object DescribeTableSchema { + def describeTableAttributes(): Seq[AttributeReference] = Seq( + AttributeReference("col_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("data_type", StringType, nullable = false, + new MetadataBuilder().putString("comment", "data type of the column").build())(), + AttributeReference("comment", StringType, nullable = true, + new MetadataBuilder().putString("comment", "comment of the column").build())()) +} 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 2698ba282f962..6f33944fc1cf4 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} +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, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ @@ -541,6 +542,13 @@ object OverwritePartitionsDynamic { } } +case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override val output = DescribeTableSchema.describeTableAttributes() +} + /** * Drop a table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala new file mode 100644 index 0000000000000..80ee262af672d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala @@ -0,0 +1,35 @@ +/* + * 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.sql + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.types.{MetadataBuilder, StringType} + +case class DescribeColumnStatement( + tableName: Seq[String], + colNameParts: Seq[String], + isExtended: Boolean) extends ParsedStatement { + override def output: Seq[Attribute] = { + Seq( + AttributeReference("info_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column info").build())(), + AttributeReference("info_value", StringType, nullable = false, + new MetadataBuilder().putString("comment", "value of the column info").build())() + ) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala new file mode 100644 index 0000000000000..bf837441bb500 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema + +case class DescribeTableStatement( + tableName: Seq[String], + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends ParsedStatement { + override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 0635f8e5e87e8..d8c3ebc5b9667 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransf import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -617,6 +617,47 @@ class DDLParserSuite extends AnalysisTest { } } + test("describe table column") { + comparePlans(parsePlan("DESCRIBE t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t `abc.xyz`"), + DescribeColumnStatement( + Seq("t"), Seq("abc.xyz"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t abc.xyz"), + DescribeColumnStatement( + Seq("t"), Seq("abc", "xyz"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"), + DescribeColumnStatement( + Seq("t"), Seq("a.b", "x.y"), isExtended = false)) + + comparePlans(parsePlan("DESCRIBE TABLE t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) + comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) + + val caught = intercept[AnalysisException]( + parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col")) + assert(caught.getMessage.contains( + "DESC TABLE COLUMN for a specific partition is not supported")) + } + + test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { + comparePlans(parsePlan("describe t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + comparePlans(parsePlan("describe table t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + comparePlans(parsePlan("describe table extended t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + comparePlans(parsePlan("describe table formatted t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + } + test("insert table: basic append") { Seq( "INSERT INTO TABLE testcat.ns1.ns2.tbl SELECT * FROM source", 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 12cd8abcad890..c4edadba278fe 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 @@ -327,38 +327,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } } - /** - * Create a [[DescribeColumnCommand]] or [[DescribeTableCommand]] logical commands. - */ - override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { - val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null - if (ctx.describeColName != null) { - if (ctx.partitionSpec != null) { - throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) - } else { - DescribeColumnCommand( - visitTableIdentifier(ctx.tableIdentifier), - ctx.describeColName.nameParts.asScala.map(_.getText), - isExtended) - } - } else { - val partitionSpec = if (ctx.partitionSpec != null) { - // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. - visitPartitionSpec(ctx.partitionSpec).map { - case (key, Some(value)) => key -> value - case (key, _) => - throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) - } - } else { - Map.empty[String, String] - } - DescribeTableCommand( - visitTableIdentifier(ctx.tableIdentifier), - partitionSpec, - isExtended) - } - } - /** * Create a [[DescribeQueryCommand]] logical command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 03aca89bc642e..ca42de39db86b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} @@ -499,15 +500,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output: Seq[Attribute] = Seq( - // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("data_type", StringType, nullable = false, - new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference("comment", StringType, nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())() - ) + override val output = DescribeTableSchema.describeTableAttributes() protected def describeSchema( schema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index a51678da2d8e4..f17b31da57312 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DropTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.TableProvider @@ -94,6 +94,18 @@ case class DataSourceResolution( convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) } + case DescribeColumnStatement( + AsTableIdentifier(tableName), colName, isExtended) => + DescribeColumnCommand(tableName, colName, isExtended) + + case DescribeColumnStatement( + CatalogObjectIdentifier(Some(catalog), ident), colName, isExtended) => + throw new AnalysisException("Describing columns is not supported for v2 tables.") + + case DescribeTableStatement( + AsTableIdentifier(tableName), partitionSpec, isExtended) => + DescribeTableCommand(tableName, partitionSpec, isExtended) + case ReplaceTableStatement( AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, V1WriteProvider(provider), options, location, comment, orCreate) => 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 52e2896536355..91fc2e068af70 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -237,6 +237,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } + case DescribeTable(r: DataSourceV2Relation, isExtended) => + DescribeTableExec(r.table, isExtended) :: Nil + case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala new file mode 100644 index 0000000000000..5db79c84a395b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -0,0 +1,84 @@ +/* + * 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.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType + +case class DescribeTableExec(table: Table, isExtended: Boolean) extends LeafExecNode { + + override val output: Seq[AttributeReference] = + DescribeTableSchema.describeTableAttributes() + + private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() + + override protected def doExecute(): RDD[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + addSchema(rows) + + if (isExtended) { + addPartitioning(rows) + addProperties(rows) + } + sparkContext.parallelize(rows) + } + + private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { + rows ++= table.schema.map{ column => + toCatalystRow( + column.name, column.dataType.simpleString, column.getComment().getOrElse("")) + } + } + + private def addPartitioning(rows: ArrayBuffer[InternalRow]): Unit = { + rows += emptyRow() + rows += toCatalystRow(" Partitioning", "", "") + rows += toCatalystRow("--------------", "", "") + if (table.partitioning.isEmpty) { + rows += toCatalystRow("Not partitioned", "", "") + } else { + rows ++= table.partitioning.zipWithIndex.map { + case (transform, index) => toCatalystRow(s"Part $index", transform.describe(), "") + } + } + } + + private def addProperties(rows: ArrayBuffer[InternalRow]): Unit = { + rows += emptyRow() + rows += toCatalystRow(" Table Property", " Value", "") + rows += toCatalystRow("----------------", "-------", "") + rows ++= table.properties.asScala.toList.sortBy(_._1).map { + case (key, value) => toCatalystRow(key, value, "") + } + } + + private def emptyRow(): InternalRow = toCatalystRow("", "", "") + + private def toCatalystRow(strs: String*): InternalRow = { + encoder.toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 46d9ec30a8a77..f58bdb5446b64 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -539,7 +539,7 @@ EXPLAIN EXTENDED DESC t struct -- !query 34 output == Parsed Logical Plan == -DescribeTableCommand `t`, false +'DescribeTableStatement [t], false == Analyzed Logical Plan == col_name: string, data_type: string, comment: string 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 92a1bf63ec944..c6fc84894a3f0 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 @@ -24,6 +24,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString @@ -374,7 +375,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Returns true if the plan is supposed to be sorted. def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case _: DescribeCommandBase | _: DescribeColumnCommand => true + case _: DescribeCommandBase + | _: DescribeColumnCommand + | _: DescribeTableStatement + | _: DescribeColumnStatement => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index df80311c0ce25..b751fb7c50438 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, Un import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -210,51 +211,12 @@ class SparkSqlParserSuite extends AnalysisTest { "no viable alternative at input") } - test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { - assertEqual("describe t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) - assertEqual("describe table t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) - assertEqual("describe table extended t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) - assertEqual("describe table formatted t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) - } - test("describe query") { val query = "SELECT * FROM t" assertEqual("DESCRIBE QUERY " + query, DescribeQueryCommand(query, parser.parsePlan(query))) assertEqual("DESCRIBE " + query, DescribeQueryCommand(query, parser.parsePlan(query))) } - test("describe table column") { - assertEqual("DESCRIBE t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = false)) - assertEqual("DESCRIBE t `abc.xyz`", - DescribeColumnCommand( - TableIdentifier("t"), Seq("abc.xyz"), isExtended = false)) - assertEqual("DESCRIBE t abc.xyz", - DescribeColumnCommand( - TableIdentifier("t"), Seq("abc", "xyz"), isExtended = false)) - assertEqual("DESCRIBE t `a.b`.`x.y`", - DescribeColumnCommand( - TableIdentifier("t"), Seq("a.b", "x.y"), isExtended = false)) - - assertEqual("DESCRIBE TABLE t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = false)) - assertEqual("DESCRIBE TABLE EXTENDED t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = true)) - assertEqual("DESCRIBE TABLE FORMATTED t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = true)) - - intercept("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col", - "DESC TABLE COLUMN for a specific partition is not supported") - } - test("analyze table statistics") { assertEqual("analyze table t compute statistics", AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index a3e029f53cecb..d95021077f9c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -71,6 +71,56 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } + test("DescribeTable using v2 catalog") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + + " USING foo" + + " PARTITIONED BY (id)") + val descriptionDf = spark.sql("DESCRIBE TABLE testcat.table_name") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === + Seq( + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) + val description = descriptionDf.collect() + assert(description === Seq( + Row("id", "bigint", ""), + Row("data", "string", ""))) + } + + test("DescribeTable with v2 catalog when table does not exist.") { + intercept[AnalysisException] { + spark.sql("DESCRIBE TABLE testcat.table_name") + } + } + + test("DescribeTable extended using v2 catalog") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + + " USING foo" + + " PARTITIONED BY (id)" + + " TBLPROPERTIES ('bar'='baz')") + val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.table_name") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) + === Seq( + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) + assert(descriptionDf.collect() + .map(_.toSeq) + .map(_.toArray.map(_.toString.trim)) === Array( + Array("id", "bigint", ""), + Array("data", "string", ""), + Array("", "", ""), + Array("Partitioning", "", ""), + Array("--------------", "", ""), + Array("Part 0", "id", ""), + Array("", "", ""), + Array("Table Property", "Value", ""), + Array("----------------", "-------", ""), + Array("bar", "baz", ""), + Array("provider", "foo", ""))) + + } + test("CreateTable: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $orc2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 19a41bee19d97..7c51a29bde905 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -71,7 +71,6 @@ class TestInMemoryTableCatalog extends TableCatalog { throw new TableAlreadyExistsException(ident) } TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties) tables.put(ident, table) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a4587abbf389d..c0158f1947d99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -30,6 +30,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution @@ -376,6 +377,8 @@ abstract class HiveComparisonTest (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeTableStatement]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive From 99de6a4240b94586b1a416fa5e6df066a6a875f6 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Wed, 7 Aug 2019 00:34:49 -0700 Subject: [PATCH 006/149] [SPARK-27924][SQL][TEST][FOLLOW-UP] Enable Boolean-Predicate syntax tests ## What changes were proposed in this pull request? This PR is a follow-up to https://github.com/apache/spark/pull/25074 ## How was this patch tested? Pass the Jenkins with the newly update test files. Closes #25366 from beliefer/uncomment-boolean-test. Authored-by: gengjiaan Signed-off-by: Dongjoon Hyun --- .../sql-tests/inputs/pgSQL/boolean.sql | 66 ++--- .../sql-tests/results/pgSQL/boolean.sql.out | 239 +++++++++++++----- 2 files changed, 207 insertions(+), 98 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql index fd0d299d7b0ff..4427d76f48d80 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql @@ -191,37 +191,37 @@ SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* -- - thomas 2000-01-04 -- --- SELECT '' AS True, f1 --- FROM BOOLTBL1 --- WHERE f1 IS TRUE; +SELECT '' AS True, f1 + FROM BOOLTBL1 + WHERE f1 IS TRUE; --- SELECT '' AS "Not False", f1 --- FROM BOOLTBL1 --- WHERE f1 IS NOT FALSE; +SELECT '' AS `Not False`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT FALSE; --- SELECT '' AS "False", f1 --- FROM BOOLTBL1 --- WHERE f1 IS FALSE; +SELECT '' AS `False`, f1 + FROM BOOLTBL1 + WHERE f1 IS FALSE; --- SELECT '' AS "Not True", f1 --- FROM BOOLTBL1 --- WHERE f1 IS NOT TRUE; +SELECT '' AS `Not True`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT TRUE; --- SELECT '' AS "True", f1 --- FROM BOOLTBL2 --- WHERE f1 IS TRUE; +SELECT '' AS `True`, f1 + FROM BOOLTBL2 + WHERE f1 IS TRUE; --- SELECT '' AS "Not False", f1 --- FROM BOOLTBL2 --- WHERE f1 IS NOT FALSE; +SELECT '' AS `Not False`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT FALSE; --- SELECT '' AS "False", f1 --- FROM BOOLTBL2 --- WHERE f1 IS FALSE; +SELECT '' AS `False`, f1 + FROM BOOLTBL2 + WHERE f1 IS FALSE; --- SELECT '' AS "Not True", f1 --- FROM BOOLTBL2 --- WHERE f1 IS NOT TRUE; +SELECT '' AS `Not True`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT TRUE; -- -- Tests for BooleanTest @@ -232,15 +232,15 @@ INSERT INTO BOOLTBL3 VALUES ('false', false, 2); INSERT INTO BOOLTBL3 VALUES ('null', null, 3); -- [SPARK-27924] E061-14: Search Conditions --- SELECT --- d, --- b IS TRUE AS istrue, --- b IS NOT TRUE AS isnottrue, --- b IS FALSE AS isfalse, --- b IS NOT FALSE AS isnotfalse, --- b IS UNKNOWN AS isunknown, --- b IS NOT UNKNOWN AS isnotunknown --- FROM booltbl3 ORDER BY o; +SELECT + d, + b IS TRUE AS istrue, + b IS NOT TRUE AS isnottrue, + b IS FALSE AS isfalse, + b IS NOT FALSE AS isnotfalse, + b IS UNKNOWN AS isunknown, + b IS NOT UNKNOWN AS isnotunknown +FROM booltbl3 ORDER BY o; -- Test to make sure short-circuiting and NULL handling is diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out index 3bcc1ad918502..332754652b0fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 83 +-- Number of queries: 92 -- !query 0 @@ -566,176 +566,285 @@ struct -- !query 61 -CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet +SELECT '' AS True, f1 + FROM BOOLTBL1 + WHERE f1 IS TRUE -- !query 61 schema -struct<> +struct -- !query 61 output - + true + true + true -- !query 62 -INSERT INTO BOOLTBL3 VALUES ('true', true, 1) +SELECT '' AS `Not False`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT FALSE -- !query 62 schema -struct<> +struct -- !query 62 output - + true + true + true -- !query 63 -INSERT INTO BOOLTBL3 VALUES ('false', false, 2) +SELECT '' AS `False`, f1 + FROM BOOLTBL1 + WHERE f1 IS FALSE -- !query 63 schema -struct<> +struct -- !query 63 output - + false -- !query 64 -INSERT INTO BOOLTBL3 VALUES ('null', null, 3) +SELECT '' AS `Not True`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT TRUE -- !query 64 schema -struct<> +struct -- !query 64 output - + false -- !query 65 -CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet +SELECT '' AS `True`, f1 + FROM BOOLTBL2 + WHERE f1 IS TRUE -- !query 65 schema -struct<> +struct -- !query 65 output -- !query 66 -INSERT INTO booltbl4 VALUES (false, true, null) +SELECT '' AS `Not False`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT FALSE -- !query 66 schema -struct<> +struct -- !query 66 output - + NULL -- !query 67 -SELECT istrue AND isnul AND istrue FROM booltbl4 +SELECT '' AS `False`, f1 + FROM BOOLTBL2 + WHERE f1 IS FALSE -- !query 67 schema -struct<((istrue AND isnul) AND istrue):boolean> +struct -- !query 67 output -NULL + false + false + false + false -- !query 68 -SELECT istrue AND istrue AND isnul FROM booltbl4 +SELECT '' AS `Not True`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT TRUE -- !query 68 schema -struct<((istrue AND istrue) AND isnul):boolean> +struct -- !query 68 output -NULL + NULL + false + false + false + false -- !query 69 -SELECT isnul AND istrue AND istrue FROM booltbl4 +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet -- !query 69 schema -struct<((isnul AND istrue) AND istrue):boolean> +struct<> -- !query 69 output -NULL + -- !query 70 -SELECT isfalse AND isnul AND istrue FROM booltbl4 +INSERT INTO BOOLTBL3 VALUES ('true', true, 1) -- !query 70 schema -struct<((isfalse AND isnul) AND istrue):boolean> +struct<> -- !query 70 output -false + -- !query 71 -SELECT istrue AND isfalse AND isnul FROM booltbl4 +INSERT INTO BOOLTBL3 VALUES ('false', false, 2) -- !query 71 schema -struct<((istrue AND isfalse) AND isnul):boolean> +struct<> -- !query 71 output -false + -- !query 72 -SELECT isnul AND istrue AND isfalse FROM booltbl4 +INSERT INTO BOOLTBL3 VALUES ('null', null, 3) -- !query 72 schema -struct<((isnul AND istrue) AND isfalse):boolean> +struct<> -- !query 72 output -false + -- !query 73 -SELECT isfalse OR isnul OR isfalse FROM booltbl4 +SELECT + d, + b IS TRUE AS istrue, + b IS NOT TRUE AS isnottrue, + b IS FALSE AS isfalse, + b IS NOT FALSE AS isnotfalse, + b IS UNKNOWN AS isunknown, + b IS NOT UNKNOWN AS isnotunknown +FROM booltbl3 ORDER BY o -- !query 73 schema -struct<((isfalse OR isnul) OR isfalse):boolean> +struct -- !query 73 output -NULL +true true false false true false true +false false true true false false true +null false true false true true false -- !query 74 -SELECT isfalse OR isfalse OR isnul FROM booltbl4 +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet -- !query 74 schema -struct<((isfalse OR isfalse) OR isnul):boolean> +struct<> -- !query 74 output -NULL + -- !query 75 -SELECT isnul OR isfalse OR isfalse FROM booltbl4 +INSERT INTO booltbl4 VALUES (false, true, null) -- !query 75 schema -struct<((isnul OR isfalse) OR isfalse):boolean> +struct<> -- !query 75 output -NULL + -- !query 76 -SELECT isfalse OR isnul OR istrue FROM booltbl4 +SELECT istrue AND isnul AND istrue FROM booltbl4 -- !query 76 schema -struct<((isfalse OR isnul) OR istrue):boolean> +struct<((istrue AND isnul) AND istrue):boolean> -- !query 76 output -true +NULL -- !query 77 -SELECT istrue OR isfalse OR isnul FROM booltbl4 +SELECT istrue AND istrue AND isnul FROM booltbl4 -- !query 77 schema -struct<((istrue OR isfalse) OR isnul):boolean> +struct<((istrue AND istrue) AND isnul):boolean> -- !query 77 output -true +NULL -- !query 78 -SELECT isnul OR istrue OR isfalse FROM booltbl4 +SELECT isnul AND istrue AND istrue FROM booltbl4 -- !query 78 schema -struct<((isnul OR istrue) OR isfalse):boolean> +struct<((isnul AND istrue) AND istrue):boolean> -- !query 78 output -true +NULL -- !query 79 -DROP TABLE BOOLTBL1 +SELECT isfalse AND isnul AND istrue FROM booltbl4 -- !query 79 schema -struct<> +struct<((isfalse AND isnul) AND istrue):boolean> -- !query 79 output - +false -- !query 80 -DROP TABLE BOOLTBL2 +SELECT istrue AND isfalse AND isnul FROM booltbl4 -- !query 80 schema -struct<> +struct<((istrue AND isfalse) AND isnul):boolean> -- !query 80 output - +false -- !query 81 -DROP TABLE BOOLTBL3 +SELECT isnul AND istrue AND isfalse FROM booltbl4 -- !query 81 schema -struct<> +struct<((isnul AND istrue) AND isfalse):boolean> -- !query 81 output - +false -- !query 82 -DROP TABLE BOOLTBL4 +SELECT isfalse OR isnul OR isfalse FROM booltbl4 -- !query 82 schema -struct<> +struct<((isfalse OR isnul) OR isfalse):boolean> -- !query 82 output +NULL + + +-- !query 83 +SELECT isfalse OR isfalse OR isnul FROM booltbl4 +-- !query 83 schema +struct<((isfalse OR isfalse) OR isnul):boolean> +-- !query 83 output +NULL + + +-- !query 84 +SELECT isnul OR isfalse OR isfalse FROM booltbl4 +-- !query 84 schema +struct<((isnul OR isfalse) OR isfalse):boolean> +-- !query 84 output +NULL + + +-- !query 85 +SELECT isfalse OR isnul OR istrue FROM booltbl4 +-- !query 85 schema +struct<((isfalse OR isnul) OR istrue):boolean> +-- !query 85 output +true + + +-- !query 86 +SELECT istrue OR isfalse OR isnul FROM booltbl4 +-- !query 86 schema +struct<((istrue OR isfalse) OR isnul):boolean> +-- !query 86 output +true + + +-- !query 87 +SELECT isnul OR istrue OR isfalse FROM booltbl4 +-- !query 87 schema +struct<((isnul OR istrue) OR isfalse):boolean> +-- !query 87 output +true + + +-- !query 88 +DROP TABLE BOOLTBL1 +-- !query 88 schema +struct<> +-- !query 88 output + + + +-- !query 89 +DROP TABLE BOOLTBL2 +-- !query 89 schema +struct<> +-- !query 89 output + + + +-- !query 90 +DROP TABLE BOOLTBL3 +-- !query 90 schema +struct<> +-- !query 90 output + + + +-- !query 91 +DROP TABLE BOOLTBL4 +-- !query 91 schema +struct<> +-- !query 91 output From 469423f33887a966aaa33eb75f5e7974a0a97beb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 7 Aug 2019 19:14:25 +0800 Subject: [PATCH 007/149] [SPARK-28595][SQL] explain should not trigger partition listing ## What changes were proposed in this pull request? Sometimes when you explain a query, you will get stuck for a while. What's worse, you will get stuck again if you explain again. This is caused by `FileSourceScanExec`: 1. In its `toString`, it needs to report the number of partitions it reads. This needs to query the hive metastore. 2. In its `outputOrdering`, it needs to get all the files. This needs to query the hive metastore. This PR fixes by: 1. `toString` do not need to report the number of partitions it reads. We should report it via SQL metrics. 2. The `outputOrdering` is not very useful. We can only apply it if a) all the bucket columns are read. b) there is only one file in each bucket. This condition is really hard to meet, and even if we meet, sorting an already sorted file is pretty fast and avoiding the sort is not that useful. I think it's worth to give up this optimization so that explain don't need to get stuck. ## How was this patch tested? existing tests Closes #25328 from cloud-fan/ui. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 8 ++++++ .../sql/execution/DataSourceScanExec.scala | 26 +++++++++++------- .../spark/sql/sources/BucketedReadSuite.scala | 10 +++++++ .../sql/hive/execution/HiveExplainSuite.scala | 27 ++++++++++++++++++- 4 files changed, 61 insertions(+), 10 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 ae65f29c32264..80a7d4efe4e52 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 @@ -1787,6 +1787,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING = + buildConf("spark.sql.legacy.bucketedTableScan.outputOrdering") + .internal() + .doc("When true, the bucketed table scan will list files during planning to figure out the " + + "output ordering, which is expensive and may make the planning quite slow.") + .booleanConf + .createWithDefault(false) + val ARITHMETIC_OPERATIONS_FAIL_ON_OVERFLOW = buildConf("spark.sql.arithmeticOperations.failOnOverFlow") .doc("If it is set to true, all arithmetic operations on non-decimal fields throw an " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 984f4d3474b03..b76cd9fc07b47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -187,6 +188,9 @@ case class FileSourceScanExec( val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() val ret = relation.location.listFiles(partitionFilters, dataFilters) + if (relation.partitionSchemaOption.isDefined) { + driverMetrics("numPartitions") = ret.length + } driverMetrics("numFiles") = ret.map(_.files.size.toLong).sum val timeTakenMs = NANOSECONDS.toMillis( (System.nanoTime() - startTime) + optimizerMetadataTimeNs) @@ -237,8 +241,12 @@ case class FileSourceScanExec( val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) + val shouldCalculateSortOrder = + conf.getConf(SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING) && + sortColumns.nonEmpty && + !hasPartitionsAvailableAtRunTime - val sortOrder = if (sortColumns.nonEmpty && !hasPartitionsAvailableAtRunTime) { + val sortOrder = if (shouldCalculateSortOrder) { // In case of bucketing, its possible to have multiple files belonging to the // same bucket in a given relation. Each of these files are locally sorted // but those files combined together are not globally sorted. Given that, @@ -287,12 +295,6 @@ case class FileSourceScanExec( "PushedFilters" -> seqToString(pushedDownFilters), "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) - val withOptPartitionCount = if (relation.partitionSchemaOption.isDefined && - !hasPartitionsAvailableAtRunTime) { - metadata + ("PartitionCount" -> selectedPartitions.size.toString) - } else { - metadata - } val withSelectedBucketsCount = relation.bucketSpec.map { spec => val numSelectedBuckets = optionalBucketSet.map { b => @@ -300,10 +302,10 @@ case class FileSourceScanExec( } getOrElse { spec.numBuckets } - withOptPartitionCount + ("SelectedBucketsCount" -> + metadata + ("SelectedBucketsCount" -> s"$numSelectedBuckets out of ${spec.numBuckets}") } getOrElse { - withOptPartitionCount + metadata } withSelectedBucketsCount @@ -346,6 +348,12 @@ case class FileSourceScanExec( } else { None } + } ++ { + if (relation.partitionSchemaOption.isDefined) { + Some("numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions read")) + } else { + None + } } protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index d52a78033e6cd..c3edec39979ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -49,6 +49,16 @@ class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedS abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { import testImplicits._ + protected override def beforeAll(): Unit = { + super.beforeAll() + spark.sessionState.conf.setConf(SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING, true) + } + + protected override def afterAll(): Unit = { + spark.sessionState.conf.unsetConf(SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING) + super.afterAll() + } + private val maxI = 5 private val maxJ = 13 private lazy val df = (0 until 50).map(i => (i % maxI, i % maxJ, i.toString)).toDF("i", "j", "k") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index d413dfb2b2dc5..68ccee5e6623a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -207,4 +207,29 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } } + + test("SPARK-28595: explain should not trigger partition listing") { + Seq(true, false).foreach { legacyBucketedScan => + withSQLConf( + SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING.key -> legacyBucketedScan.toString) { + HiveCatalogMetrics.reset() + withTable("t") { + sql( + """ + |CREATE TABLE t USING json + |PARTITIONED BY (j) + |CLUSTERED BY (i) SORTED BY (i) INTO 4 BUCKETS + |AS SELECT 1 i, 2 j + """.stripMargin) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount == 0) + spark.table("t").sort($"i").explain() + if (legacyBucketedScan) { + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount > 0) + } else { + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount == 0) + } + } + } + } + } } From a59fdc4b5783be591a236bfc60d1107caa818412 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 7 Aug 2019 08:51:58 -0700 Subject: [PATCH 008/149] [SPARK-28472][SQL][TEST] Add test for thriftserver protocol versions ## What changes were proposed in this pull request? This pr adds a test(`SparkThriftServerProtocolVersionsSuite`) to test different versions of the thrift protocol because we use different logic to handle the `RowSet`: https://github.com/apache/spark/blob/02c33694c8254f69cb36c71c0876194dccdbc014/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java#L28-L40 When adding this test cases, found three bugs: [SPARK-26969](https://issues.apache.org/jira/browse/SPARK-26969): Using ODBC not able to see the data in table when datatype is decimal [SPARK-28463](https://issues.apache.org/jira/browse/SPARK-28463): Thriftserver throws BigDecimal incompatible with HiveDecimal [SPARK-28474](https://issues.apache.org/jira/browse/SPARK-28474): Lower JDBC client version(Hive 0.12) cannot read binary type ## How was this patch tested? N/A Closes #25228 from wangyum/SPARK-28472. Authored-by: Yuming Wang Signed-off-by: gatorsmile --- ...arkThriftServerProtocolVersionsSuite.scala | 316 ++++++++++++++++++ .../thriftserver/ThriftserverShimUtils.scala | 13 + .../thriftserver/ThriftserverShimUtils.scala | 15 + 3 files changed, 344 insertions(+) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala new file mode 100644 index 0000000000000..e33e42f3cc8ca --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.sql.{Date, Timestamp} +import java.util.{List => JList, Properties} + +import org.apache.hive.jdbc.{HiveConnection, HiveQueryResultSet} +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.GetInfoType +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.TSocket + +class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { + + override def mode: ServerMode.Value = ServerMode.binary + + def testExecuteStatementWithProtocolVersion( + version: ThriftserverShimUtils.TProtocolVersion, + sql: String)(f: HiveQueryResultSet => Unit): Unit = { + val rawTransport = new TSocket("localhost", serverPort) + val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) + val user = System.getProperty("user.name") + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + transport.open() + var rs: HiveQueryResultSet = null + try { + val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val openResp = client.OpenSession(clientProtocol) + val sessHandle = openResp.getSessionHandle + val execReq = new ThriftserverShimUtils.TExecuteStatementReq(sessHandle, sql) + val execResp = client.ExecuteStatement(execReq) + val stmtHandle = execResp.getOperationHandle + + // Set the HiveConnection protocol to our testing protocol version. + // RowSetFactory uses this protocol version to construct different RowSet. + val protocol = connection.getClass.getDeclaredField("protocol") + protocol.setAccessible(true) + protocol.set(connection, version) + assert(connection.getProtocol === version) + + rs = new HiveQueryResultSet.Builder(connection) + .setClient(client) + .setSessionHandle(sessHandle) + .setStmtHandle(stmtHandle).setMaxRows(Int.MaxValue).setFetchSize(Int.MaxValue) + .build() + f(rs) + } finally { + rs.close() + connection.close() + transport.close() + rawTransport.close() + } + } + + def testGetInfoWithProtocolVersion(version: ThriftserverShimUtils.TProtocolVersion): Unit = { + val rawTransport = new TSocket("localhost", serverPort) + val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + transport.open() + try { + val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val openResp = client.OpenSession(clientProtocol) + val sessHandle = openResp.getSessionHandle + + val dbVersionReq = + new ThriftserverShimUtils.TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_VER.toTGetInfoType) + val dbVersion = client.GetInfo(dbVersionReq).getInfoValue.getStringValue + + val dbNameReq = + new ThriftserverShimUtils.TGetInfoReq(sessHandle, GetInfoType.CLI_DBMS_NAME.toTGetInfoType) + val dbName = client.GetInfo(dbNameReq).getInfoValue.getStringValue + + assert(dbVersion === org.apache.spark.SPARK_VERSION) + assert(dbName === "Spark SQL") + } finally { + connection.close() + transport.close() + rawTransport.close() + } + } + + def testGetTablesWithProtocolVersion( + version: ThriftserverShimUtils.TProtocolVersion, + schema: String, + tableNamePattern: String, + tableTypes: JList[String])(f: HiveQueryResultSet => Unit): Unit = { + val rawTransport = new TSocket("localhost", serverPort) + val connection = new HiveConnection(s"jdbc:hive2://localhost:$serverPort", new Properties) + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val client = new ThriftserverShimUtils.Client(new TBinaryProtocol(transport)) + transport.open() + var rs: HiveQueryResultSet = null + try { + val clientProtocol = new ThriftserverShimUtils.TOpenSessionReq(version) + val openResp = client.OpenSession(clientProtocol) + val sessHandle = openResp.getSessionHandle + val getTableReq = new ThriftserverShimUtils.TGetTablesReq(sessHandle) + getTableReq.setSchemaName(schema) + getTableReq.setTableName(tableNamePattern) + getTableReq.setTableTypes(tableTypes) + + val getTableResp = client.GetTables(getTableReq) + + // Set the HiveConnection protocol to our testing protocol version. + // RowSetFactory uses this protocol version to construct different RowSet. + val protocol = connection.getClass.getDeclaredField("protocol") + protocol.setAccessible(true) + protocol.set(connection, version) + assert(connection.getProtocol === version) + + rs = new HiveQueryResultSet.Builder(connection) + .setClient(client) + .setSessionHandle(sessHandle) + .setStmtHandle(getTableResp.getOperationHandle) + .build() + f(rs) + } finally { + rs.close() + connection.close() + transport.close() + rawTransport.close() + } + } + + ThriftserverShimUtils.testedProtocolVersions.foreach { version => + test(s"$version get byte type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as byte)") { rs => + assert(rs.next()) + assert(rs.getByte(1) === 1.toByte) + } + } + + test(s"$version get short type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as short)") { rs => + assert(rs.next()) + assert(rs.getShort(1) === 1.toShort) + } + } + + test(s"$version get int type") { + testExecuteStatementWithProtocolVersion(version, "SELECT 1") { rs => + assert(rs.next()) + assert(rs.getInt(1) === 1) + } + } + + test(s"$version get bigint type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as bigint)") { rs => + assert(rs.next()) + assert(rs.getLong(1) === 1L) + } + } + + test(s"$version get float type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast(1.2 as float)") { rs => + assert(rs.next()) + assert(rs.getFloat(1) === 1.2F) + } + } + + test(s"$version get double type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast(1.2 as double)") { rs => + assert(rs.next()) + assert(rs.getDouble(1) === 1.2D) + } + } + + // TODO: enable this test case after SPARK-28463 and SPARK-26969 + ignore(s"$version get decimal type") { + testExecuteStatementWithProtocolVersion(version, + "SELECT cast(1 as decimal(18, 2)) as c") { rs => + assert(rs.next()) + assert(rs.getBigDecimal(1) === new java.math.BigDecimal("1.00")) + } + } + + test(s"$version get string type") { + testExecuteStatementWithProtocolVersion(version, "SELECT 'str'") { rs => + assert(rs.next()) + assert(rs.getString(1) === "str") + } + } + + test(s"$version get char type") { + testExecuteStatementWithProtocolVersion(version, + "SELECT cast('char-str' as char(10))") { rs => + assert(rs.next()) + assert(rs.getString(1) === "char-str") + } + } + + test(s"$version get varchar type") { + testExecuteStatementWithProtocolVersion(version, + "SELECT cast('varchar-str' as varchar(10))") { rs => + assert(rs.next()) + assert(rs.getString(1) === "varchar-str") + } + } + + // TODO: enable this test case after SPARK-28474 + ignore(s"$version get binary type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast('ABC' as binary)") { rs => + assert(rs.next()) + assert(rs.getString(1) === "ABC") + } + } + + test(s"$version get boolean type") { + testExecuteStatementWithProtocolVersion(version, "SELECT true") { rs => + assert(rs.next()) + assert(rs.getBoolean(1) === true) + } + } + + test(s"$version get date type") { + testExecuteStatementWithProtocolVersion(version, "SELECT cast('2019-07-22' as date)") { rs => + assert(rs.next()) + assert(rs.getDate(1) === Date.valueOf("2019-07-22")) + } + } + + test(s"$version get timestamp type") { + testExecuteStatementWithProtocolVersion(version, + "SELECT cast('2019-07-22 18:14:00' as timestamp)") { rs => + assert(rs.next()) + assert(rs.getTimestamp(1) === Timestamp.valueOf("2019-07-22 18:14:00")) + } + } + + // TODO: enable this test case after port HIVE-10646 + ignore(s"$version get void") { + testExecuteStatementWithProtocolVersion(version, "SELECT null") { rs => + assert(rs.next()) + assert(rs.getString(1) === null) + } + } + + // We do not fully support interval type + ignore(s"$version get interval type") { + testExecuteStatementWithProtocolVersion(version, "SELECT interval '1' year '2' day") { rs => + assert(rs.next()) + } + } + + test(s"$version get array type") { + testExecuteStatementWithProtocolVersion(version, "SELECT array(1, 2)") { rs => + assert(rs.next()) + assert(rs.getString(1) === "[1,2]") + } + } + + test(s"$version get map type") { + testExecuteStatementWithProtocolVersion(version, "SELECT map(1, 2)") { rs => + assert(rs.next()) + assert(rs.getString(1) === "{1:2}") + } + } + + test(s"$version get struct type") { + testExecuteStatementWithProtocolVersion(version, + "SELECT struct('alpha' AS A, 'beta' AS B)") { rs => + assert(rs.next()) + assert(rs.getString(1) === """{"A":"alpha","B":"beta"}""") + } + } + + test(s"$version get info") { + testGetInfoWithProtocolVersion(version) + } + + test(s"$version get tables") { + def checkResult(tableNames: Seq[String], rs: HiveQueryResultSet): Unit = { + if (tableNames.nonEmpty) { + for (i <- tableNames.indices) { + assert(rs.next()) + assert(rs.getString("TABLE_NAME") === tableNames(i)) + } + } else { + assert(!rs.next()) + } + } + + withJdbcStatement("table1", "table2") { statement => + Seq( + "CREATE TABLE table1(key INT, val STRING)", + "CREATE TABLE table2(key INT, val STRING)").foreach(statement.execute) + + testGetTablesWithProtocolVersion(version, "%", "%", null) { rs => + checkResult(Seq("table1", "table2"), rs) + } + + testGetTablesWithProtocolVersion(version, "%", "table1", null) { rs => + checkResult(Seq("table1"), rs) + } + } + } + } +} diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index b0702ade2cedc..4eb5f5da8fdcc 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} +import org.apache.hive.service.cli.thrift.TProtocolVersion._ /** * Various utilities for hive-thriftserver used to upgrade the built-in Hive. @@ -32,6 +33,9 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] type TGetSchemasReq = org.apache.hive.service.cli.thrift.TGetSchemasReq private[thriftserver] type TGetTablesReq = org.apache.hive.service.cli.thrift.TGetTablesReq private[thriftserver] type TGetColumnsReq = org.apache.hive.service.cli.thrift.TGetColumnsReq + private[thriftserver] type TGetInfoReq = org.apache.hive.service.cli.thrift.TGetInfoReq + private[thriftserver] type TExecuteStatementReq = + org.apache.hive.service.cli.thrift.TExecuteStatementReq private[thriftserver] def getConsole: SessionState.LogHelper = { val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) @@ -46,4 +50,13 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + private[thriftserver] val testedProtocolVersions = Seq( + HIVE_CLI_SERVICE_PROTOCOL_V1, + HIVE_CLI_SERVICE_PROTOCOL_V2, + HIVE_CLI_SERVICE_PROTOCOL_V3, + HIVE_CLI_SERVICE_PROTOCOL_V4, + HIVE_CLI_SERVICE_PROTOCOL_V5, + HIVE_CLI_SERVICE_PROTOCOL_V6, + HIVE_CLI_SERVICE_PROTOCOL_V7, + HIVE_CLI_SERVICE_PROTOCOL_V8) } diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index 75637a81a3ab5..d586c0e1b6eae 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.thrift.Type import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema} +import org.apache.hive.service.rpc.thrift.TProtocolVersion._ import org.slf4j.LoggerFactory /** @@ -33,6 +34,9 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] type TGetSchemasReq = org.apache.hive.service.rpc.thrift.TGetSchemasReq private[thriftserver] type TGetTablesReq = org.apache.hive.service.rpc.thrift.TGetTablesReq private[thriftserver] type TGetColumnsReq = org.apache.hive.service.rpc.thrift.TGetColumnsReq + private[thriftserver] type TGetInfoReq = org.apache.hive.service.rpc.thrift.TGetInfoReq + private[thriftserver] type TExecuteStatementReq = + org.apache.hive.service.rpc.thrift.TExecuteStatementReq private[thriftserver] def getConsole: SessionState.LogHelper = { val LOG = LoggerFactory.getLogger(classOf[SparkSQLCLIDriver]) @@ -47,4 +51,15 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + private[thriftserver] val testedProtocolVersions = Seq( + HIVE_CLI_SERVICE_PROTOCOL_V1, + HIVE_CLI_SERVICE_PROTOCOL_V2, + HIVE_CLI_SERVICE_PROTOCOL_V3, + HIVE_CLI_SERVICE_PROTOCOL_V4, + HIVE_CLI_SERVICE_PROTOCOL_V5, + HIVE_CLI_SERVICE_PROTOCOL_V6, + HIVE_CLI_SERVICE_PROTOCOL_V7, + HIVE_CLI_SERVICE_PROTOCOL_V8, + HIVE_CLI_SERVICE_PROTOCOL_V9, + HIVE_CLI_SERVICE_PROTOCOL_V10) } From 325bc8e9c6187a96b33a033fbb0145dfca619135 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Wed, 7 Aug 2019 22:10:17 +0200 Subject: [PATCH 009/149] [SPARK-28583][SQL] Subqueries should not call `onUpdatePlan` in Adaptive Query Execution ## What changes were proposed in this pull request? Subqueries do not have their own execution id, thus when calling `AdaptiveSparkPlanExec.onUpdatePlan`, it will actually get the `QueryExecution` instance of the main query, which is wasteful and problematic. It could cause issues like stack overflow or dead locks in some circumstances. This PR fixes this issue by making `AdaptiveSparkPlanExec` compare the `QueryExecution` object retrieved by current execution ID against the `QueryExecution` object from which this plan is created, and only update the UI when the two instances are the same. ## How was this patch tested? Manual tests on TPC-DS queries. Closes #25316 from maryannxue/aqe-updateplan-fix. Authored-by: maryannxue Signed-off-by: herman --- .../spark/sql/execution/QueryExecution.scala | 2 +- .../adaptive/AdaptiveSparkPlanExec.scala | 17 +++++++++++++---- .../adaptive/InsertAdaptiveSparkPlan.scala | 12 ++++++++---- 3 files changed, 22 insertions(+), 9 deletions(-) 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 1583b8d3a1f91..5f67b17f8e6fe 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 @@ -118,7 +118,7 @@ class QueryExecution( protected def preparations: Seq[Rule[SparkPlan]] = Seq( // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op // as the original plan is hidden behind `AdaptiveSparkPlanExec`. - InsertAdaptiveSparkPlan(sparkSession), + InsertAdaptiveSparkPlan(sparkSession, this), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 61dbc58297388..e7bbbd7bf4642 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -61,7 +61,8 @@ case class AdaptiveSparkPlanExec( initialPlan: SparkPlan, @transient session: SparkSession, @transient subqueryMap: Map[Long, ExecSubqueryExpression], - @transient stageCache: TrieMap[SparkPlan, QueryStageExec]) + @transient stageCache: TrieMap[SparkPlan, QueryStageExec], + @transient queryExecution: QueryExecution) extends LeafExecNode { @transient private val lock = new Object() @@ -118,8 +119,15 @@ case class AdaptiveSparkPlanExec( if (isFinalPlan) { currentPhysicalPlan.execute() } else { + // Make sure we only update Spark UI if this plan's `QueryExecution` object matches the one + // retrieved by the `sparkContext`'s current execution ID. Note that sub-queries do not have + // their own execution IDs and therefore rely on the main query to update UI. val executionId = Option( - session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) + session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).flatMap { idStr => + val id = idStr.toLong + val qe = SQLExecution.getQueryExecution(id) + if (qe.eq(queryExecution)) Some(id) else None + } var currentLogicalPlan = currentPhysicalPlan.logicalLink.get var result = createQueryStages(currentPhysicalPlan) val events = new LinkedBlockingQueue[StageMaterializationEvent]() @@ -171,10 +179,11 @@ case class AdaptiveSparkPlanExec( currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) currentPhysicalPlan.setTagValue(SparkPlan.LOGICAL_PLAN_TAG, currentLogicalPlan) isFinalPlan = true + + val ret = currentPhysicalPlan.execute() logDebug(s"Final plan: $currentPhysicalPlan") executionId.foreach(onUpdatePlan) - - currentPhysicalPlan.execute() + ret } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 14ca2b41a442b..50c10a154a3c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -35,14 +35,18 @@ import org.apache.spark.sql.types.StructType * * Note that this rule is stateful and thus should not be reused across query executions. */ -case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan] { +case class InsertAdaptiveSparkPlan( + session: SparkSession, + queryExecution: QueryExecution) extends Rule[SparkPlan] { private val conf = session.sessionState.conf // Exchange-reuse is shared across the entire query, including sub-queries. private val stageCache = new TrieMap[SparkPlan, QueryStageExec]() - override def apply(plan: SparkPlan): SparkPlan = plan match { + override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, queryExecution) + + private def applyInternal(plan: SparkPlan, qe: QueryExecution): SparkPlan = plan match { case _: ExecutedCommandExec => plan case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) => try { @@ -54,7 +58,7 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan session.sessionState.conf, subqueryMap) val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preparations) logDebug(s"Adaptive execution enabled for plan: $plan") - AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache) + AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache, qe) } catch { case SubqueryAdaptiveNotSupportedException(subquery) => logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + @@ -120,7 +124,7 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan val queryExec = new QueryExecution(session, plan) // Apply the same instance of this rule to sub-queries so that sub-queries all share the // same `stageCache` for Exchange reuse. - val adaptivePlan = this.apply(queryExec.sparkPlan) + val adaptivePlan = this.applyInternal(queryExec.sparkPlan, queryExec) if (!adaptivePlan.isInstanceOf[AdaptiveSparkPlanExec]) { throw SubqueryAdaptiveNotSupportedException(plan) } From 8617bf6ff8c1a2021bc0132b5f668aaef8aed892 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 8 Aug 2019 08:10:21 +0900 Subject: [PATCH 010/149] [SPARK-28470][SQL] Cast to decimal throws ArithmeticException on overflow ## What changes were proposed in this pull request? The flag `spark.sql.decimalOperations.nullOnOverflow` is not honored by the `Cast` operator. This means that a casting which causes an overflow currently returns `null`. The PR makes `Cast` respecting that flag, ie. when it is turned to false and a decimal overflow occurs, an exception id thrown. ## How was this patch tested? Added UT Closes #25253 from mgaido91/SPARK-28470. Authored-by: Marco Gaido Signed-off-by: Takeshi Yamamuro --- .../spark/sql/catalyst/expressions/Cast.scala | 36 +++++++++++++++---- .../sql/catalyst/expressions/CastSuite.scala | 22 ++++++++++++ 2 files changed, 52 insertions(+), 6 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 85bad74850dc1..10464dac8d55e 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -499,22 +500,37 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } + private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + /** * Change the precision / scale in a given decimal to those set in `decimalType` (if any), - * returning null if it overflows or modifying `value` in-place and returning it if successful. + * modifying `value` in-place and returning it if successful. If an overflow occurs, it + * either returns null or throws an exception according to the value set for + * `spark.sql.decimalOperations.nullOnOverflow`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ private[this] def changePrecision(value: Decimal, decimalType: DecimalType): Decimal = { - if (value.changePrecision(decimalType.precision, decimalType.scale)) value else null + if (value.changePrecision(decimalType.precision, decimalType.scale)) { + value + } else { + if (nullOnOverflow) { + null + } else { + throw new ArithmeticException(s"${value.toDebugString} cannot be represented as " + + s"Decimal(${decimalType.precision}, ${decimalType.scale}).") + } + } } /** - * Create new `Decimal` with precision and scale given in `decimalType` (if any), - * returning null if it overflows or creating a new `value` and returning it if successful. + * Create new `Decimal` with precision and scale given in `decimalType` (if any). + * If overflow occurs, if `spark.sql.decimalOperations.nullOnOverflow` is true, null is returned; + * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = - value.toPrecision(decimalType.precision, decimalType.scale) + value.toPrecision( + decimalType.precision, decimalType.scale, Decimal.ROUND_HALF_UP, nullOnOverflow) private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { @@ -964,11 +980,19 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String |$evPrim = $d; """.stripMargin } else { + val overflowCode = if (nullOnOverflow) { + s"$evNull = true;" + } else { + s""" + |throw new ArithmeticException($d.toDebugString() + " cannot be represented as " + + | "Decimal(${decimalType.precision}, ${decimalType.scale})."); + """.stripMargin + } code""" |if ($d.changePrecision(${decimalType.precision}, ${decimalType.scale})) { | $evPrim = $d; |} else { - | $evNull = true; + | $overflowCode |} """.stripMargin } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 44825c79781d9..bbb3cb516b7d5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -1023,4 +1024,25 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ret, InternalRow(null)) } } + + test("SPARK-28470: Cast should honor nullOnOverflow property") { + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) + checkEvaluation( + Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(134.12), DecimalType(3, 2)), null) + } + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + checkExceptionInExpression[ArithmeticException]( + Cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") + checkExceptionInExpression[ArithmeticException]( + Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), + "cannot be represented") + checkExceptionInExpression[ArithmeticException]( + Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), "cannot be represented") + checkExceptionInExpression[ArithmeticException]( + Cast(Literal(134.12), DecimalType(3, 2)), "cannot be represented") + } + } } From c88df2ccf670db62aed6565c9dbdb58d5d5cca3f Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 7 Aug 2019 16:14:34 -0700 Subject: [PATCH 011/149] [SPARK-28331][SQL] Catalogs.load() should be able to load built-in catalogs ## What changes were proposed in this pull request? In `Catalogs.load`, the `pluginClassName` in the following code ``` String pluginClassName = conf.getConfString("spark.sql.catalog." + name, null); ``` is always null for built-in catalogs, e.g there is a SQLConf entry `spark.sql.catalog.session`. This is because of https://github.com/apache/spark/pull/18852: SQLConf.conf.getConfString(key, null) always returns null. ## How was this patch tested? Apply code changes of https://github.com/apache/spark/pull/24768 and tried loading session catalog. Closes #25094 from gengliangwang/fixCatalogLoad. Authored-by: Gengliang Wang Signed-off-by: Burak Yavuz --- .../java/org/apache/spark/sql/catalog/v2/Catalogs.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java index 7511d947615a6..f471a4e71cf43 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java @@ -26,6 +26,7 @@ import java.lang.reflect.InvocationTargetException; import java.util.HashMap; import java.util.Map; +import java.util.NoSuchElementException; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -50,8 +51,10 @@ private Catalogs() { */ public static CatalogPlugin load(String name, SQLConf conf) throws CatalogNotFoundException, SparkException { - String pluginClassName = conf.getConfString("spark.sql.catalog." + name, null); - if (pluginClassName == null) { + String pluginClassName; + try { + pluginClassName = conf.getConfString("spark.sql.catalog." + name); + } catch (NoSuchElementException e){ throw new CatalogNotFoundException(String.format( "Catalog '%s' plugin class not found: spark.sql.catalog.%s is not defined", name, name)); } From eeaf1851b2eced82afad10eb731f0ecebf22c6e3 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 7 Aug 2019 16:45:23 -0700 Subject: [PATCH 012/149] [SPARK-28617][SQL][TEST] Fix misplacement when comment is at the end of the query ## What changes were proposed in this pull request? This PR fixes the issue of misplacement when the comment at the end of the query. Example: Comment for ` SELECT date '5874898-01-01'`: https://github.com/apache/spark/blob/2d74f14d74e7b24109f347822600ebf9819b04c1/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql#L200 But the golden file is: https://github.com/apache/spark/blob/a5a5da78cfbb1b439c139fd2ffd5744fc60ebafe/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out#L484-L507 After this PR: https://github.com/apache/spark/blob/eeb7405ad0c7cc1004e2cad36929d20d95ab2726/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out#L482-L501 ## How was this patch tested? N/A Closes #25357 from wangyum/SPARK-28617. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../results/pgSQL/aggregates_part2.sql.out | 6 --- .../sql-tests/results/pgSQL/boolean.sql.out | 3 -- .../sql-tests/results/pgSQL/date.sql.out | 8 +--- .../sql-tests/results/pgSQL/join.sql.out | 42 +++---------------- .../sql-tests/results/pgSQL/select.sql.out | 1 - .../sql-tests/results/pgSQL/with.sql.out | 5 +-- ...in-unit-tests-multi-column-literal.sql.out | 6 --- .../not-in-unit-tests-multi-column.sql.out | 14 ------- ...n-unit-tests-single-column-literal.sql.out | 12 ------ .../not-in-unit-tests-single-column.sql.out | 18 -------- .../udf/pgSQL/udf-aggregates_part2.sql.out | 6 --- .../sql-tests/results/udf/udf-udaf.sql.out | 2 - .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +- 13 files changed, 10 insertions(+), 117 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out index 2606d2eba7468..2b5371a657196 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out @@ -18,14 +18,11 @@ struct<> -- !query 1 SELECT - -- boolean and transitions - -- null because strict (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, (FALSE AND NULL) IS NULL AS `t`, (NULL AND TRUE) IS NULL AS `t`, (NULL AND FALSE) IS NULL AS `t`, - -- and actual computations (TRUE AND TRUE) AS `t`, NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, @@ -38,14 +35,11 @@ true true false true false true true true true -- !query 2 SELECT - -- boolean or transitions - -- null because strict (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, (FALSE OR NULL) IS NULL AS `t`, (NULL OR TRUE) IS NULL AS `t`, (NULL OR FALSE) IS NULL AS `t`, - -- actual computations (TRUE OR TRUE) AS `t`, (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out index 332754652b0fb..c7903c8a34ef4 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out @@ -316,7 +316,6 @@ NULL -- !query 39 --- error SELECT boolean(string('')) AS invalid -- !query 39 schema struct @@ -325,8 +324,6 @@ NULL -- !query 40 --- error - CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet -- !query 40 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 3f45c386fb90a..46101ebce8113 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -452,17 +452,15 @@ struct -- !query 44 --- out of range SELECT date '5874897-12-31' -- !query 44 schema struct<> -- !query 44 output org.apache.spark.sql.catalyst.parser.ParseException -Cannot parse the DATE value: 5874897-12-31(line 2, pos 7) +Cannot parse the DATE value: 5874897-12-31(line 1, pos 7) == SQL == --- out of range SELECT date '5874897-12-31' -------^^^ @@ -482,10 +480,6 @@ SELECT date '5874898-01-01' -- !query 46 --- out of range - - - SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL -- !query 46 schema struct diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out index 0730066719764..f75fe0519645b 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out @@ -3257,18 +3257,16 @@ Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63 -- !query 168 --- error select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 -- !query 168 schema struct<> -- !query 168 output org.apache.spark.sql.AnalysisException -cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 3 pos 63 +cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 -- !query 169 --- error select * from int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 -- !query 169 schema @@ -3278,72 +3276,42 @@ struct -- !query 170 --- ok - - select t1.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query 170 schema struct<> -- !query 170 output org.apache.spark.sql.AnalysisException -cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 4 pos 7 +cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 -- !query 171 --- error, prefer "t1" suggestion select t2.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query 171 schema struct<> -- !query 171 output org.apache.spark.sql.AnalysisException -cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 2 pos 7 +cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 -- !query 172 --- error, prefer "t2" suggestion select uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query 172 schema struct<> -- !query 172 output org.apache.spark.sql.AnalysisException -cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 2 pos 7 +cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 -- !query 173 --- error, suggest both at once - - - - - - - - - - - - - - - - - - - - - - - - select f1,g from int4_tbl a, (select f1 as g) ss -- !query 173 schema struct<> -- !query 173 output org.apache.spark.sql.AnalysisException -cannot resolve '`f1`' given input columns: []; line 26 pos 37 +cannot resolve '`f1`' given input columns: []; line 1 pos 37 -- !query 174 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out index 797f808dad11e..e54de1d6fdbdc 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out @@ -399,7 +399,6 @@ NULL -- !query 22 --- same thing SELECT * FROM foo ORDER BY f1 NULLS FIRST -- !query 22 schema struct diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out index 366b65f3659cd..91b0ff20b6ab0 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out @@ -358,23 +358,20 @@ create table foo (with baz) -- !query 38 --- fail, WITH is a reserved word create table foo (with ordinality) -- !query 38 schema struct<> -- !query 38 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'with'(line 2, pos 18) +no viable alternative at input 'with'(line 1, pos 18) == SQL == --- fail, WITH is a reserved word create table foo (with ordinality) ------------------^^^ -- !query 39 --- fail, WITH is a reserved word with ordinality as (select 1 as x) select * from ordinality -- !query 39 schema struct diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out index a16e98af9a417..f02f760727976 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -16,8 +16,6 @@ struct<> -- !query 1 --- Case 5 - -- (one null column with no match -> row is returned) SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) @@ -29,8 +27,6 @@ NULL 1 -- !query 2 --- Case 6 - -- (no null columns with match -> row not returned) SELECT * FROM m WHERE b = 3.0 -- Matches (2, 3.0) @@ -42,8 +38,6 @@ struct -- !query 3 --- Case 7 - -- (no null columns with no match -> row is returned) SELECT * FROM m WHERE b = 5.0 -- Matches (4, 5.0) diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out index aa5f64b8ebf55..a27a66e3f27f5 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -29,8 +29,6 @@ struct<> -- !query 2 --- Case 1 - -- (subquery is empty -> row is returned) SELECT * FROM m WHERE (a, b) NOT IN (SELECT * @@ -46,8 +44,6 @@ NULL NULL -- !query 3 --- Case 2 - -- (subquery contains a row with null in all columns -> row not returned) SELECT * FROM m WHERE (a, b) NOT IN (SELECT * @@ -60,8 +56,6 @@ struct -- !query 4 --- Case 3 - -- (probe-side columns are all null -> row not returned) SELECT * FROM m WHERE a IS NULL AND b IS NULL -- Matches only (null, null) @@ -75,8 +69,6 @@ struct -- !query 5 --- Case 4 - -- (one column null, other column matches a row in the subquery result -> row not returned) SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) @@ -90,8 +82,6 @@ struct -- !query 6 --- Case 5 - -- (one null column with no match -> row is returned) SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) @@ -105,8 +95,6 @@ NULL 1 -- !query 7 --- Case 6 - -- (no null columns with match -> row not returned) SELECT * FROM m WHERE b = 3.0 -- Matches (2, 3.0) @@ -120,8 +108,6 @@ struct -- !query 8 --- Case 7 - -- (no null columns with no match -> row is returned) SELECT * FROM m WHERE b = 5.0 -- Matches (4, 5.0) diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out index 446447e890449..cf8f03eaa9311 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -15,12 +15,6 @@ struct<> -- !query 1 --- Uncorrelated NOT IN Subquery test cases - -- Case 1 (not possible to write a literal with no rows, so we ignore it.) - -- (empty subquery -> all rows returned) - - -- Case 2 - -- (subquery includes null -> no rows returned) SELECT * FROM m WHERE a NOT IN (null) @@ -31,8 +25,6 @@ struct -- !query 2 --- Case 3 - -- (probe column is null -> row not returned) SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) @@ -44,8 +36,6 @@ struct -- !query 3 --- Case 4 - -- (probe column matches subquery row -> row not returned) SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) @@ -57,8 +47,6 @@ struct -- !query 4 --- Case 5 - -- (probe column does not match subquery row -> row is returned) SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out index f58ebeacc2872..d07981cfd11e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -27,9 +27,6 @@ struct<> -- !query 2 --- Uncorrelated NOT IN Subquery test cases - -- Case 1 - -- (empty subquery -> all rows returned) SELECT * FROM m WHERE a NOT IN (SELECT c @@ -44,8 +41,6 @@ NULL 1 -- !query 3 --- Case 2 - -- (subquery includes null -> no rows returned) SELECT * FROM m WHERE a NOT IN (SELECT c @@ -58,8 +53,6 @@ struct -- !query 4 --- Case 3 - -- (probe column is null -> row not returned) SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) @@ -73,8 +66,6 @@ struct -- !query 5 --- Case 4 - -- (probe column matches subquery row -> row not returned) SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) @@ -88,8 +79,6 @@ struct -- !query 6 --- Case 5 - -- (probe column does not match subquery row -> row is returned) SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) @@ -103,9 +92,6 @@ struct -- !query 7 --- Correlated NOT IN subquery test cases - -- Case 2->1 - -- (subquery had nulls but they are removed by correlated subquery -> all rows returned) SELECT * FROM m WHERE a NOT IN (SELECT c @@ -120,8 +106,6 @@ NULL 1 -- !query 8 --- Case 3->1 - -- (probe column is null but subquery returns no rows -> row is returned) SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) @@ -135,8 +119,6 @@ NULL 1 -- !query 9 --- Case 4->1 - -- (probe column matches row which is filtered out by correlated subquery -> row is returned) SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out index 9fe943874c3e5..ad2f1bdf77d7a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out @@ -18,14 +18,11 @@ struct<> -- !query 1 SELECT - -- boolean and transitions - -- null because strict (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, (FALSE AND NULL) IS NULL AS `t`, (NULL AND TRUE) IS NULL AS `t`, (NULL AND FALSE) IS NULL AS `t`, - -- and actual computations (TRUE AND TRUE) AS `t`, NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, @@ -38,14 +35,11 @@ true true false true false true true true true -- !query 2 SELECT - -- boolean or transitions - -- null because strict (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, (FALSE OR NULL) IS NULL AS `t`, (NULL OR TRUE) IS NULL AS `t`, (NULL OR FALSE) IS NULL AS `t`, - -- actual computations (TRUE OR TRUE) AS `t`, (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index e1747f466779f..f8e5fe6a62f33 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -3,8 +3,6 @@ -- !query 0 --- This test file was converted from udaf.sql. - CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1), (2), (3), (4) as t1(int_col1) 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 c6fc84894a3f0..4bdf25051127c 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 @@ -225,11 +225,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { private def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) - val (comments, code) = input.split("\n").partition(_.startsWith("--")) + val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) // List of SQL queries to run // note: this is not a robust way to split queries using semicolon, but works for now. val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq + // Fix misplacement when comment is at the end of the query. + .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") // When we are regenerating the golden files, we don't need to set any config as they // all need to return the same result From 6ea4a737eaf6de41deb3ca9595c84a19b1b35554 Mon Sep 17 00:00:00 2001 From: Rob Vesse Date: Wed, 7 Aug 2019 16:55:11 -0700 Subject: [PATCH 013/149] [SPARK-28649][INFRA] Add Python .eggs to .gitignore ## What changes were proposed in this pull request? If you build Spark distributions you potentially end up with a `python/.eggs` directory in your working copy which is not currently ignored by Spark's `.gitignore` file. Since these are transient build artifacts there is no reason to ever commit these to Git so this should be placed in the `.gitignore` list ## How was this patch tested? Verified the offending artifacts were no longer reported as untracked content by Git Closes #25380 from rvesse/patch-1. Authored-by: Rob Vesse Signed-off-by: Dongjoon Hyun --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index d5cf66d1db126..4b1ba1c92bc04 100644 --- a/.gitignore +++ b/.gitignore @@ -61,6 +61,7 @@ project/plugins/project/build.properties project/plugins/src_managed/ project/plugins/target/ python/lib/pyspark.zip +python/.eggs/ python/deps python/test_coverage/coverage_data python/test_coverage/htmlcov From 3586cdd24d9f5cb7d3f642a3da6a26ced1f88cea Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 8 Aug 2019 10:42:24 +0900 Subject: [PATCH 014/149] [SPARK-28395][FOLLOW-UP][SQL] Make spark.sql.function.preferIntegralDivision internal ## What changes were proposed in this pull request? This PR makes `spark.sql.function.preferIntegralDivision` to internal configuration because it is only used for PostgreSQL test cases. More details: https://github.com/apache/spark/pull/25158#discussion_r309764541 ## How was this patch tested? N/A Closes #25376 from wangyum/SPARK-28395-2. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 1 file changed, 2 insertions(+), 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 80a7d4efe4e52..f779bc835dfb6 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 @@ -1532,8 +1532,9 @@ object SQLConf { .createWithDefault(false) val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") + .internal() .doc("When true, will perform integral division with the / operator " + - "if both sides are integral types.") + "if both sides are integral types. This is for PostgreSQL test cases only.") .booleanConf .createWithDefault(false) From e58dd4af60dbb0e3378184c56ae9ded3db288852 Mon Sep 17 00:00:00 2001 From: Yishuang Lu Date: Thu, 8 Aug 2019 11:12:18 +0900 Subject: [PATCH 015/149] [MINOR][DOC] Fix a typo 'lister' -> 'listener' ## What changes were proposed in this pull request? Fix the typo in java doc. ## How was this patch tested? N/A Signed-off-by: Yishuang Lu Closes #25377 from lys0716/dev. Authored-by: Yishuang Lu Signed-off-by: HyukjinKwon --- .../sql/execution/datasources/parquet/ParquetFileFormat.scala | 4 ++-- .../v2/parquet/ParquetPartitionReaderFactory.scala | 4 ++-- .../sql/streaming/StreamingQueryListenersConfSuite.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) 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 9caa34b2d9652..815b62dfbf898 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 @@ -315,7 +315,7 @@ class ParquetFileFormat val vectorizedReader = new VectorizedParquetRecordReader( convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) val iter = new RecordReaderIterator(vectorizedReader) - // SPARK-23457 Register a task completion lister before `initialization`. + // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) vectorizedReader.initialize(split, hadoopAttemptContext) logDebug(s"Appending $partitionSchema ${file.partitionValues}") @@ -337,7 +337,7 @@ class ParquetFileFormat new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) - // SPARK-23457 Register a task completion lister before `initialization`. + // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) reader.initialize(split, hadoopAttemptContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index 4a281ba46eb5f..a0f19c3dd2eb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -197,7 +197,7 @@ case class ParquetPartitionReaderFactory( new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) - // SPARK-23457 Register a task completion lister before `initialization`. + // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) reader } @@ -219,7 +219,7 @@ case class ParquetPartitionReaderFactory( val vectorizedReader = new VectorizedParquetRecordReader( convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) val iter = new RecordReaderIterator(vectorizedReader) - // SPARK-23457 Register a task completion lister before `initialization`. + // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) logDebug(s"Appending $partitionSchema $partitionValues") vectorizedReader diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index da2f221aaf101..7801d968e901d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -33,7 +33,7 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { super.sparkConf.set(STREAMING_QUERY_LISTENERS.key, "org.apache.spark.sql.streaming.TestListener") - test("test if the configured query lister is loaded") { + test("test if the configured query listener is loaded") { testStream(MemoryStream[Int].toDS)( StartStream(), StopStream From bda5b51576e525724315d4892e34c8fa7e27f0c7 Mon Sep 17 00:00:00 2001 From: Anton Yanchenko Date: Thu, 8 Aug 2019 11:47:25 +0900 Subject: [PATCH 016/149] [SPARK-28454][PYTHON] Validate LongType in `createDataFrame(verifySchema=True)` ## What changes were proposed in this pull request? Add missing validation for `LongType` in `pyspark.sql.types._make_type_verifier`. ## How was this patch tested? Doctests / unittests / manual tests. Unpatched version: ``` In [23]: s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect() Out[23]: [Row(x=None)] ``` Patched: ``` In [5]: s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect() --------------------------------------------------------------------------- ValueError Traceback (most recent call last) in ----> 1 s.createDataFrame([{'x': 1 << 64}], StructType([StructField('x', LongType())])).collect() /usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in createDataFrame(self, data, schema, samplingRatio, verifySchema) 689 rdd, schema = self._createFromRDD(data.map(prepare), schema, samplingRatio) 690 else: --> 691 rdd, schema = self._createFromLocal(map(prepare, data), schema) 692 jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) 693 jdf = self._jsparkSession.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) /usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in _createFromLocal(self, data, schema) 405 # make sure data could consumed multiple times 406 if not isinstance(data, list): --> 407 data = list(data) 408 409 if schema is None or isinstance(schema, (list, tuple)): /usr/local/lib/python3.5/site-packages/pyspark/sql/session.py in prepare(obj) 671 672 def prepare(obj): --> 673 verify_func(obj) 674 return obj 675 elif isinstance(schema, DataType): /usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify(obj) 1427 def verify(obj): 1428 if not verify_nullability(obj): -> 1429 verify_value(obj) 1430 1431 return verify /usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify_struct(obj) 1397 if isinstance(obj, dict): 1398 for f, verifier in verifiers: -> 1399 verifier(obj.get(f)) 1400 elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): 1401 # the order in obj could be different than dataType.fields /usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify(obj) 1427 def verify(obj): 1428 if not verify_nullability(obj): -> 1429 verify_value(obj) 1430 1431 return verify /usr/local/lib/python3.5/site-packages/pyspark/sql/types.py in verify_long(obj) 1356 if obj < -9223372036854775808 or obj > 9223372036854775807: 1357 raise ValueError( -> 1358 new_msg("object of LongType out of range, got: %s" % obj)) 1359 1360 verify_value = verify_long ValueError: field x: object of LongType out of range, got: 18446744073709551616 ``` Closes #25117 from simplylizz/master. Authored-by: Anton Yanchenko Signed-off-by: HyukjinKwon --- docs/sql-migration-guide-upgrade.md | 2 ++ python/pyspark/sql/tests/test_types.py | 3 ++- python/pyspark/sql/types.py | 14 ++++++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index efd88d0eb1886..b2bd8cefc3f96 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -149,6 +149,8 @@ license: | - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. + - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 5132eec61d6f5..1cd84e0cd24e8 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -830,7 +830,8 @@ def __init__(self, **kwargs): (2**31 - 1, IntegerType()), # Long - (2**64, LongType()), + (-(2**63), LongType()), + (2**63 - 1, LongType()), # Float & Double (1.0, FloatType()), diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index da84fc1e0066c..0c7f4ce3ddc67 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1211,6 +1211,10 @@ def _make_type_verifier(dataType, nullable=True, name=None): >>> _make_type_verifier(StructType([]))(None) >>> _make_type_verifier(StringType())("") >>> _make_type_verifier(LongType())(0) + >>> _make_type_verifier(LongType())(1 << 64) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... >>> _make_type_verifier(ArrayType(ShortType()))(list(range(3))) >>> _make_type_verifier(ArrayType(StringType()))(set()) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): @@ -1319,6 +1323,16 @@ def verify_integer(obj): verify_value = verify_integer + elif isinstance(dataType, LongType): + def verify_long(obj): + assert_acceptable_types(obj) + verify_acceptable_types(obj) + if obj < -9223372036854775808 or obj > 9223372036854775807: + raise ValueError( + new_msg("object of LongType out of range, got: %s" % obj)) + + verify_value = verify_long + elif isinstance(dataType, ArrayType): element_verifier = _make_type_verifier( dataType.elementType, dataType.containsNull, name="element in array %s" % name) From c4acfe7761bde41e9d26c5ab3a670ab86165cf9b Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 8 Aug 2019 17:01:25 +0900 Subject: [PATCH 017/149] [SPARK-28474][SQL] Hive 0.12 JDBC client can not handle binary type ## What changes were proposed in this pull request? This PR fix Hive 0.12 JDBC client can not handle binary type: ```sql Connected to: Hive (version 3.0.0-SNAPSHOT) Driver: Hive (version 0.12.0) Transaction isolation: TRANSACTION_REPEATABLE_READ Beeline version 0.12.0 by Apache Hive 0: jdbc:hive2://localhost:10000> SELECT cast('ABC' as binary); Error: java.lang.ClassCastException: [B incompatible with java.lang.String (state=,code=0) ``` Server log: ``` 19/08/07 10:10:04 WARN ThriftCLIService: Error fetching results: java.lang.RuntimeException: java.lang.ClassCastException: [B incompatible with java.lang.String at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:83) at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36) at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63) at java.security.AccessController.doPrivileged(AccessController.java:770) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746) at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59) at com.sun.proxy.$Proxy26.fetchResults(Unknown Source) at org.apache.hive.service.cli.CLIService.fetchResults(CLIService.java:455) at org.apache.hive.service.cli.thrift.ThriftCLIService.FetchResults(ThriftCLIService.java:621) at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1553) at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1538) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:38) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:819) Caused by: java.lang.ClassCastException: [B incompatible with java.lang.String at org.apache.hive.service.cli.ColumnValue.toTColumnValue(ColumnValue.java:198) at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:60) at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:32) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.$anonfun$getNextRowSet$1(SparkExecuteStatementOperation.scala:151) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$Lambda$1923.000000009113BFE0.apply(Unknown Source) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withSchedulerPool(SparkExecuteStatementOperation.scala:299) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.getNextRowSet(SparkExecuteStatementOperation.scala:113) at org.apache.hive.service.cli.operation.OperationManager.getOperationNextRowSet(OperationManager.java:220) at org.apache.hive.service.cli.session.HiveSessionImpl.fetchResults(HiveSessionImpl.java:785) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78) ... 18 more ``` ## How was this patch tested? unit tests Closes #25379 from wangyum/SPARK-28474. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- .../SparkThriftServerProtocolVersionsSuite.scala | 14 ++++++++++++-- .../org/apache/hive/service/cli/ColumnValue.java | 5 ++++- .../org/apache/hive/service/cli/ColumnValue.java | 5 ++++- 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index e33e42f3cc8ca..604c21f82c197 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -26,6 +26,9 @@ import org.apache.hive.service.cli.GetInfoType import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket +import org.apache.spark.sql.catalyst.util.NumberConverter +import org.apache.spark.unsafe.types.UTF8String + class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary @@ -215,12 +218,19 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - // TODO: enable this test case after SPARK-28474 - ignore(s"$version get binary type") { + test(s"$version get binary type") { testExecuteStatementWithProtocolVersion(version, "SELECT cast('ABC' as binary)") { rs => assert(rs.next()) assert(rs.getString(1) === "ABC") } + testExecuteStatementWithProtocolVersion(version, "SELECT cast(49960 as binary)") { rs => + assert(rs.next()) + assert(rs.getString(1) === UTF8String.fromBytes(NumberConverter.toBinary(49960)).toString) + } + testExecuteStatementWithProtocolVersion(version, "SELECT cast(null as binary)") { rs => + assert(rs.next()) + assert(rs.getString(1) === null) + } } test(s"$version get boolean type") { diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java index 40144cfe33fa3..9e9d6ccc21169 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -36,6 +36,8 @@ import org.apache.hive.service.cli.thrift.TI64Value; import org.apache.hive.service.cli.thrift.TStringValue; +import org.apache.spark.unsafe.types.UTF8String; + /** * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) * @@ -195,7 +197,8 @@ public static TColumnValue toTColumnValue(Type type, Object value) { case DECIMAL_TYPE: return stringValue(((HiveDecimal)value)); case BINARY_TYPE: - return stringValue((String)value); + String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); + return stringValue(strVal); case ARRAY_TYPE: case MAP_TYPE: case STRUCT_TYPE: diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java index 28149e1e5edb5..12d49b25142a4 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -37,6 +37,8 @@ import org.apache.hive.service.rpc.thrift.TI64Value; import org.apache.hive.service.rpc.thrift.TStringValue; +import org.apache.spark.unsafe.types.UTF8String; + /** * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) * @@ -199,7 +201,8 @@ public static TColumnValue toTColumnValue(TypeDescriptor typeDescriptor, Object case DECIMAL_TYPE: return stringValue((HiveDecimal)value, typeDescriptor); case BINARY_TYPE: - return stringValue((String)value); + String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); + return stringValue(strVal); case ARRAY_TYPE: case MAP_TYPE: case STRUCT_TYPE: From 1941d35d1e250f707c0d70782e568c0887741dfa Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 8 Aug 2019 17:28:10 +0900 Subject: [PATCH 018/149] [SPARK-28644][SQL] Port HIVE-10646: ColumnValue does not handle NULL_TYPE ## What changes were proposed in this pull request? This PR port [HIVE-10646](https://issues.apache.org/jira/browse/HIVE-10646) to fix Hive 0.12's JDBC client can not handle `NULL_TYPE`: ```sql Connected to: Hive (version 3.0.0-SNAPSHOT) Driver: Hive (version 0.12.0) Transaction isolation: TRANSACTION_REPEATABLE_READ Beeline version 0.12.0 by Apache Hive 0: jdbc:hive2://localhost:10000> select null; org.apache.thrift.transport.TTransportException at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132) at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84) at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:346) at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:423) at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:405) ``` Server log: ``` 19/08/07 09:34:07 ERROR TThreadPoolServer: Error occurred during processing of message. java.lang.NullPointerException at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:388) at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:338) at org.apache.hive.service.cli.thrift.TRow.write(TRow.java:288) at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:605) at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:525) at org.apache.hive.service.cli.thrift.TRowSet.write(TRowSet.java:455) at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:550) at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:486) at org.apache.hive.service.cli.thrift.TFetchResultsResp.write(TFetchResultsResp.java:412) at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13192) at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13156) at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result.write(TCLIService.java:13107) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:58) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:819) ``` ## How was this patch tested? unit tests Closes #25378 from wangyum/SPARK-28644. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- .../thriftserver/SparkThriftServerProtocolVersionsSuite.scala | 3 +-- .../src/main/java/org/apache/hive/service/cli/ColumnValue.java | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index 604c21f82c197..9682a2a80eca9 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -255,8 +255,7 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - // TODO: enable this test case after port HIVE-10646 - ignore(s"$version get void") { + test(s"$version get void") { testExecuteStatementWithProtocolVersion(version, "SELECT null") { rs => assert(rs.next()) assert(rs.getString(1) === null) diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java index 9e9d6ccc21169..ce833695f5867 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -205,6 +205,8 @@ public static TColumnValue toTColumnValue(Type type, Object value) { case UNION_TYPE: case USER_DEFINED_TYPE: return stringValue((String)value); + case NULL_TYPE: + return stringValue((String)value); default: return null; } From 8c0dc386401060feedffb40389c37ea74e903faa Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 8 Aug 2019 20:21:07 +0800 Subject: [PATCH 019/149] [SPARK-28654][SQL] Move "Extract Python UDFs" to the last in optimizer ## What changes were proposed in this pull request? Plans after "Extract Python UDFs" are very flaky and error-prone to other rules. For instance, if we add some rules, for instance, `PushDownPredicates` in `postHocOptimizationBatches`, the test in `BatchEvalPythonExecSuite` fails: ```scala test("Python UDF refers to the attributes from more than one child") { val df = Seq(("Hello", 4)).toDF("a", "b") val df2 = Seq(("Hello", 4)).toDF("c", "d") val joinDF = df.crossJoin(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)") val qualifiedPlanNodes = joinDF.queryExecution.executedPlan.collect { case b: BatchEvalPythonExec => b } assert(qualifiedPlanNodes.size == 1) } ``` ``` Invalid PythonUDF dummyUDF(a#63, c#74), requires attributes from more than one child. ``` This is because Python UDF extraction optimization is rolled back as below: ``` === Applying Rule org.apache.spark.sql.catalyst.optimizer.PushDownPredicates === !Filter (dummyUDF(a#7, c#18) = dummyUDF(d#19, c#18)) Join Cross, (dummyUDF(a#7, c#18) = dummyUDF(d#19, c#18)) !+- Join Cross :- Project [_1#2 AS a#7, _2#3 AS b#8] ! :- Project [_1#2 AS a#7, _2#3 AS b#8] : +- LocalRelation [_1#2, _2#3] ! : +- LocalRelation [_1#2, _2#3] +- Project [_1#13 AS c#18, _2#14 AS d#19] ! +- Project [_1#13 AS c#18, _2#14 AS d#19] +- LocalRelation [_1#13, _2#14] ! +- LocalRelation [_1#13, _2#14] ``` Seems we should do Python UDFs cases at the last even after post hoc rules. Note that this actually rather follows the way in previous versions when those were in physical plans (see SPARK-24721 and SPARK-12981). Those optimization rules were supposed to be placed at the end. Note that I intentionally didn't move `ExperimentalMethods` (`spark.experimental.extraStrategies`). This is an explicit experimental API and I wanted to just-in-case workaround after this change for now. ## How was this patch tested? Existing tests should cover. Closes #25386 from HyukjinKwon/SPARK-28654. Authored-by: HyukjinKwon Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/execution/SparkOptimizer.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index a1135f7d6e6ac..98060e981ed86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -31,6 +31,9 @@ class SparkOptimizer( override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ + Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ + Batch("Schema Pruning", Once, SchemaPruning)) ++ + postHocOptimizationBatches :+ Batch("Extract Python UDFs", Once, ExtractPythonUDFFromJoinCondition, // `ExtractPythonUDFFromJoinCondition` can convert a join to a cartesian product. @@ -45,9 +48,6 @@ class SparkOptimizer( ColumnPruning, PushPredicateThroughNonJoin, RemoveNoopOperators) :+ - Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ - Batch("Schema Pruning", Once, SchemaPruning)) ++ - postHocOptimizationBatches :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ @@ -65,6 +65,8 @@ class SparkOptimizer( * Optimization batches that are executed after the regular optimization batches, but before the * batch executing the [[ExperimentalMethods]] optimizer rules. This hook can be used to add * custom optimizer batches to the Spark optimizer. + * + * Note that 'Extract Python UDFs' batch is an exception and ran after the batches defined here. */ def postHocOptimizationBatches: Seq[Batch] = Nil } From d19a56f9dbef4c995d80d4b46d03bfbfa4843c5c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 8 Aug 2019 16:24:43 -0700 Subject: [PATCH 020/149] [SPARK-28642][SQL] Hide credentials in SHOW CREATE TABLE ## What changes were proposed in this pull request? [SPARK-17783](https://issues.apache.org/jira/browse/SPARK-17783) hided Credentials in `CREATE` and `DESC FORMATTED/EXTENDED` a PERSISTENT/TEMP Table for JDBC. But `SHOW CREATE TABLE` exposed the credentials: ```sql spark-sql> show create table mysql_federated_sample; CREATE TABLE `mysql_federated_sample` (`TBL_ID` BIGINT, `CREATE_TIME` INT, `DB_ID` BIGINT, `LAST_ACCESS_TIME` INT, `OWNER` STRING, `RETENTION` INT, `SD_ID` BIGINT, `TBL_NAME` STRING, `TBL_TYPE` STRING, `VIEW_EXPANDED_TEXT` STRING, `VIEW_ORIGINAL_TEXT` STRING, `IS_REWRITE_ENABLED` BOOLEAN) USING org.apache.spark.sql.jdbc OPTIONS ( `url` 'jdbc:mysql://localhost/hive?user=root&password=mypasswd', `driver` 'com.mysql.jdbc.Driver', `dbtable` 'TBLS' ) ``` This pr fix this issue. ## How was this patch tested? unit tests and manual tests: ```sql spark-sql> show create table mysql_federated_sample; CREATE TABLE `mysql_federated_sample` (`TBL_ID` BIGINT, `CREATE_TIME` INT, `DB_ID` BIGINT, `LAST_ACCESS_TIME` INT, `OWNER` STRING, `RETENTION` INT, `SD_ID` BIGINT, `TBL_NAME` STRING, `TBL_TYPE` STRING, `VIEW_EXPANDED_TEXT` STRING, `VIEW_ORIGINAL_TEXT` STRING, `IS_REWRITE_ENABLED` BOOLEAN) USING org.apache.spark.sql.jdbc OPTIONS ( `url` '*********(redacted)', `driver` 'com.mysql.jdbc.Driver', `dbtable` 'TBLS' ) ``` Closes #25375 from wangyum/SPARK-28642. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/command/tables.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 29 ++++++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index ca42de39db86b..af1e1bc59cbbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -1099,7 +1099,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman private def showDataSourceTableOptions(metadata: CatalogTable, builder: StringBuilder): Unit = { builder ++= s"USING ${metadata.provider.get}\n" - val dataSourceOptions = metadata.storage.properties.map { + val dataSourceOptions = SQLConf.get.redactOptions(metadata.storage.properties).map { case (key, value) => s"${quoteIdentifier(key)} '${escapeSingleQuotedString(value)}'" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 89eaac8e5927c..9f7faa2baf562 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -26,10 +26,11 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeTestUtils} import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.execution.command.{ExplainCommand, ShowCreateTableCommand} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRDD, JDBCRelation, JdbcUtils} import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper @@ -1029,6 +1030,32 @@ class JDBCSuite extends QueryTest } } + test("Hide credentials in show create table") { + val password = "testPass" + val tableName = "tab1" + withTable(tableName) { + sql( + s""" + |CREATE TABLE $tableName + |USING org.apache.spark.sql.jdbc + |OPTIONS ( + | url '$urlWithUserAndPass', + | dbtable 'TEST.PEOPLE', + | user 'testUser', + | password '$password') + """.stripMargin) + + val show = ShowCreateTableCommand(TableIdentifier(tableName)) + spark.sessionState.executePlan(show).executedPlan.executeCollect().foreach { r => + assert(!r.toString.contains(password)) + } + + sql(s"SHOW CREATE TABLE $tableName").collect().foreach { r => + assert(!r.toString().contains(password)) + } + } + } + test("SPARK 12941: The data type mapping for StringType to Oracle") { val oracleDialect = JdbcDialects.get("jdbc:oracle://127.0.0.1/db") assert(oracleDialect.getJDBCType(StringType). From 2580c1bfe2a4bc61ea976cb9c06275a9c2604b99 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 8 Aug 2019 16:39:32 -0700 Subject: [PATCH 021/149] [SPARK-28660][SQL][TEST] Port AGGREGATES.sql [Part 4] ## What changes were proposed in this pull request? This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/aggregates.out#L1615-L2289 When porting the test cases, found five PostgreSQL specific features that do not exist in Spark SQL: [SPARK-27980](https://issues.apache.org/jira/browse/SPARK-27980): Ordered-Set Aggregate Functions [SPARK-28661](https://issues.apache.org/jira/browse/SPARK-28661): Hypothetical-Set Aggregate Functions [SPARK-28382](https://issues.apache.org/jira/browse/SPARK-28382): Array Functions: unnest [SPARK-28663](https://issues.apache.org/jira/browse/SPARK-28663): Aggregate Functions for Statistics [SPARK-28664](https://issues.apache.org/jira/browse/SPARK-28664): ORDER BY in aggregate function [SPARK-28669](https://issues.apache.org/jira/browse/SPARK-28669): System Information Functions ## How was this patch tested? N/A Closes #25392 from wangyum/SPARK-28660. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../inputs/pgSQL/aggregates_part4.sql | 419 ++++++++++++++++++ .../results/pgSQL/aggregates_part4.sql.out | 5 + 2 files changed, 424 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql new file mode 100644 index 0000000000000..6fa2306cf1475 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part4.sql @@ -0,0 +1,419 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 4] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- ordered-set aggregates + +-- select p, percentile_cont(p) within group (order by x::float8) +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p order by p) within group (order by x) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, sum() within group (order by x::float8) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p,p) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select percentile_cont(0.5) within group (order by b) from aggtest; +-- select percentile_cont(0.5) within group (order by b), sum(b) from aggtest; +-- select percentile_cont(0.5) within group (order by thousand) from tenk1; +-- select percentile_disc(0.5) within group (order by thousand) from tenk1; +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- select rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select cume_dist(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select percent_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4),(5)) v(x); +-- select dense_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- select percentile_disc(array[0,0.1,0.25,0.5,0.75,0.9,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,0.25,0.5,0.75,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_disc(array[[null,1,0.5],[0.75,0.25,null]]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,1,0.25,0.75,0.5,1,0.3,0.32,0.35,0.38,0.4]) within group (order by x) +-- from generate_series(1,6) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- [SPARK-28382] Array Functions: unnest +-- select ten, mode() within group (order by string4) from tenk1 group by ten; + +-- select percentile_disc(array[0.25,0.5,0.75]) within group (order by x) +-- from unnest('{fred,jim,fred,jack,jill,fred,jill,jim,jim,sheila,jim,sheila}'::text[]) u(x); + +-- [SPARK-28669] System Information Functions +-- check collation propagates up in suitable cases: +-- select pg_collation_for(percentile_disc(1) within group (order by x collate "POSIX")) +-- from (values ('fred'),('jim')) v(x); + +-- test_rank and test_percentile_disc function created by create_aggregate.sql +-- ordered-set aggs created with CREATE AGGREGATE +-- select test_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select test_percentile_disc(0.5) within group (order by thousand) from tenk1; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- ordered-set aggs can't use ungrouped vars in direct args: +-- select rank(x) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- outer-level agg can't use a grouped arg of a lower level, either: +-- select array(select percentile_disc(a) within group (order by x) +-- from (values (0.3),(0.7)) v(a) group by a) +-- from generate_series(1,5) g(x); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- agg in the direct args is a grouping violation, too: +--select rank(sum(x)) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- hypothetical-set type unification and argument-count failures: +-- select rank(3) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank(3) within group (order by stringu1,stringu2) from tenk1; +-- select rank('fred') within group (order by x) from generate_series(1,5) x; +-- select rank('adam'::text collate "C") within group (order by x collate "POSIX") +-- from (values ('fred'),('jim')) v(x); +-- hypothetical-set type unification successes: +-- select rank('adam'::varchar) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank('3') within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- divide by zero check +-- select percent_rank(0) within group (order by x) from generate_series(1,0) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- deparse and multiple features: +-- create view aggordview1 as +-- select ten, +-- percentile_disc(0.5) within group (order by thousand) as p50, +-- percentile_disc(0.5) within group (order by thousand) filter (where hundred=1) as px, +-- rank(5,'AZZZZ',50) within group (order by hundred, string4 desc, hundred) +-- from tenk1 +-- group by ten order by ten; + +-- select pg_get_viewdef('aggordview1'); +-- select * from aggordview1 order by ten; +-- drop view aggordview1; + +-- least_agg created by create_aggregate.sql +-- variadic aggregates +-- select least_agg(q1,q2) from int8_tbl; +-- select least_agg(variadic array[q1,q2]) from int8_tbl; + + +-- Skip these tests because we do not support create type +-- test aggregates with common transition functions share the same states +-- begin work; + +-- create type avg_state as (total bigint, count bigint); + +-- create or replace function avg_transfn(state avg_state, n int) returns avg_state as +-- $$ +-- declare new_state avg_state; +-- begin +-- raise notice 'avg_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state.total := n; +-- new_state.count := 1; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state.total := state.total + n; +-- state.count := state.count + 1; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function avg_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total / state.count; +-- end if; +-- end +-- $$ language plpgsql; + +-- create function sum_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_avg(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn +-- ); +-- +-- create aggregate my_sum(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn +-- ); + +-- aggregate state should be shared as aggs are the same. +-- select my_avg(one),my_avg(one) from (values(1),(3)) t(one); + +-- aggregate state should be shared as transfn is the same for both aggs. +-- select my_avg(one),my_sum(one) from (values(1),(3)) t(one); + +-- same as previous one, but with DISTINCT, which requires sorting the input. +-- select my_avg(distinct one),my_sum(distinct one) from (values(1),(3),(1)) t(one); + +-- shouldn't share states due to the distinctness not matching. +-- select my_avg(distinct one),my_sum(one) from (values(1),(3)) t(one); + +-- shouldn't share states due to the filter clause not matching. +-- select my_avg(one) filter (where one > 1),my_sum(one) from (values(1),(3)) t(one); + +-- this should not share the state due to different input columns. +-- select my_avg(one),my_sum(two) from (values(1,2),(3,4)) t(one,two); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- exercise cases where OSAs share state +-- select +-- percentile_cont(0.5) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- select +-- percentile_cont(0.25) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- these can't share state currently +-- select +-- rank(4) within group (order by a), +-- dense_rank(4) within group (order by a) +-- from (values(1),(3),(5),(7)) t(a); + +-- test that aggs with the same sfunc and initcond share the same agg state +-- create aggregate my_sum_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init2(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(4,0)' +-- ); + +-- state should be shared if INITCONDs are matching +-- select my_sum_init(one),my_avg_init(one) from (values(1),(3)) t(one); + +-- Varying INITCONDs should cause the states not to be shared. +-- select my_sum_init(one),my_avg_init2(one) from (values(1),(3)) t(one); + +-- rollback; + +-- test aggregate state sharing to ensure it works if one aggregate has a +-- finalfn and the other one has none. +-- begin work; + +-- create or replace function sum_transfn(state int4, n int4) returns int4 as +-- $$ +-- declare new_state int4; +-- begin +-- raise notice 'sum_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state := n; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state := state + n; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function halfsum_finalfn(state int4) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state / 2; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn +-- ); + +-- create aggregate my_half_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn, +-- finalfunc = halfsum_finalfn +-- ); + +-- Agg state should be shared even though my_sum has no finalfn +-- select my_sum(one),my_half_sum(one) from (values(1),(2),(3),(4)) t(one); + +-- rollback; + + +-- test that the aggregate transition logic correctly handles +-- transition / combine functions returning NULL + +-- First test the case of a normal transition function returning NULL +-- BEGIN; +-- CREATE FUNCTION balkifnull(int8, int4) +-- RETURNS int8 +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = balkifnull(int8, int4), +-- STYPE = int8, +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- Secondly test the case of a parallel aggregate combiner function +-- returning NULL. For that use normal transition function, but a +-- combiner function returning NULL. +-- BEGIN ISOLATION LEVEL REPEATABLE READ; +-- CREATE FUNCTION balkifnull(int8, int8) +-- RETURNS int8 +-- PARALLEL SAFE +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = int4_sum(int8, int4), +-- STYPE = int8, +-- COMBINEFUNC = balkifnull(int8, int8), +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- force use of parallelism +-- ALTER TABLE tenk1 set (parallel_workers = 4); +-- SET LOCAL parallel_setup_cost=0; +-- SET LOCAL max_parallel_workers_per_gather=4; + +-- EXPLAIN (COSTS OFF) SELECT balk(hundred) FROM tenk1; +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- test coverage for aggregate combine/serial/deserial functions +-- BEGIN ISOLATION LEVEL REPEATABLE READ; + +-- SET parallel_setup_cost = 0; +-- SET parallel_tuple_cost = 0; +-- SET min_parallel_table_scan_size = 0; +-- SET max_parallel_workers_per_gather = 4; +-- SET enable_indexonlyscan = off; + +-- [SPARK-28663] Aggregate Functions for Statistics +-- variance(int4) covers numeric_poly_combine +-- sum(int8) covers int8_avg_combine +-- regr_count(float8, float8) covers int8inc_float8_float8 and aggregates with > 1 arg +-- EXPLAIN (COSTS OFF, VERBOSE) +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- ROLLBACK; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- test coverage for dense_rank +-- SELECT dense_rank(x) WITHIN GROUP (ORDER BY x) FROM (VALUES (1),(1),(2),(2),(3),(3)) v(x) GROUP BY (x) ORDER BY 1; + + +-- [SPARK-28664] ORDER BY in aggregate function +-- Ensure that the STRICT checks for aggregates does not take NULLness +-- of ORDER BY columns into account. See bug report around +-- 2a505161-2727-2473-7c46-591ed108ac52@email.cz +-- SELECT min(x ORDER BY y) FROM (VALUES(1, NULL)) AS d(x,y); +-- SELECT min(x ORDER BY y) FROM (VALUES(1, 2)) AS d(x,y); + +-- [SPARK-28382] Array Functions: unnest +-- check collation-sensitive matching between grouping expressions +-- select v||'a', case v||'a' when 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; +-- select v||'a', case when v||'a' = 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; + +-- Make sure that generation of HashAggregate for uniqification purposes +-- does not lead to array overflow due to unexpected duplicate hash keys +-- see CAFeeJoKKu0u+A_A9R9316djW-YW3-+Gtgvy3ju655qRHR3jtdA@mail.gmail.com +-- explain (costs off) +-- select 1 from tenk1 +-- where (hundred, thousand) in (select twothousand, twothousand from onek); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out new file mode 100644 index 0000000000000..b7bbdc50dfb41 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part4.sql.out @@ -0,0 +1,5 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 0 + + + From c80430f5c9189b37ac1209db0453dbd9bb5c767e Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 9 Aug 2019 12:04:28 +0800 Subject: [PATCH 022/149] [SPARK-28572][SQL] Simple analyzer checks for v2 table creation code paths ## What changes were proposed in this pull request? Adds checks around: - The existence of transforms in the table schema (even in nested fields) - Duplications of transforms - Case sensitivity checks around column names in the V2 table creation code paths. ## How was this patch tested? Unit tests. Closes #25305 from brkyvz/v2CreateTable. Authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../catalog/v2/expressions/expressions.scala | 39 +++- .../sql/catalyst/analysis/CheckAnalysis.scala | 7 +- .../plans/logical/basicLogicalOperators.scala | 39 +++- .../apache/spark/sql/util/SchemaUtils.scala | 155 ++++++++++++++- .../datasources/DataSourceResolution.scala | 7 +- .../sql/execution/datasources/rules.scala | 43 ++++- .../datasources/v2/V2SessionCatalog.scala | 5 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 182 +++++++++++++++++- 8 files changed, 452 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala index ea5fc05dd5ff3..bceea147dddd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala @@ -59,10 +59,18 @@ private[sql] object LogicalExpressions { def hours(column: String): HoursTransform = HoursTransform(reference(column)) } +/** + * Allows Spark to rewrite the given references of the transform during analysis. + */ +sealed trait RewritableTransform extends Transform { + /** Creates a copy of this transform with the new analyzed references. */ + def withReferences(newReferences: Seq[NamedReference]): Transform +} + /** * Base class for simple transforms of a single column. */ -private[sql] abstract class SingleColumnTransform(ref: NamedReference) extends Transform { +private[sql] abstract class SingleColumnTransform(ref: NamedReference) extends RewritableTransform { def reference: NamedReference = ref @@ -73,18 +81,24 @@ private[sql] abstract class SingleColumnTransform(ref: NamedReference) extends T override def describe: String = name + "(" + reference.describe + ")" override def toString: String = describe + + protected def withNewRef(ref: NamedReference): Transform + + override def withReferences(newReferences: Seq[NamedReference]): Transform = { + assert(newReferences.length == 1, + s"Tried rewriting a single column transform (${this}) with multiple references.") + withNewRef(newReferences.head) + } } private[sql] final case class BucketTransform( numBuckets: Literal[Int], - columns: Seq[NamedReference]) extends Transform { + columns: Seq[NamedReference]) extends RewritableTransform { override val name: String = "bucket" override def references: Array[NamedReference] = { - arguments - .filter(_.isInstanceOf[NamedReference]) - .map(_.asInstanceOf[NamedReference]) + arguments.collect { case named: NamedReference => named } } override def arguments: Array[Expression] = numBuckets +: columns.toArray @@ -92,6 +106,10 @@ private[sql] final case class BucketTransform( override def describe: String = s"bucket(${arguments.map(_.describe).mkString(", ")})" override def toString: String = describe + + override def withReferences(newReferences: Seq[NamedReference]): Transform = { + this.copy(columns = newReferences) + } } private[sql] object BucketTransform { @@ -112,9 +130,7 @@ private[sql] final case class ApplyTransform( override def arguments: Array[Expression] = args.toArray override def references: Array[NamedReference] = { - arguments - .filter(_.isInstanceOf[NamedReference]) - .map(_.asInstanceOf[NamedReference]) + arguments.collect { case named: NamedReference => named } } override def describe: String = s"$name(${arguments.map(_.describe).mkString(", ")})" @@ -143,7 +159,7 @@ private object Ref { /** * Convenience extractor for any Transform. */ -private object NamedTransform { +private[sql] object NamedTransform { def unapply(transform: Transform): Some[(String, Seq[Expression])] = { Some((transform.name, transform.arguments)) } @@ -153,6 +169,7 @@ private[sql] final case class IdentityTransform( ref: NamedReference) extends SingleColumnTransform(ref) { override val name: String = "identity" override def describe: String = ref.describe + override protected def withNewRef(ref: NamedReference): Transform = this.copy(ref) } private[sql] object IdentityTransform { @@ -167,6 +184,7 @@ private[sql] object IdentityTransform { private[sql] final case class YearsTransform( ref: NamedReference) extends SingleColumnTransform(ref) { override val name: String = "years" + override protected def withNewRef(ref: NamedReference): Transform = this.copy(ref) } private[sql] object YearsTransform { @@ -181,6 +199,7 @@ private[sql] object YearsTransform { private[sql] final case class MonthsTransform( ref: NamedReference) extends SingleColumnTransform(ref) { override val name: String = "months" + override protected def withNewRef(ref: NamedReference): Transform = this.copy(ref) } private[sql] object MonthsTransform { @@ -195,6 +214,7 @@ private[sql] object MonthsTransform { private[sql] final case class DaysTransform( ref: NamedReference) extends SingleColumnTransform(ref) { override val name: String = "days" + override protected def withNewRef(ref: NamedReference): Transform = this.copy(ref) } private[sql] object DaysTransform { @@ -209,6 +229,7 @@ private[sql] object DaysTransform { private[sql] final case class HoursTransform( ref: NamedReference) extends SingleColumnTransform(ref) { override val name: String = "hours" + override protected def withNewRef(ref: NamedReference): Transform = this.copy(ref) } private[sql] object HoursTransform { 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 ae19d02e4475e..519c558d12770 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils /** * Throws user facing errors when passed invalid queries that fail to analyze. @@ -299,10 +300,10 @@ trait CheckAnalysis extends PredicateHelper { } } - case CreateTableAsSelect(_, _, partitioning, query, _, _, _) => - val references = partitioning.flatMap(_.references).toSet + case create: V2CreateTablePlan => + val references = create.partitioning.flatMap(_.references).toSet val badReferences = references.map(_.fieldNames).flatMap { column => - query.schema.findNestedField(column) match { + create.tableSchema.findNestedField(column) match { case Some(_) => None case _ => 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 6f33944fc1cf4..d9c370af47fb8 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 @@ -418,7 +418,11 @@ case class CreateV2Table( tableSchema: StructType, partitioning: Seq[Transform], properties: Map[String, String], - ignoreIfExists: Boolean) extends Command + ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} /** * Create a new table from a select query with a v2 catalog. @@ -430,8 +434,9 @@ case class CreateTableAsSelect( query: LogicalPlan, properties: Map[String, String], writeOptions: Map[String, String], - ignoreIfExists: Boolean) extends Command { + ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { + override def tableSchema: StructType = query.schema override def children: Seq[LogicalPlan] = Seq(query) override lazy val resolved: Boolean = childrenResolved && { @@ -440,6 +445,10 @@ case class CreateTableAsSelect( val references = partitioning.flatMap(_.references).toSet references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } } /** @@ -456,7 +465,11 @@ case class ReplaceTable( tableSchema: StructType, partitioning: Seq[Transform], properties: Map[String, String], - orCreate: Boolean) extends Command + orCreate: Boolean) extends Command with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} /** * Replaces a table from a select query with a v2 catalog. @@ -471,8 +484,9 @@ case class ReplaceTableAsSelect( query: LogicalPlan, properties: Map[String, String], writeOptions: Map[String, String], - orCreate: Boolean) extends Command { + orCreate: Boolean) extends Command with V2CreateTablePlan { + override def tableSchema: StructType = query.schema override def children: Seq[LogicalPlan] = Seq(query) override lazy val resolved: Boolean = { @@ -481,6 +495,10 @@ case class ReplaceTableAsSelect( val references = partitioning.flatMap(_.references).toSet references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } } /** @@ -1201,3 +1219,16 @@ case class Deduplicate( override def output: Seq[Attribute] = child.output } + +/** A trait used for logical plan nodes that create or replace V2 table definitions. */ +trait V2CreateTablePlan extends LogicalPlan { + def tableName: Identifier + def partitioning: Seq[Transform] + def tableSchema: StructType + + /** + * Creates a copy of this node with the new partitoning transforms. This method is used to + * rewrite the partition transforms normalized according to the table schema. + */ + def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan +} 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 052014ab86744..d15440632f324 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 @@ -17,9 +17,12 @@ package org.apache.spark.sql.util +import java.util.Locale + import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} /** @@ -88,4 +91,154 @@ private[spark] object SchemaUtils { s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") } } + + /** + * Returns all column names in this schema as a flat list. For example, a schema like: + * | - a + * | | - 1 + * | | - 2 + * | - b + * | - c + * | | - nest + * | | - 3 + * will get flattened to: "a", "a.1", "a.2", "b", "c", "c.nest", "c.nest.3" + */ + def explodeNestedFieldNames(schema: StructType): Seq[String] = { + def explode(schema: StructType): Seq[Seq[String]] = { + def recurseIntoComplexTypes(complexType: DataType): Seq[Seq[String]] = { + complexType match { + case s: StructType => explode(s) + case a: ArrayType => recurseIntoComplexTypes(a.elementType) + case m: MapType => + recurseIntoComplexTypes(m.keyType).map(Seq("key") ++ _) ++ + recurseIntoComplexTypes(m.valueType).map(Seq("value") ++ _) + case _ => Nil + } + } + + schema.flatMap { + case StructField(name, s: StructType, _, _) => + Seq(Seq(name)) ++ explode(s).map(nested => Seq(name) ++ nested) + case StructField(name, a: ArrayType, _, _) => + Seq(Seq(name)) ++ recurseIntoComplexTypes(a).map(nested => Seq(name) ++ nested) + case StructField(name, m: MapType, _, _) => + Seq(Seq(name)) ++ recurseIntoComplexTypes(m).map(nested => Seq(name) ++ nested) + case f => Seq(f.name) :: Nil + } + } + + explode(schema).map(UnresolvedAttribute.apply(_).name) + } + + /** + * Checks if the partitioning transforms are being duplicated or not. Throws an exception if + * duplication exists. + * + * @param transforms the schema to check for duplicates + * @param checkType contextual information around the check, used in an exception message + * @param isCaseSensitive Whether to be case sensitive when comparing column names + */ + def checkTransformDuplication( + transforms: Seq[Transform], + checkType: String, + isCaseSensitive: Boolean): Unit = { + val extractedTransforms = transforms.map { + case b: BucketTransform => + val colNames = b.columns.map(c => UnresolvedAttribute(c.fieldNames()).name) + // We need to check that we're not duplicating columns within our bucketing transform + checkColumnNameDuplication(colNames, "in the bucket definition", isCaseSensitive) + b.name -> colNames + case NamedTransform(transformName, refs) => + val fieldNameParts = + refs.collect { case FieldReference(parts) => UnresolvedAttribute(parts).name } + // We could also check that we're not duplicating column names here as well if + // fieldNameParts.length > 1, but we're specifically not, because certain transforms can + // be defined where this is a legitimate use case. + transformName -> fieldNameParts + } + val normalizedTransforms = if (isCaseSensitive) { + extractedTransforms + } else { + extractedTransforms.map(t => t._1 -> t._2.map(_.toLowerCase(Locale.ROOT))) + } + + if (normalizedTransforms.distinct.length != normalizedTransforms.length) { + val duplicateColumns = normalizedTransforms.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"${x._2.mkString(".")}" + } + throw new AnalysisException( + s"Found duplicate column(s) $checkType: ${duplicateColumns.mkString(", ")}") + } + } + + /** + * Returns the given column's ordinal within the given `schema`. The length of the returned + * position will be as long as how nested the column is. + * + * @param column The column to search for in the given struct. If the length of `column` is + * greater than 1, we expect to enter a nested field. + * @param schema The current struct we are looking at. + * @param resolver The resolver to find the column. + */ + def findColumnPosition( + column: Seq[String], + schema: StructType, + resolver: Resolver): Seq[Int] = { + def find(column: Seq[String], schema: StructType, stack: Seq[String]): Seq[Int] = { + if (column.isEmpty) return Nil + val thisCol = column.head + lazy val columnPath = UnresolvedAttribute(stack :+ thisCol).name + val pos = schema.indexWhere(f => resolver(f.name, thisCol)) + if (pos == -1) { + throw new IndexOutOfBoundsException(columnPath) + } + val children = schema(pos).dataType match { + case s: StructType => + find(column.tail, s, stack :+ thisCol) + case ArrayType(s: StructType, _) => + find(column.tail, s, stack :+ thisCol) + case o => + if (column.length > 1) { + throw new AnalysisException( + s"""Expected $columnPath to be a nested data type, but found $o. Was looking for the + |index of ${UnresolvedAttribute(column).name} in a nested field + """.stripMargin) + } + Nil + } + Seq(pos) ++ children + } + + try { + find(column, schema, Nil) + } catch { + case i: IndexOutOfBoundsException => + throw new AnalysisException( + s"Couldn't find column ${i.getMessage} in:\n${schema.treeString}") + case e: AnalysisException => + throw new AnalysisException(e.getMessage + s":\n${schema.treeString}") + } + } + + /** + * Gets the name of the column in the given position. + */ + def getColumnName(position: Seq[Int], schema: StructType): Seq[String] = { + val topLevel = schema(position.head) + val field = position.tail.foldLeft(Seq(topLevel.name) -> topLevel) { + case (nameAndField, pos) => + nameAndField._2.dataType match { + case s: StructType => + val nowField = s(pos) + (nameAndField._1 :+ nowField.name) -> nowField + case ArrayType(s: StructType, _) => + val nowField = s(pos) + (nameAndField._1 :+ nowField.name) -> nowField + case _ => + throw new AnalysisException( + s"The positions provided ($pos) cannot be resolved in\n${schema.treeString}.") + } + } + field._1 + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index f17b31da57312..a150a049f33e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.CastSupport +import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSour import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} +import org.apache.spark.sql.util.SchemaUtils case class DataSourceResolution( conf: SQLConf, @@ -123,7 +124,7 @@ case class DataSourceResolution( case replace: ReplaceTableStatement => // the provider was not a v1 source, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = replace.tableName - val catalog = maybeCatalog.orElse(defaultCatalog) + val catalog = maybeCatalog.orElse(sessionCatalog) .getOrElse(throw new AnalysisException( s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) .asTableCatalog @@ -132,7 +133,7 @@ case class DataSourceResolution( case rtas: ReplaceTableAsSelectStatement => // the provider was not a v1 source, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = rtas.tableName - val catalog = maybeCatalog.orElse(defaultCatalog) + val catalog = maybeCatalog.orElse(sessionCatalog) .getOrElse(throw new AnalysisException( s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) .asTableCatalog 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 e8951bc8e7164..a0a90503caf7b 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} @@ -29,7 +30,7 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation -import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.types.{ArrayType, AtomicType, StructField, StructType} import org.apache.spark.sql.util.SchemaUtils /** @@ -236,6 +237,46 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi c.copy(tableDesc = normalizedTable.copy(schema = reorderedSchema)) } + + case create: V2CreateTablePlan => + val schema = create.tableSchema + val partitioning = create.partitioning + val identifier = create.tableName + val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + // Check that columns are not duplicated in the schema + val flattenedSchema = SchemaUtils.explodeNestedFieldNames(schema) + SchemaUtils.checkColumnNameDuplication( + flattenedSchema, + s"in the table definition of $identifier", + isCaseSensitive) + + // Check that columns are not duplicated in the partitioning statement + SchemaUtils.checkTransformDuplication( + partitioning, "in the partitioning", isCaseSensitive) + + if (schema.isEmpty) { + if (partitioning.nonEmpty) { + throw new AnalysisException("It is not allowed to specify partitioning when the " + + "table schema is not defined.") + } + + create + } else { + // Resolve and normalize partition columns as necessary + val resolver = sparkSession.sessionState.conf.resolver + val normalizedPartitions = partitioning.map { + case transform: RewritableTransform => + val rewritten = transform.references().map { ref => + // Throws an exception if the reference cannot be resolved + val position = SchemaUtils.findColumnPosition(ref.fieldNames(), schema, resolver) + FieldReference(SchemaUtils.getColumnName(position, schema)) + } + transform.withReferences(rewritten) + case other => other + } + + create.withPartitioning(normalizedPartitions) + } } private def fallBackV2ToV1(cls: Class[_]): Class[_] = cls.newInstance match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 4cd0346b57e7e..a3b8f28fc5c39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util -import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable @@ -90,10 +89,11 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { val location = Option(properties.get("location")) val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) .copy(locationUri = location.map(CatalogUtils.stringToURI)) + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED val tableDesc = CatalogTable( identifier = ident.asTableIdentifier, - tableType = CatalogTableType.MANAGED, + tableType = tableType, storage = storage, schema = schema, provider = Some(provider), @@ -252,4 +252,3 @@ private[sql] object V2SessionCatalog { (identityCols, bucketSpec) } } - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index d95021077f9c0..9ae51d577b562 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -23,19 +23,22 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ private val orc2 = classOf[OrcDataSourceV2].getName + private val v2Source = classOf[FakeV2Provider].getName before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) @@ -1696,4 +1699,181 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } } } + + test("tableCreation: partition column case insensitive resolution") { + val testCatalog = spark.catalog("testcat").asTableCatalog + val sessionCatalog = spark.catalog("session").asTableCatalog + + def checkPartitioning(cat: TableCatalog, partition: String): Unit = { + val table = cat.loadTable(Identifier.of(Array.empty, "tbl")) + val partitions = table.partitioning().map(_.references()) + assert(partitions.length === 1) + val fieldNames = partitions.flatMap(_.map(_.fieldNames())) + assert(fieldNames === Array(Array(partition))) + } + + sql(s"CREATE TABLE tbl (a int, b string) USING $v2Source PARTITIONED BY (A)") + checkPartitioning(sessionCatalog, "a") + sql(s"CREATE TABLE testcat.tbl (a int, b string) USING $v2Source PARTITIONED BY (A)") + checkPartitioning(testCatalog, "a") + sql(s"CREATE OR REPLACE TABLE tbl (a int, b string) USING $v2Source PARTITIONED BY (B)") + checkPartitioning(sessionCatalog, "b") + sql(s"CREATE OR REPLACE TABLE testcat.tbl (a int, b string) USING $v2Source PARTITIONED BY (B)") + checkPartitioning(testCatalog, "b") + } + + test("tableCreation: partition column case sensitive resolution") { + def checkFailure(statement: String): Unit = { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val e = intercept[AnalysisException] { + sql(statement) + } + assert(e.getMessage.contains("Couldn't find column")) + } + } + + checkFailure(s"CREATE TABLE tbl (a int, b string) USING $v2Source PARTITIONED BY (A)") + checkFailure(s"CREATE TABLE testcat.tbl (a int, b string) USING $v2Source PARTITIONED BY (A)") + checkFailure( + s"CREATE OR REPLACE TABLE tbl (a int, b string) USING $v2Source PARTITIONED BY (B)") + checkFailure( + s"CREATE OR REPLACE TABLE testcat.tbl (a int, b string) USING $v2Source PARTITIONED BY (B)") + } + + test("tableCreation: duplicate column names in the table definition") { + val errorMsg = "Found duplicate column(s) in the table definition of `t`" + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + testCreateAnalysisError( + s"CREATE TABLE t ($c0 INT, $c1 INT) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE t ($c0 INT, $c1 INT) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE testcat.t ($c0 INT, $c1 INT) USING $v2Source", + errorMsg + ) + } + } + } + + test("tableCreation: duplicate nested column names in the table definition") { + val errorMsg = "Found duplicate column(s) in the table definition of `t`" + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + testCreateAnalysisError( + s"CREATE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE t (d struct<$c0: INT, $c1: INT>) USING $v2Source", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE testcat.t (d struct<$c0: INT, $c1: INT>) USING $v2Source", + errorMsg + ) + } + } + } + + test("tableCreation: bucket column names not in table definition") { + val errorMsg = "Couldn't find column c in" + testCreateAnalysisError( + s"CREATE TABLE tbl (a int, b string) USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE TABLE testcat.tbl (a int, b string) USING $v2Source CLUSTERED BY (c) INTO 4 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE tbl (a int, b string) USING $v2Source " + + "CLUSTERED BY (c) INTO 4 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE testcat.tbl (a int, b string) USING $v2Source " + + "CLUSTERED BY (c) INTO 4 BUCKETS", + errorMsg + ) + } + + test("tableCreation: column repeated in partition columns") { + val errorMsg = "Found duplicate column(s) in the partitioning" + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + testCreateAnalysisError( + s"CREATE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", + errorMsg + ) + testCreateAnalysisError( + s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) USING $v2Source PARTITIONED BY ($c0, $c1)", + errorMsg + ) + } + } + } + + test("tableCreation: column repeated in bucket columns") { + val errorMsg = "Found duplicate column(s) in the bucket definition" + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + testCreateAnalysisError( + s"CREATE TABLE t ($c0 INT) USING $v2Source " + + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE TABLE testcat.t ($c0 INT) USING $v2Source " + + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE t ($c0 INT) USING $v2Source " + + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", + errorMsg + ) + testCreateAnalysisError( + s"CREATE OR REPLACE TABLE testcat.t ($c0 INT) USING $v2Source " + + s"CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS", + errorMsg + ) + } + } + } + + private def testCreateAnalysisError(sqlStatement: String, expectedError: String): Unit = { + val errMsg = intercept[AnalysisException] { + sql(sqlStatement) + }.getMessage + assert(errMsg.contains(expectedError)) + } +} + + +/** Used as a V2 DataSource for V2SessionCatalog DDL */ +class FakeV2Provider extends TableProvider { + override def getTable(options: CaseInsensitiveStringMap): Table = { + throw new UnsupportedOperationException("Unnecessary for DDL tests") + } } From 997d153e54e3034f469191f69118897cad3e9d46 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 9 Aug 2019 12:29:44 +0800 Subject: [PATCH 023/149] [SPARK-28017][SQL] Support additional levels of truncations by DATE_TRUNC/TRUNC ## What changes were proposed in this pull request? I propose new levels of truncations for the `date_trunc()` and `trunc()` functions: 1. `MICROSECOND` and `MILLISECOND` truncate values of the `TIMESTAMP` type to microsecond and millisecond precision. 2. `DECADE`, `CENTURY` and `MILLENNIUM` truncate dates/timestamps to lowest date of current decade/century/millennium. Also the `WEEK` and `QUARTER` levels have been supported by the `trunc()` function. The function is implemented similarly to `date_trunc` in PostgreSQL: https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-TRUNC to maintain feature parity with it. Here are examples of `TRUNC`: ```sql spark-sql> SELECT TRUNC('2015-10-27', 'DECADE'); 2010-01-01 spark-sql> set spark.sql.datetime.java8API.enabled=true; spark.sql.datetime.java8API.enabled true spark-sql> SELECT TRUNC('1999-10-27', 'millennium'); 1001-01-01 ``` Examples of `DATE_TRUNC`: ```sql spark-sql> SELECT DATE_TRUNC('CENTURY', '2015-03-05T09:32:05.123456'); 2001-01-01T00:00:00Z ``` ## How was this patch tested? Added new tests to `DateTimeUtilsSuite`, `DateExpressionsSuite` and `DateFunctionsSuite`, and uncommented existing tests in `pgSQL/date.sql`. Closes #25336 from MaxGekk/date_truct-ext. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 46 ++++--- .../sql/catalyst/util/DateTimeUtils.scala | 118 ++++++++++-------- .../expressions/DateExpressionsSuite.scala | 8 ++ .../catalyst/util/DateTimeUtilsSuite.scala | 12 +- .../resources/sql-tests/inputs/pgSQL/date.sql | 23 ++-- .../sql-tests/results/pgSQL/date.sql.out | 116 ++++++++++++++--- .../apache/spark/sql/DateFunctionsSuite.scala | 25 +++- 7 files changed, 249 insertions(+), 99 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index e50abebe57987..43cef22da8341 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1393,18 +1393,18 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes { /** * @param input internalRow (time) - * @param maxLevel Maximum level that can be used for truncation (e.g MONTH for Date input) + * @param minLevel Minimum level that can be used for truncation (e.g WEEK for Date input) * @param truncFunc function: (time, level) => time */ - protected def evalHelper(input: InternalRow, maxLevel: Int)( + protected def evalHelper(input: InternalRow, minLevel: Int)( truncFunc: (Any, Int) => Any): Any = { val level = if (format.foldable) { truncLevel } else { DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String]) } - if (level == DateTimeUtils.TRUNC_INVALID || level > maxLevel) { - // unknown format or too large level + if (level < minLevel) { + // unknown format or too small level null } else { val t = instant.eval(input) @@ -1419,7 +1419,7 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes { protected def codeGenHelper( ctx: CodegenContext, ev: ExprCode, - maxLevel: Int, + minLevel: Int, orderReversed: Boolean = false)( truncFunc: (String, String) => String) : ExprCode = { @@ -1427,7 +1427,7 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes { val javaType = CodeGenerator.javaType(dataType) if (format.foldable) { - if (truncLevel == DateTimeUtils.TRUNC_INVALID || truncLevel > maxLevel) { + if (truncLevel < minLevel) { ev.copy(code = code""" boolean ${ev.isNull} = true; $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};""") @@ -1453,7 +1453,7 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes { val truncFuncStr = truncFunc(dateVal, form) s""" int $form = $dtu.parseTruncLevel($fmt); - if ($form == -1 || $form > $maxLevel) { + if ($form < $minLevel) { ${ev.isNull} = true; } else { ${ev.value} = $dtu.$truncFuncStr @@ -1471,14 +1471,24 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes { @ExpressionDescription( usage = """ _FUNC_(date, fmt) - Returns `date` with the time portion of the day truncated to the unit specified by the format model `fmt`. - `fmt` should be one of ["year", "yyyy", "yy", "mon", "month", "mm"] + `fmt` should be one of ["week", "mon", "month", "mm", "quarter", "year", "yyyy", "yy", "decade", "century", "millennium"] """, examples = """ Examples: + > SELECT _FUNC_('2019-08-04', 'week'); + 2019-07-29 + > SELECT _FUNC_('2019-08-04', 'quarter'); + 2019-07-01 > SELECT _FUNC_('2009-02-12', 'MM'); 2009-02-01 > SELECT _FUNC_('2015-10-27', 'YEAR'); 2015-01-01 + > SELECT _FUNC_('2015-10-27', 'DECADE'); + 2010-01-01 + > SELECT _FUNC_('1981-01-19', 'century'); + 1901-01-01 + > SELECT _FUNC_('1981-01-19', 'millennium'); + 1001-01-01 """, since = "1.5.0") // scalastyle:on line.size.limit @@ -1493,14 +1503,14 @@ case class TruncDate(date: Expression, format: Expression) override val instant = date override def eval(input: InternalRow): Any = { - evalHelper(input, maxLevel = DateTimeUtils.TRUNC_TO_MONTH) { (d: Any, level: Int) => + evalHelper(input, minLevel = MIN_LEVEL_OF_DATE_TRUNC) { (d: Any, level: Int) => DateTimeUtils.truncDate(d.asInstanceOf[Int], level) } } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - codeGenHelper(ctx, ev, maxLevel = DateTimeUtils.TRUNC_TO_MONTH) { (date: String, fmt: String) => - s"truncDate($date, $fmt);" + codeGenHelper(ctx, ev, minLevel = MIN_LEVEL_OF_DATE_TRUNC) { + (date: String, fmt: String) => s"truncDate($date, $fmt);" } } } @@ -1512,7 +1522,9 @@ case class TruncDate(date: Expression, format: Expression) @ExpressionDescription( usage = """ _FUNC_(fmt, ts) - Returns timestamp `ts` truncated to the unit specified by the format model `fmt`. - `fmt` should be one of ["YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", "SECOND", "WEEK", "QUARTER"] + `fmt` should be one of ["MILLENNIUM", "CENTURY", "DECADE", "YEAR", "YYYY", "YY", + "QUARTER", "MON", "MONTH", "MM", "WEEK", "DAY", "DD", + "HOUR", "MINUTE", "SECOND", "MILLISECOND", "MICROSECOND"] """, examples = """ Examples: @@ -1524,6 +1536,12 @@ case class TruncDate(date: Expression, format: Expression) 2015-03-05 00:00:00 > SELECT _FUNC_('HOUR', '2015-03-05T09:32:05.359'); 2015-03-05 09:00:00 + > SELECT _FUNC_('MILLISECOND', '2015-03-05T09:32:05.123456'); + 2015-03-05 09:32:05.123 + > SELECT _FUNC_('DECADE', '2015-03-05T09:32:05.123456'); + 2010-01-01 00:00:00 + > SELECT _FUNC_('CENTURY', '2015-03-05T09:32:05.123456'); + 2001-01-01 00:00:00 """, since = "2.3.0") // scalastyle:on line.size.limit @@ -1545,14 +1563,14 @@ case class TruncTimestamp( def this(format: Expression, timestamp: Expression) = this(format, timestamp, None) override def eval(input: InternalRow): Any = { - evalHelper(input, maxLevel = DateTimeUtils.TRUNC_TO_SECOND) { (t: Any, level: Int) => + evalHelper(input, minLevel = MIN_LEVEL_OF_TIMESTAMP_TRUNC) { (t: Any, level: Int) => DateTimeUtils.truncTimestamp(t.asInstanceOf[Long], level, timeZone) } } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val tz = ctx.addReferenceObj("timeZone", timeZone) - codeGenHelper(ctx, ev, maxLevel = DateTimeUtils.TRUNC_TO_SECOND, true) { + codeGenHelper(ctx, ev, minLevel = MIN_LEVEL_OF_TIMESTAMP_TRUNC, true) { (date: String, fmt: String) => s"truncTimestamp($date, $fmt, $tz);" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 10a7f9bd550e2..987b56d12c099 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -617,91 +617,107 @@ object DateTimeUtils { (date - localDate.getDayOfMonth) + localDate.lengthOfMonth() } - // Visible for testing. - private[sql] val TRUNC_TO_YEAR = 1 - private[sql] val TRUNC_TO_MONTH = 2 - private[sql] val TRUNC_TO_QUARTER = 3 - private[sql] val TRUNC_TO_WEEK = 4 - private[sql] val TRUNC_TO_DAY = 5 - private[sql] val TRUNC_TO_HOUR = 6 - private[sql] val TRUNC_TO_MINUTE = 7 - private[sql] val TRUNC_TO_SECOND = 8 + // The constants are visible for testing purpose only. private[sql] val TRUNC_INVALID = -1 + // The levels from TRUNC_TO_MICROSECOND to TRUNC_TO_DAY are used in truncations + // of TIMESTAMP values only. + private[sql] val TRUNC_TO_MICROSECOND = 0 + private[sql] val MIN_LEVEL_OF_TIMESTAMP_TRUNC = TRUNC_TO_MICROSECOND + private[sql] val TRUNC_TO_MILLISECOND = 1 + private[sql] val TRUNC_TO_SECOND = 2 + private[sql] val TRUNC_TO_MINUTE = 3 + private[sql] val TRUNC_TO_HOUR = 4 + private[sql] val TRUNC_TO_DAY = 5 + // The levels from TRUNC_TO_WEEK to TRUNC_TO_MILLENNIUM are used in truncations + // of DATE and TIMESTAMP values. + private[sql] val TRUNC_TO_WEEK = 6 + private[sql] val MIN_LEVEL_OF_DATE_TRUNC = TRUNC_TO_WEEK + private[sql] val TRUNC_TO_MONTH = 7 + private[sql] val TRUNC_TO_QUARTER = 8 + private[sql] val TRUNC_TO_YEAR = 9 + private[sql] val TRUNC_TO_DECADE = 10 + private[sql] val TRUNC_TO_CENTURY = 11 + private[sql] val TRUNC_TO_MILLENNIUM = 12 /** * Returns the trunc date from original date and trunc level. - * Trunc level should be generated using `parseTruncLevel()`, should only be 1 or 2. + * Trunc level should be generated using `parseTruncLevel()`, should be between 0 and 6. */ def truncDate(d: SQLDate, level: Int): SQLDate = { - if (level == TRUNC_TO_YEAR) { - d - DateTimeUtils.getDayInYear(d) + 1 - } else if (level == TRUNC_TO_MONTH) { - d - DateTimeUtils.getDayOfMonth(d) + 1 - } else { - // caller make sure that this should never be reached - sys.error(s"Invalid trunc level: $level") + def truncToYearLevel(divider: Int, adjust: Int): SQLDate = { + val oldYear = getYear(d) + var newYear = Math.floorDiv(oldYear, divider) + if (adjust > 0 && Math.floorMod(oldYear, divider) == 0) { + newYear -= 1 + } + newYear = newYear * divider + adjust + localDateToDays(LocalDate.of(newYear, 1, 1)) + } + level match { + case TRUNC_TO_WEEK => getNextDateForDayOfWeek(d - 7, MONDAY) + case TRUNC_TO_MONTH => d - DateTimeUtils.getDayOfMonth(d) + 1 + case TRUNC_TO_QUARTER => + localDateToDays(daysToLocalDate(d).`with`(IsoFields.DAY_OF_QUARTER, 1L)) + case TRUNC_TO_YEAR => d - DateTimeUtils.getDayInYear(d) + 1 + case TRUNC_TO_DECADE => truncToYearLevel(10, 0) + case TRUNC_TO_CENTURY => truncToYearLevel(100, 1) + case TRUNC_TO_MILLENNIUM => truncToYearLevel(1000, 1) + case _ => + // caller make sure that this should never be reached + sys.error(s"Invalid trunc level: $level") } } /** * Returns the trunc date time from original date time and trunc level. - * Trunc level should be generated using `parseTruncLevel()`, should be between 1 and 8 + * Trunc level should be generated using `parseTruncLevel()`, should be between 0 and 12. */ def truncTimestamp(t: SQLTimestamp, level: Int, timeZone: TimeZone): SQLTimestamp = { + if (level == TRUNC_TO_MICROSECOND) return t var millis = MICROSECONDS.toMillis(t) val truncated = level match { - case TRUNC_TO_YEAR => - val dDays = millisToDays(millis, timeZone) - daysToMillis(truncDate(dDays, level), timeZone) - case TRUNC_TO_MONTH => - val dDays = millisToDays(millis, timeZone) - daysToMillis(truncDate(dDays, level), timeZone) - case TRUNC_TO_DAY => - val offset = timeZone.getOffset(millis) - millis += offset - millis - millis % MILLIS_PER_DAY - offset + case TRUNC_TO_MILLISECOND => millis + case TRUNC_TO_SECOND => + millis - millis % MILLIS_PER_SECOND + case TRUNC_TO_MINUTE => + millis - millis % MILLIS_PER_MINUTE case TRUNC_TO_HOUR => val offset = timeZone.getOffset(millis) millis += offset millis - millis % MILLIS_PER_HOUR - offset - case TRUNC_TO_MINUTE => - millis - millis % MILLIS_PER_MINUTE - case TRUNC_TO_SECOND => - millis - millis % MILLIS_PER_SECOND - case TRUNC_TO_WEEK => - val dDays = millisToDays(millis, timeZone) - val prevMonday = getNextDateForDayOfWeek(dDays - 7, MONDAY) - daysToMillis(prevMonday, timeZone) - case TRUNC_TO_QUARTER => + case TRUNC_TO_DAY => + val offset = timeZone.getOffset(millis) + millis += offset + millis - millis % MILLIS_PER_DAY - offset + case _ => // Try to truncate date levels val dDays = millisToDays(millis, timeZone) - val daysOfQuarter = LocalDate.ofEpochDay(dDays) - .`with`(IsoFields.DAY_OF_QUARTER, 1L).toEpochDay.toInt - daysToMillis(daysOfQuarter, timeZone) - case _ => - // caller make sure that this should never be reached - sys.error(s"Invalid trunc level: $level") + daysToMillis(truncDate(dDays, level), timeZone) } truncated * MICROS_PER_MILLIS } /** - * Returns the truncate level, could be TRUNC_YEAR, TRUNC_MONTH, TRUNC_TO_DAY, TRUNC_TO_HOUR, - * TRUNC_TO_MINUTE, TRUNC_TO_SECOND, TRUNC_TO_WEEK, TRUNC_TO_QUARTER or TRUNC_INVALID, - * TRUNC_INVALID means unsupported truncate level. + * Returns the truncate level, could be from TRUNC_TO_MICROSECOND to TRUNC_TO_MILLENNIUM, + * or TRUNC_INVALID, TRUNC_INVALID means unsupported truncate level. */ def parseTruncLevel(format: UTF8String): Int = { if (format == null) { TRUNC_INVALID } else { format.toString.toUpperCase(Locale.ROOT) match { - case "YEAR" | "YYYY" | "YY" => TRUNC_TO_YEAR - case "MON" | "MONTH" | "MM" => TRUNC_TO_MONTH - case "DAY" | "DD" => TRUNC_TO_DAY - case "HOUR" => TRUNC_TO_HOUR - case "MINUTE" => TRUNC_TO_MINUTE + case "MICROSECOND" => TRUNC_TO_MICROSECOND + case "MILLISECOND" => TRUNC_TO_MILLISECOND case "SECOND" => TRUNC_TO_SECOND + case "MINUTE" => TRUNC_TO_MINUTE + case "HOUR" => TRUNC_TO_HOUR + case "DAY" | "DD" => TRUNC_TO_DAY case "WEEK" => TRUNC_TO_WEEK + case "MON" | "MONTH" | "MM" => TRUNC_TO_MONTH case "QUARTER" => TRUNC_TO_QUARTER + case "YEAR" | "YYYY" | "YY" => TRUNC_TO_YEAR + case "DECADE" => TRUNC_TO_DECADE + case "CENTURY" => TRUNC_TO_CENTURY + case "MILLENNIUM" => TRUNC_TO_MILLENNIUM case _ => TRUNC_INVALID } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 30e10c5527ad6..65f150e78bc5f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -596,6 +596,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { testTrunc(date, null, null) testTrunc(null, "MON", null) testTrunc(null, null, null) + + testTrunc(Date.valueOf("2000-03-08"), "decade", Date.valueOf("2000-01-01")) + testTrunc(Date.valueOf("2000-03-08"), "century", Date.valueOf("1901-01-01")) } test("TruncTimestamp") { @@ -664,6 +667,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { testTrunc(inputDate, null, null) testTrunc(null, "MON", null) testTrunc(null, null, null) + + testTrunc(Timestamp.valueOf("2000-03-08 11:12:13"), "decade", + Timestamp.valueOf("2000-01-01 00:00:00")) + testTrunc(Timestamp.valueOf("2000-03-08 11:12:13"), "century", + Timestamp.valueOf("1901-01-01 00:00:00")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 8ff691fb17f27..195690f88cdf2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -462,7 +462,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { } val defaultInputTS = DateTimeUtils.stringToTimestamp( - UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId) + UTF8String.fromString("2015-03-05T09:32:05.359123"), defaultZoneId) val defaultInputTS1 = DateTimeUtils.stringToTimestamp( UTF8String.fromString("2015-03-31T20:32:05.359"), defaultZoneId) val defaultInputTS2 = DateTimeUtils.stringToTimestamp( @@ -486,6 +486,11 @@ class DateTimeUtilsSuite extends SparkFunSuite { testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", defaultInputTS.get) testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", defaultInputTS1.get) testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS2.get) + testTrunc(DateTimeUtils.TRUNC_TO_MICROSECOND, "2015-03-05T09:32:05.359123", defaultInputTS.get) + testTrunc(DateTimeUtils.TRUNC_TO_MILLISECOND, "2015-03-05T09:32:05.359", defaultInputTS.get) + testTrunc(DateTimeUtils.TRUNC_TO_DECADE, "2010-01-01", defaultInputTS.get) + testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "2001-01-01", defaultInputTS.get) + testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01", defaultInputTS.get) for (tz <- ALL_TIMEZONES) { withDefaultTimeZone(tz) { @@ -499,6 +504,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { UTF8String.fromString("2015-03-30T02:32:05.359"), defaultZoneId) val inputTS4 = DateTimeUtils.stringToTimestamp( UTF8String.fromString("2015-03-29T02:32:05.359"), defaultZoneId) + val inputTS5 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("1999-03-29T01:02:03.456789"), defaultZoneId) testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", inputTS.get, tz) testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", inputTS.get, tz) @@ -514,6 +521,9 @@ class DateTimeUtilsSuite extends SparkFunSuite { testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS.get, tz) testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS1.get, tz) testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", inputTS2.get, tz) + testTrunc(DateTimeUtils.TRUNC_TO_DECADE, "1990-01-01", inputTS5.get, tz) + testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "1901-01-01", inputTS5.get, tz) + testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01", inputTS.get, tz) } } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index 6cd3856e8672e..771bdc934ea55 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -278,20 +278,17 @@ SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; -- SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); -- 0 -- SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- -1 --- [SPARK-28017] Enhance date DATE_TRUNC --- -- test trunc function! --- --- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 --- SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 --- SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 --- SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 --- SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 --- SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 --- SELECT DATE_TRUNC('CENTURY', DATE '0055-08-10 BC'); -- 0100-01-01 BC --- SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 --- SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC --- SELECT DATE_TRUNC('DECADE', DATE '0002-12-31 BC'); -- 0011-01-01 BC +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- [SPARK-28141] Date type can not accept special values -- diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 46101ebce8113..4dd2b4724688c 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 53 +-- Number of queries: 63 -- !query 0 @@ -502,48 +502,128 @@ struct -- !query 47 -select make_date(2013, 7, 15) +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 47 schema -struct +struct -- !query 47 output -2013-07-15 +1001-01-01 00:07:02 -- !query 48 -select make_date(-44, 3, 15) +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query 48 schema -struct +struct -- !query 48 output --0044-03-15 +1001-01-01 00:07:02 -- !query 49 -select make_date(2013, 2, 30) +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 49 schema -struct +struct -- !query 49 output -NULL +1901-01-01 00:00:00 -- !query 50 -select make_date(2013, 13, 1) +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') -- !query 50 schema -struct +struct -- !query 50 output -NULL +1901-01-01 00:00:00 -- !query 51 -select make_date(2013, 11, -1) +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') -- !query 51 schema -struct +struct -- !query 51 output -NULL +2001-01-01 00:00:00 -- !query 52 -DROP TABLE DATE_TBL +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query 52 schema -struct<> +struct -- !query 52 output +0001-01-01 00:07:02 + + +-- !query 53 +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +-- !query 53 schema +struct +-- !query 53 output +-0099-01-01 00:07:02 + + +-- !query 54 +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +-- !query 54 schema +struct +-- !query 54 output +1990-01-01 00:00:00 + + +-- !query 55 +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +-- !query 55 schema +struct +-- !query 55 output +0000-01-01 00:07:02 + + +-- !query 56 +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +-- !query 56 schema +struct +-- !query 56 output +-0010-01-01 00:07:02 + + +-- !query 57 +select make_date(2013, 7, 15) +-- !query 57 schema +struct +-- !query 57 output +2013-07-15 + + +-- !query 58 +select make_date(-44, 3, 15) +-- !query 58 schema +struct +-- !query 58 output +-0044-03-15 + + +-- !query 59 +select make_date(2013, 2, 30) +-- !query 59 schema +struct +-- !query 59 output +NULL + + +-- !query 60 +select make_date(2013, 13, 1) +-- !query 60 schema +struct +-- !query 60 output +NULL + + +-- !query 61 +select make_date(2013, 11, -1) +-- !query 61 schema +struct +-- !query 61 output +NULL + + +-- !query 62 +DROP TABLE DATE_TBL +-- !query 62 schema +struct<> +-- !query 62 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index ff48ac8d7a6c4..69f17f5e9c2dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -443,12 +443,16 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer( df.selectExpr("trunc(t, 'Month')"), Seq(Row(Date.valueOf("2015-07-01")), Row(Date.valueOf("2014-12-01")))) + + checkAnswer( + df.selectExpr("trunc(t, 'decade')"), + Seq(Row(Date.valueOf("2010-01-01")), Row(Date.valueOf("2010-01-01")))) } test("function date_trunc") { val df = Seq( - (1, Timestamp.valueOf("2015-07-22 10:01:40.523")), - (2, Timestamp.valueOf("2014-12-31 05:29:06.876"))).toDF("i", "t") + (1, Timestamp.valueOf("2015-07-22 10:01:40.123456")), + (2, Timestamp.valueOf("2014-12-31 05:29:06.123456"))).toDF("i", "t") checkAnswer( df.select(date_trunc("YY", col("t"))), @@ -489,6 +493,23 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("date_trunc('QUARTER', t)"), Seq(Row(Timestamp.valueOf("2015-07-01 00:00:00")), Row(Timestamp.valueOf("2014-10-01 00:00:00")))) + + checkAnswer( + df.selectExpr("date_trunc('MILLISECOND', t)"), + Seq(Row(Timestamp.valueOf("2015-07-22 10:01:40.123")), + Row(Timestamp.valueOf("2014-12-31 05:29:06.123")))) + + checkAnswer( + df.selectExpr("date_trunc('DECADE', t)"), + Seq(Row(Timestamp.valueOf("2010-01-01 00:00:00")), + Row(Timestamp.valueOf("2010-01-01 00:00:00")))) + + Seq("century", "millennium").foreach { level => + checkAnswer( + df.selectExpr(s"date_trunc('$level', t)"), + Seq(Row(Timestamp.valueOf("2001-01-01 00:00:00")), + Row(Timestamp.valueOf("2001-01-01 00:00:00")))) + } } test("from_unixtime") { From 5368eaa2fc33f6d50b482bd1e90e0437b1887cd2 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 8 Aug 2019 22:30:00 -0700 Subject: [PATCH 024/149] [SPARK-28565][SQL] DataFrameWriter saveAsTable support for V2 catalogs ## What changes were proposed in this pull request? Adds support for V2 catalogs and the V2SessionCatalog for V2 tables for saveAsTable. If the table can resolve through the V2SessionCatalog, we use SaveMode for datasource v1 for backwards compatibility to select the code path we're going to hit. Depending on the SaveMode: - SaveMode.Append: a) If table exists: Use AppendData.byName b) If table doesn't exist, use CTAS (ignoreIfExists = false) - SaveMode.Overwrite: Use RTAS (orCreate = true) - SaveMode.Ignore: Use CTAS (ignoreIfExists = true) - SaveMode.ErrorIfExists: Use CTAS (ignoreIfExists = false) ## How was this patch tested? Unit tests in DataSourceV2DataFrameSuite Closes #25330 from brkyvz/saveAsTable. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Burak Yavuz --- .../apache/spark/sql/DataFrameWriter.scala | 81 +++++++++++++++++-- .../v2/DataSourceV2DataFrameSuite.scala | 64 ++++++++++++++- 2 files changed, 138 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index ae82670a63d5c..af7ddd756ae89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -22,22 +22,22 @@ import java.util.{Locale, Properties, UUID} import scala.collection.JavaConverters._ import org.apache.spark.annotation.Stable -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier} +import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister} -import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap /** @@ -360,6 +360,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def insertInto(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ assertNotBucketed("insertInto") @@ -374,8 +375,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { df.sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case CatalogObjectIdentifier(Some(catalog), ident) => insertInto(catalog, ident) + // TODO(SPARK-28667): Support the V2SessionCatalog case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) + case other => + throw new AnalysisException( + s"Couldn't find a catalog to handle the identifier ${other.quoted}.") } } @@ -485,7 +490,71 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def saveAsTable(tableName: String): Unit = { - saveAsTable(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)) + import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + val session = df.sparkSession + + session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { + case CatalogObjectIdentifier(Some(catalog), ident) => + saveAsTable(catalog.asTableCatalog, ident, modeForDSV2) + // TODO(SPARK-28666): This should go through V2SessionCatalog + + case AsTableIdentifier(tableIdentifier) => + saveAsTable(tableIdentifier) + + case other => + throw new AnalysisException( + s"Couldn't find a catalog to handle the identifier ${other.quoted}.") + } + } + + + private def saveAsTable(catalog: TableCatalog, ident: Identifier, mode: SaveMode): Unit = { + val partitioning = partitioningColumns.map { colNames => + colNames.map(name => IdentityTransform(FieldReference(name))) + }.getOrElse(Seq.empty[Transform]) + val bucketing = bucketColumnNames.map { cols => + Seq(BucketTransform(LiteralValue(numBuckets.get, IntegerType), cols.map(FieldReference(_)))) + }.getOrElse(Seq.empty[Transform]) + val partitionTransforms = partitioning ++ bucketing + + val tableOpt = try Option(catalog.loadTable(ident)) catch { + case _: NoSuchTableException => None + } + + val command = (mode, tableOpt) match { + case (SaveMode.Append, Some(table)) => + AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan) + + case (SaveMode.Overwrite, _) => + ReplaceTableAsSelect( + catalog, + ident, + partitionTransforms, + df.queryExecution.analyzed, + Map.empty, // properties can't be specified through this API + extraOptions.toMap, + orCreate = true) // Create the table if it doesn't exist + + case (other, _) => + // We have a potential race condition here in AppendMode, if the table suddenly gets + // created between our existence check and physical execution, but this can't be helped + // in any case. + CreateTableAsSelect( + catalog, + ident, + partitionTransforms, + df.queryExecution.analyzed, + Map.empty, + extraOptions.toMap, + ignoreIfExists = other == SaveMode.Ignore) + } + + runCommand(df.sparkSession, "saveAsTable") { + command + } } private def saveAsTable(tableIdent: TableIdentifier): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala index 755cabc620023..8909c41ddaa8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources.v2 import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} import org.apache.spark.sql.test.SharedSQLContext @@ -141,4 +141,66 @@ class DataSourceV2DataFrameSuite extends QueryTest with SharedSQLContext with Be } } } + + testQuietly("saveAsTable: table doesn't exist => create table") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.saveAsTable(t1) + checkAnswer(spark.table(t1), df) + } + } + + testQuietly("saveAsTable: table exists => append by name") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING foo") + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + // Default saveMode is append, therefore this doesn't throw a table already exists exception + df.write.saveAsTable(t1) + checkAnswer(spark.table(t1), df) + + // also appends are by name not by position + df.select('data, 'id).write.saveAsTable(t1) + checkAnswer(spark.table(t1), df.union(df)) + } + } + + testQuietly("saveAsTable: table overwrite and table doesn't exist => create table") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.mode("overwrite").saveAsTable(t1) + checkAnswer(spark.table(t1), df) + } + } + + testQuietly("saveAsTable: table overwrite and table exists => replace table") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 USING foo AS SELECT 'c', 'd'") + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.mode("overwrite").saveAsTable(t1) + checkAnswer(spark.table(t1), df) + } + } + + testQuietly("saveAsTable: ignore mode and table doesn't exist => create table") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.mode("ignore").saveAsTable(t1) + checkAnswer(spark.table(t1), df) + } + } + + testQuietly("saveAsTable: ignore mode and table exists => do nothing") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + sql(s"CREATE TABLE $t1 USING foo AS SELECT 'c', 'd'") + df.write.mode("ignore").saveAsTable(t1) + checkAnswer(spark.table(t1), Seq(Row("c", "d"))) + } + } } From 5663386f4b97cf19e82b5b540a8fae7844331c38 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 9 Aug 2019 17:08:11 +0800 Subject: [PATCH 025/149] [SPARK-28163][SS] Use CaseInsensitiveMap for KafkaOffsetReader ## What changes were proposed in this pull request? There are "unsafe" conversions in the Kafka connector. `CaseInsensitiveStringMap` comes in which is then converted the following way: ``` ... options.asScala.toMap ... ``` The main problem with this is that such case it looses its case insensitive nature (case insensitive map is converting the key to lower case when get/contains called). In this PR I'm using `CaseInsensitiveMap` to solve this problem. ## How was this patch tested? Existing + additional unit tests. Closes #24967 from gaborgsomogyi/SPARK-28163. Authored-by: Gabor Somogyi Signed-off-by: Wenchen Fan --- .../spark/sql/kafka010/KafkaBatch.scala | 5 +- .../sql/kafka010/KafkaContinuousStream.scala | 4 +- .../sql/kafka010/KafkaMicroBatchStream.scala | 8 +- .../sql/kafka010/KafkaOffsetReader.scala | 7 +- .../spark/sql/kafka010/KafkaRelation.scala | 4 +- .../sql/kafka010/KafkaSourceProvider.scala | 73 ++++++------- .../kafka010/KafkaSourceProviderSuite.scala | 100 +++++++++++------- 7 files changed, 112 insertions(+), 89 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index e3c8536d22070..839a64ed31322 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -22,12 +22,13 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory} private[kafka010] class KafkaBatch( strategy: ConsumerStrategy, - sourceOptions: Map[String, String], + sourceOptions: CaseInsensitiveMap[String], specifiedKafkaParams: Map[String, String], failOnDataLoss: Boolean, startingOffsets: KafkaOffsetRangeLimit, @@ -38,7 +39,7 @@ private[kafka010] class KafkaBatch( assert(endingOffsets != EarliestOffsetRangeLimit, "Ending offset not allowed to be set to earliest offsets.") - private val pollTimeoutMs = sourceOptions.getOrElse( + private[kafka010] val pollTimeoutMs = sourceOptions.getOrElse( KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, (SparkEnv.get.conf.get(NETWORK_TIMEOUT) * 1000L).toString ).toLong diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 8c26bc0d172ba..18d740eaa968f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -46,7 +46,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * properly read. */ class KafkaContinuousStream( - offsetReader: KafkaOffsetReader, + private[kafka010] val offsetReader: KafkaOffsetReader, kafkaParams: ju.Map[String, Object], options: CaseInsensitiveStringMap, metadataPath: String, @@ -54,7 +54,7 @@ class KafkaContinuousStream( failOnDataLoss: Boolean) extends ContinuousStream with Logging { - private val pollTimeoutMs = + private[kafka010] val pollTimeoutMs = options.getLong(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, 512) // Initialized when creating reader factories. If this diverges from the partitions at the latest diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 9fb338f0e6688..57cb0fae41841 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -56,19 +56,19 @@ import org.apache.spark.util.UninterruptibleThread * and not use wrong broker addresses. */ private[kafka010] class KafkaMicroBatchStream( - kafkaOffsetReader: KafkaOffsetReader, + private[kafka010] val kafkaOffsetReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], options: CaseInsensitiveStringMap, metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) extends RateControlMicroBatchStream with Logging { - private val pollTimeoutMs = options.getLong( + private[kafka010] val pollTimeoutMs = options.getLong( KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, SparkEnv.get.conf.get(NETWORK_TIMEOUT) * 1000L) - private val maxOffsetsPerTrigger = Option(options.get(KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER)) - .map(_.toLong) + private[kafka010] val maxOffsetsPerTrigger = Option(options.get( + KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER)).map(_.toLong) private val rangeCalculator = KafkaOffsetRangeCalculator(options) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index ad608ecafe59f..f3effd5300a79 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -30,6 +30,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsume import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.types._ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} @@ -47,7 +48,7 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} private[kafka010] class KafkaOffsetReader( consumerStrategy: ConsumerStrategy, val driverKafkaParams: ju.Map[String, Object], - readerOptions: Map[String, String], + readerOptions: CaseInsensitiveMap[String], driverGroupIdPrefix: String) extends Logging { /** * Used to ensure execute fetch operations execute in an UninterruptibleThread @@ -88,10 +89,10 @@ private[kafka010] class KafkaOffsetReader( _consumer } - private val maxOffsetFetchAttempts = + private[kafka010] val maxOffsetFetchAttempts = readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, "3").toInt - private val offsetFetchAttemptIntervalMs = + private[kafka010] val offsetFetchAttemptIntervalMs = readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, "1000").toLong private def nextGroupId(): String = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index b2950cff98a0d..dd584a5987a07 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -24,7 +24,7 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String @@ -33,7 +33,7 @@ import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( override val sqlContext: SQLContext, strategy: ConsumerStrategy, - sourceOptions: Map[String, String], + sourceOptions: CaseInsensitiveMap[String], specifiedKafkaParams: Map[String, String], failOnDataLoss: Boolean, startingOffsets: KafkaOffsetRangeLimit, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index a3ea918a8bc10..372bcab1cab30 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -78,32 +78,32 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = { - validateStreamOptions(parameters) + val caseInsensitiveParameters = CaseInsensitiveMap(parameters) + validateStreamOptions(caseInsensitiveParameters) // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath) + val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveParameters, metadataPath) - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } val specifiedKafkaParams = convertToSpecifiedParams(parameters) - val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, - STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( - strategy(caseInsensitiveParams), + strategy(caseInsensitiveParameters), kafkaParamsForDriver(specifiedKafkaParams), - parameters, + caseInsensitiveParameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") new KafkaSource( sqlContext, kafkaOffsetReader, kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), - parameters, + caseInsensitiveParameters, metadataPath, startingStreamOffsets, - failOnDataLoss(caseInsensitiveParams)) + failOnDataLoss(caseInsensitiveParameters)) } override def getTable(options: CaseInsensitiveStringMap): KafkaTable = { @@ -119,24 +119,24 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - validateBatchOptions(parameters) - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val caseInsensitiveParameters = CaseInsensitiveMap(parameters) + validateBatchOptions(caseInsensitiveParameters) val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) assert(startingRelationOffsets != LatestOffsetRangeLimit) - val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, - ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) assert(endingRelationOffsets != EarliestOffsetRangeLimit) new KafkaRelation( sqlContext, - strategy(caseInsensitiveParams), - sourceOptions = parameters, + strategy(caseInsensitiveParameters), + sourceOptions = caseInsensitiveParameters, specifiedKafkaParams = specifiedKafkaParams, - failOnDataLoss = failOnDataLoss(caseInsensitiveParams), + failOnDataLoss = failOnDataLoss(caseInsensitiveParameters), startingOffsets = startingRelationOffsets, endingOffsets = endingRelationOffsets) } @@ -420,23 +420,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { - val parameters = options.asScala.toMap - validateStreamOptions(parameters) + val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) + validateStreamOptions(caseInsensitiveOptions) // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = streamingUniqueGroupId(parameters, checkpointLocation) + val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions, checkpointLocation) - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = convertToSpecifiedParams(parameters) + val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( - strategy(parameters), + strategy(caseInsensitiveOptions), kafkaParamsForDriver(specifiedKafkaParams), - parameters, + caseInsensitiveOptions, driverGroupIdPrefix = s"$uniqueGroupId-driver") new KafkaMicroBatchStream( @@ -445,32 +444,26 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister options, checkpointLocation, startingStreamOffsets, - failOnDataLoss(caseInsensitiveParams)) + failOnDataLoss(caseInsensitiveOptions)) } override def toContinuousStream(checkpointLocation: String): ContinuousStream = { - val parameters = options.asScala.toMap - validateStreamOptions(parameters) + val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) + validateStreamOptions(caseInsensitiveOptions) // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = streamingUniqueGroupId(parameters, checkpointLocation) + val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions, checkpointLocation) - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( - strategy(caseInsensitiveParams), + strategy(caseInsensitiveOptions), kafkaParamsForDriver(specifiedKafkaParams), - parameters, + caseInsensitiveOptions, driverGroupIdPrefix = s"$uniqueGroupId-driver") new KafkaContinuousStream( @@ -479,7 +472,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister options, checkpointLocation, startingStreamOffsets, - failOnDataLoss(caseInsensitiveParams)) + failOnDataLoss(caseInsensitiveOptions)) } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala index 2fcf37a184684..8e6de88865e06 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala @@ -22,60 +22,92 @@ import java.util.Locale import scala.collection.JavaConverters._ import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.util.CaseInsensitiveStringMap -class KafkaSourceProviderSuite extends SparkFunSuite with PrivateMethodTester { +class KafkaSourceProviderSuite extends SparkFunSuite { - private val pollTimeoutMsMethod = PrivateMethod[Long]('pollTimeoutMs) - private val maxOffsetsPerTriggerMethod = PrivateMethod[Option[Long]]('maxOffsetsPerTrigger) + private val expected = "1111" override protected def afterEach(): Unit = { SparkEnv.set(null) super.afterEach() } + test("batch mode - options should be handled as case-insensitive") { + verifyFieldsInBatch(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, expected, batch => { + assert(expected.toLong === batch.pollTimeoutMs) + }) + } + test("micro-batch mode - options should be handled as case-insensitive") { - def verifyFieldsInMicroBatchStream( - options: CaseInsensitiveStringMap, - expectedPollTimeoutMs: Long, - expectedMaxOffsetsPerTrigger: Option[Long]): Unit = { - // KafkaMicroBatchStream reads Spark conf from SparkEnv for default value - // hence we set mock SparkEnv here before creating KafkaMicroBatchStream - val sparkEnv = mock(classOf[SparkEnv]) - when(sparkEnv.conf).thenReturn(new SparkConf()) - SparkEnv.set(sparkEnv) + verifyFieldsInMicroBatchStream(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, expected, stream => { + assert(expected.toLong === stream.pollTimeoutMs) + }) + verifyFieldsInMicroBatchStream(KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER, expected, stream => { + assert(Some(expected.toLong) === stream.maxOffsetsPerTrigger) + }) + verifyFieldsInMicroBatchStream(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, expected, stream => { + assert(expected.toInt === stream.kafkaOffsetReader.maxOffsetFetchAttempts) + }) + verifyFieldsInMicroBatchStream(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, expected, + stream => { + assert(expected.toLong === stream.kafkaOffsetReader.offsetFetchAttemptIntervalMs) + }) + } - val scan = getKafkaDataSourceScan(options) - val stream = scan.toMicroBatchStream("dummy").asInstanceOf[KafkaMicroBatchStream] + test("continuous mode - options should be handled as case-insensitive") { + verifyFieldsInContinuousStream(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, expected, stream => { + assert(expected.toLong === stream.pollTimeoutMs) + }) + verifyFieldsInContinuousStream(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, expected, stream => { + assert(expected.toInt === stream.offsetReader.maxOffsetFetchAttempts) + }) + verifyFieldsInContinuousStream(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, expected, + stream => { + assert(expected.toLong === stream.offsetReader.offsetFetchAttemptIntervalMs) + }) + } - assert(expectedPollTimeoutMs === getField(stream, pollTimeoutMsMethod)) - assert(expectedMaxOffsetsPerTrigger === getField(stream, maxOffsetsPerTriggerMethod)) + private def verifyFieldsInBatch( + key: String, + value: String, + validate: (KafkaBatch) => Unit): Unit = { + buildCaseInsensitiveStringMapForUpperAndLowerKey(key -> value).foreach { options => + val scan = getKafkaDataSourceScan(options) + val batch = scan.toBatch().asInstanceOf[KafkaBatch] + validate(batch) } + } - val expectedValue = 1000L - buildCaseInsensitiveStringMapForUpperAndLowerKey( - KafkaSourceProvider.CONSUMER_POLL_TIMEOUT -> expectedValue.toString, - KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER -> expectedValue.toString) - .foreach(verifyFieldsInMicroBatchStream(_, expectedValue, Some(expectedValue))) + private def verifyFieldsInMicroBatchStream( + key: String, + value: String, + validate: (KafkaMicroBatchStream) => Unit): Unit = { + // KafkaMicroBatchStream reads Spark conf from SparkEnv for default value + // hence we set mock SparkEnv here before creating KafkaMicroBatchStream + val sparkEnv = mock(classOf[SparkEnv]) + when(sparkEnv.conf).thenReturn(new SparkConf()) + SparkEnv.set(sparkEnv) + + buildCaseInsensitiveStringMapForUpperAndLowerKey(key -> value).foreach { options => + val scan = getKafkaDataSourceScan(options) + val stream = scan.toMicroBatchStream("dummy").asInstanceOf[KafkaMicroBatchStream] + validate(stream) + } } - test("SPARK-28142 - continuous mode - options should be handled as case-insensitive") { - def verifyFieldsInContinuousStream( - options: CaseInsensitiveStringMap, - expectedPollTimeoutMs: Long): Unit = { + private def verifyFieldsInContinuousStream( + key: String, + value: String, + validate: (KafkaContinuousStream) => Unit): Unit = { + buildCaseInsensitiveStringMapForUpperAndLowerKey(key -> value).foreach { options => val scan = getKafkaDataSourceScan(options) val stream = scan.toContinuousStream("dummy").asInstanceOf[KafkaContinuousStream] - assert(expectedPollTimeoutMs === getField(stream, pollTimeoutMsMethod)) + validate(stream) } - - val expectedValue = 1000 - buildCaseInsensitiveStringMapForUpperAndLowerKey( - KafkaSourceProvider.CONSUMER_POLL_TIMEOUT -> expectedValue.toString) - .foreach(verifyFieldsInContinuousStream(_, expectedValue)) } private def buildCaseInsensitiveStringMapForUpperAndLowerKey( @@ -95,8 +127,4 @@ class KafkaSourceProviderSuite extends SparkFunSuite with PrivateMethodTester { val provider = new KafkaSourceProvider() provider.getTable(options).newScanBuilder(options).build() } - - private def getField[T](obj: AnyRef, method: PrivateMethod[T]): T = { - obj.invokePrivate(method()) - } } From 5bb69945e4aaf519cd10a5c5083332f618039af0 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 9 Aug 2019 18:54:55 +0900 Subject: [PATCH 026/149] [SPARK-28651][SS] Force the schema of Streaming file source to be nullable ## What changes were proposed in this pull request? Right now, batch DataFrame always changes the schema to nullable automatically (See this line: https://github.com/apache/spark/blob/325bc8e9c6187a96b33a033fbb0145dfca619135/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L399). But streaming file source is missing this. This PR updates the streaming file source schema to force it be nullable. I also added a flag `spark.sql.streaming.fileSource.schema.forceNullable` to disable this change since some users may rely on the old behavior. ## How was this patch tested? The new unit test. Closes #25382 from zsxwing/SPARK-28651. Authored-by: Shixiong Zhu Signed-off-by: HyukjinKwon --- python/pyspark/sql/streaming.py | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 9 +++++++++ .../execution/datasources/DataSource.scala | 5 ++++- .../sql/streaming/FileStreamSourceSuite.scala | 19 +++++++++++++++++++ 4 files changed, 33 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 64a00c0b8bfcd..7faa8662ebf03 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -1155,7 +1155,7 @@ def _test(): globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext) globs['sdf'] = \ spark.readStream.format('text').load('python/test_support/sql/streaming') - globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) + globs['sdf_schema'] = StructType([StructField("data", StringType(), True)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') 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 f779bc835dfb6..51853b9248da7 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 @@ -1140,6 +1140,15 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(TimeUnit.MINUTES.toMillis(10)) // 10 minutes + val FILE_SOURCE_SCHEMA_FORCE_NULLABLE = + buildConf("spark.sql.streaming.fileSource.schema.forceNullable") + .internal() + .doc("When true, force the schema of streaming file source to be nullable (including all " + + "the fields). Otherwise, the schema might not be compatible with actual data, which " + + "leads to corruptions.") + .booleanConf + .createWithDefault(true) + val STREAMING_SCHEMA_INFERENCE = buildConf("spark.sql.streaming.schemaInference") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 04ae528a1f6b3..0ccd99c88cf28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -254,9 +254,12 @@ case class DataSource( checkAndGlobPathIfNecessary(checkEmptyGlobPath = false, checkFilesExist = false) createInMemoryFileIndex(globbedPaths) }) + val forceNullable = + sparkSession.sessionState.conf.getConf(SQLConf.FILE_SOURCE_SCHEMA_FORCE_NULLABLE) + val sourceDataSchema = if (forceNullable) dataSchema.asNullable else dataSchema SourceInfo( s"FileSource[$path]", - StructType(dataSchema ++ partitionSchema), + StructType(sourceDataSchema ++ partitionSchema), partitionSchema.fieldNames) case _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 72f893845172d..f3f03715ee83a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1577,6 +1577,25 @@ class FileStreamSourceSuite extends FileStreamSourceTest { ) } } + + test("SPARK-28651: force streaming file source to be nullable") { + withTempDir { temp => + val schema = StructType(Seq(StructField("foo", LongType, false))) + val nullableSchema = StructType(Seq(StructField("foo", LongType, true))) + val streamingSchema = spark.readStream.schema(schema).json(temp.getCanonicalPath).schema + assert(nullableSchema === streamingSchema) + + // Verify we have the same behavior as batch DataFrame. + val batchSchema = spark.read.schema(schema).json(temp.getCanonicalPath).schema + assert(batchSchema === streamingSchema) + + // Verify the flag works + withSQLConf(SQLConf.FILE_SOURCE_SCHEMA_FORCE_NULLABLE.key -> "false") { + val streamingSchema = spark.readStream.schema(schema).json(temp.getCanonicalPath).schema + assert(schema === streamingSchema) + } + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { From f21bc1874a0101da3615b92d3a375337c5038f31 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 9 Aug 2019 18:55:48 +0900 Subject: [PATCH 027/149] [SPARK-27889][INFRA] Make development scripts under dev/ support Python 3 ## What changes were proposed in this pull request? I made an audit and update all dev scripts to support python3. (except `merge_spark_pr.py` which already updated) ## How was this patch tested? Manual. Closes #25289 from WeichenXu123/dev_py3. Authored-by: WeichenXu Signed-off-by: HyukjinKwon --- dev/github_jira_sync.py | 25 ++++++++++++++--------- dev/merge_spark_pr.py | 4 ++-- dev/run-tests-jenkins.py | 32 +++++++++++++++++++----------- dev/run-tests.py | 2 +- dev/sparktestsupport/shellutils.py | 2 +- 5 files changed, 40 insertions(+), 25 deletions(-) diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py index 7b693cb64f327..fa1736163d4c6 100755 --- a/dev/github_jira_sync.py +++ b/dev/github_jira_sync.py @@ -22,7 +22,14 @@ import os import re import sys -import urllib2 +if sys.version < '3': + from urllib2 import urlopen + from urllib2 import Request + from urllib2 import HTTPError +else: + from urllib.request import urlopen + from urllib.request import Request + from urllib.error import HTTPError try: import jira.client @@ -52,16 +59,16 @@ def get_url(url): try: - request = urllib2.Request(url) + request = Request(url) request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) - return urllib2.urlopen(request) - except urllib2.HTTPError: + return urlopen(request) + except HTTPError: print("Unable to fetch URL, exiting: %s" % url) sys.exit(-1) def get_json(urllib_response): - return json.load(urllib_response) + return json.loads(urllib_response.read().decode("utf-8")) # Return a list of (JIRA id, JSON dict) tuples: @@ -80,7 +87,7 @@ def get_jira_prs(): result = result + [(jira, pull)] # Check if there is another page - link_headers = filter(lambda k: k.startswith("Link"), page.info().headers) + link_headers = list(filter(lambda k: k.startswith("Link"), page.headers)) if not link_headers or "next" not in link_headers[0]: has_next_page = False else: @@ -122,13 +129,13 @@ def reset_pr_labels(pr_num, jira_components): url = '%s/issues/%s/labels' % (GITHUB_API_BASE, pr_num) labels = ', '.join(('"%s"' % c) for c in jira_components) try: - request = urllib2.Request(url, data='{"labels":[%s]}' % labels) + request = Request(url, data=('{"labels":[%s]}' % labels).encode('utf-8')) request.add_header('Content-Type', 'application/json') request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) request.get_method = lambda: 'PUT' - urllib2.urlopen(request) + urlopen(request) print("Set %s with labels %s" % (pr_num, labels)) - except urllib2.HTTPError: + except HTTPError: print("Unable to update PR labels, exiting: %s" % url) sys.exit(-1) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 4c3eb0a170ca2..6901acca728a8 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -97,9 +97,9 @@ def fail(msg): def run_cmd(cmd): print(cmd) if isinstance(cmd, list): - return subprocess.check_output(cmd).decode(sys.stdout.encoding) + return subprocess.check_output(cmd).decode(sys.getdefaultencoding()) else: - return subprocess.check_output(cmd.split(" ")).decode(sys.stdout.encoding) + return subprocess.check_output(cmd.split(" ")).decode(sys.getdefaultencoding()) def continue_maybe(prompt): diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index fdc4f2b48685e..4b91a5fa423c7 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python2 +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -21,9 +21,17 @@ import os import sys import json -import urllib2 import functools import subprocess +if sys.version < '3': + from urllib2 import urlopen + from urllib2 import Request + from urllib2 import HTTPError, URLError +else: + from urllib.request import urlopen + from urllib.request import Request + from urllib.error import HTTPError, URLError + from sparktestsupport import SPARK_HOME, ERROR_CODES from sparktestsupport.shellutils import run_cmd @@ -44,25 +52,25 @@ def post_message_to_github(msg, ghprb_pull_id): github_oauth_key = os.environ["GITHUB_OAUTH_KEY"] posted_message = json.dumps({"body": msg}) - request = urllib2.Request(url, - headers={ - "Authorization": "token %s" % github_oauth_key, - "Content-Type": "application/json" - }, - data=posted_message) + request = Request(url, + headers={ + "Authorization": "token %s" % github_oauth_key, + "Content-Type": "application/json" + }, + data=posted_message.encode('utf-8')) try: - response = urllib2.urlopen(request) + response = urlopen(request) if response.getcode() == 201: print(" > Post successful.") - except urllib2.HTTPError as http_e: + except HTTPError as http_e: print_err("Failed to post message to Github.") print_err(" > http_code: %s" % http_e.code) print_err(" > api_response: %s" % http_e.read()) print_err(" > data: %s" % posted_message) - except urllib2.URLError as url_e: + except URLError as url_e: print_err("Failed to post message to Github.") - print_err(" > urllib2_status: %s" % url_e.reason[1]) + print_err(" > urllib_status: %s" % url_e.reason[1]) print_err(" > data: %s" % posted_message) diff --git a/dev/run-tests.py b/dev/run-tests.py index 1b4f6b73654ca..bad3b007cdce3 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python2 +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index c7644da88f770..ec6ea86269f5e 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -55,7 +55,7 @@ def run_cmd(cmd, return_output=False): cmd = cmd.split() try: if return_output: - return subprocess_check_output(cmd) + return subprocess_check_output(cmd).decode(sys.getdefaultencoding()) else: return subprocess_check_call(cmd) except subprocess.CalledProcessError as e: From 515987641507c0da7fd2bda7f9ce7dde6998701a Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Fri, 9 Aug 2019 19:05:41 +0900 Subject: [PATCH 028/149] [SPARK-28077][SQL][TEST][FOLLOW-UP] Enable Overlay function tests ## What changes were proposed in this pull request? This PR is a follow-up to https://github.com/apache/spark/pull/24918 ## How was this patch tested? Pass the Jenkins with the newly update test files. Closes #25393 from beliefer/enable-overlay-tests. Authored-by: gengjiaan Signed-off-by: Takeshi Yamamuro --- .../sql-tests/inputs/pgSQL/strings.sql | 8 +- .../sql-tests/results/pgSQL/strings.sql.out | 422 ++++++++++-------- 2 files changed, 231 insertions(+), 199 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql index a6844284cb620..05841af27dd2e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/strings.sql @@ -234,13 +234,13 @@ SELECT POSITION('5' IN '1234567890') = '5' AS `5`; -- [SPARK-28077] Add support string functions: OVERLAY -- T312 character overlay function --- SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS "abc45f"; +SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f`; --- SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS "yabadaba"; +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba`; --- SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS "yabadabadoo"; +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo`; --- SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS "bubba"; +SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba`; -- -- test LIKE diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out index 41440e28431d7..1e4e6e5021de8 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/strings.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 88 +-- Number of queries: 92 -- !query 0 @@ -137,87 +137,87 @@ true -- !query 16 -SELECT 'hawkeye' LIKE 'h%' AS `true` +SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f` -- !query 16 schema -struct +struct -- !query 16 output -true +abc45f -- !query 17 -SELECT 'hawkeye' NOT LIKE 'h%' AS `false` +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba` -- !query 17 schema -struct +struct -- !query 17 output -false +yabadaba -- !query 18 -SELECT 'hawkeye' LIKE 'H%' AS `false` +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo` -- !query 18 schema -struct +struct -- !query 18 output -false +yabadabadoo -- !query 19 -SELECT 'hawkeye' NOT LIKE 'H%' AS `true` +SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba` -- !query 19 schema -struct +struct -- !query 19 output -true +bubba -- !query 20 -SELECT 'hawkeye' LIKE 'indio%' AS `false` +SELECT 'hawkeye' LIKE 'h%' AS `true` -- !query 20 schema -struct +struct -- !query 20 output -false +true -- !query 21 -SELECT 'hawkeye' NOT LIKE 'indio%' AS `true` +SELECT 'hawkeye' NOT LIKE 'h%' AS `false` -- !query 21 schema -struct +struct -- !query 21 output -true +false -- !query 22 -SELECT 'hawkeye' LIKE 'h%eye' AS `true` +SELECT 'hawkeye' LIKE 'H%' AS `false` -- !query 22 schema -struct +struct -- !query 22 output -true +false -- !query 23 -SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false` +SELECT 'hawkeye' NOT LIKE 'H%' AS `true` -- !query 23 schema -struct +struct -- !query 23 output -false +true -- !query 24 -SELECT 'indio' LIKE '_ndio' AS `true` +SELECT 'hawkeye' LIKE 'indio%' AS `false` -- !query 24 schema -struct +struct -- !query 24 output -true +false -- !query 25 -SELECT 'indio' NOT LIKE '_ndio' AS `false` +SELECT 'hawkeye' NOT LIKE 'indio%' AS `true` -- !query 25 schema -struct +struct -- !query 25 output -false +true -- !query 26 -SELECT 'indio' LIKE 'in__o' AS `true` +SELECT 'hawkeye' LIKE 'h%eye' AS `true` -- !query 26 schema struct -- !query 26 output @@ -225,7 +225,7 @@ true -- !query 27 -SELECT 'indio' NOT LIKE 'in__o' AS `false` +SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false` -- !query 27 schema struct -- !query 27 output @@ -233,111 +233,111 @@ false -- !query 28 -SELECT 'indio' LIKE 'in_o' AS `false` +SELECT 'indio' LIKE '_ndio' AS `true` -- !query 28 schema -struct +struct -- !query 28 output -false +true -- !query 29 -SELECT 'indio' NOT LIKE 'in_o' AS `true` +SELECT 'indio' NOT LIKE '_ndio' AS `false` -- !query 29 schema -struct +struct -- !query 29 output -true +false -- !query 30 -SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f +SELECT 'indio' LIKE 'in__o' AS `true` -- !query 30 schema -struct +struct -- !query 30 output -true true false +true -- !query 31 -SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f +SELECT 'indio' NOT LIKE 'in__o' AS `false` -- !query 31 schema -struct +struct -- !query 31 output -true true false +false -- !query 32 -SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f +SELECT 'indio' LIKE 'in_o' AS `false` -- !query 32 schema -struct +struct -- !query 32 output -true true false +false -- !query 33 -SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f +SELECT 'indio' NOT LIKE 'in_o' AS `true` -- !query 33 schema -struct +struct -- !query 33 output -true true false +true -- !query 34 -SELECT 'jack' LIKE '%____%' AS t +SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f -- !query 34 schema -struct +struct -- !query 34 output -true +true true false -- !query 35 -SELECT 'unknown' || ' and unknown' AS `Concat unknown types` +SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f -- !query 35 schema -struct +struct -- !query 35 output -unknown and unknown +true true false -- !query 36 -SELECT string('text') || ' and unknown' AS `Concat text to unknown type` +SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f -- !query 36 schema -struct +struct -- !query 36 output -text and unknown +true true false -- !query 37 -CREATE TABLE toasttest(f1 string) USING parquet +SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f -- !query 37 schema -struct<> +struct -- !query 37 output - +true true false -- !query 38 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'jack' LIKE '%____%' AS t -- !query 38 schema -struct<> +struct -- !query 38 output - +true -- !query 39 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'unknown' || ' and unknown' AS `Concat unknown types` -- !query 39 schema -struct<> +struct -- !query 39 output - +unknown and unknown -- !query 40 -insert into toasttest values(repeat('1234567890',10000)) +SELECT string('text') || ' and unknown' AS `Concat text to unknown type` -- !query 40 schema -struct<> +struct -- !query 40 output - +text and unknown -- !query 41 -insert into toasttest values(repeat('1234567890',10000)) +CREATE TABLE toasttest(f1 string) USING parquet -- !query 41 schema struct<> -- !query 41 output @@ -345,125 +345,125 @@ struct<> -- !query 42 -SELECT substr(f1, 99995) from toasttest +insert into toasttest values(repeat('1234567890',10000)) -- !query 42 schema -struct +struct<> -- !query 42 output -567890 -567890 -567890 -567890 + -- !query 43 -SELECT substr(f1, 99995, 10) from toasttest +insert into toasttest values(repeat('1234567890',10000)) -- !query 43 schema -struct +struct<> -- !query 43 output -567890 -567890 -567890 -567890 + -- !query 44 -SELECT length('abcdef') AS `length_6` +insert into toasttest values(repeat('1234567890',10000)) -- !query 44 schema -struct +struct<> -- !query 44 output -6 + -- !query 45 -SELECT position('cd', 'abcdef') AS `pos_3` +insert into toasttest values(repeat('1234567890',10000)) -- !query 45 schema -struct +struct<> -- !query 45 output -3 + -- !query 46 -SELECT position('xy', 'abcdef') AS `pos_0` +SELECT substr(f1, 99995) from toasttest -- !query 46 schema -struct +struct -- !query 46 output -0 +567890 +567890 +567890 +567890 -- !query 47 -SELECT replace('abcdef', 'de', '45') AS `abc45f` +SELECT substr(f1, 99995, 10) from toasttest -- !query 47 schema -struct +struct -- !query 47 output -abc45f +567890 +567890 +567890 +567890 -- !query 48 -SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` +SELECT length('abcdef') AS `length_6` -- !query 48 schema -struct +struct -- !query 48 output -ya123da123doo +6 -- !query 49 -SELECT replace('yabadoo', 'bad', '') AS `yaoo` +SELECT position('cd', 'abcdef') AS `pos_3` -- !query 49 schema -struct +struct -- !query 49 output -yaoo +3 -- !query 50 -select hex(256*256*256 - 1) AS `ffffff` +SELECT position('xy', 'abcdef') AS `pos_0` -- !query 50 schema -struct +struct -- !query 50 output -FFFFFF +0 -- !query 51 -select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` +SELECT replace('abcdef', 'de', '45') AS `abc45f` -- !query 51 schema -struct +struct -- !query 51 output -FFFFFFFF +abc45f -- !query 52 -select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` -- !query 52 schema -struct +struct -- !query 52 output -true +ya123da123doo -- !query 53 -select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +SELECT replace('yabadoo', 'bad', '') AS `yaoo` -- !query 53 schema -struct +struct -- !query 53 output -true +yaoo -- !query 54 -select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +select hex(256*256*256 - 1) AS `ffffff` -- !query 54 schema -struct +struct -- !query 54 output -true +FFFFFF -- !query 55 -select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` -- !query 55 schema -struct +struct -- !query 55 output -true +FFFFFFFF -- !query 56 -select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` -- !query 56 schema struct -- !query 56 output @@ -471,7 +471,7 @@ true -- !query 57 -select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` -- !query 57 schema struct -- !query 57 output @@ -479,7 +479,7 @@ true -- !query 58 -select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` -- !query 58 schema struct -- !query 58 output @@ -487,7 +487,7 @@ true -- !query 59 -select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` -- !query 59 schema struct -- !query 59 output @@ -495,7 +495,7 @@ true -- !query 60 -select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` -- !query 60 schema struct -- !query 60 output @@ -503,7 +503,7 @@ true -- !query 61 -select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` -- !query 61 schema struct -- !query 61 output @@ -511,7 +511,7 @@ true -- !query 62 -select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` -- !query 62 schema struct -- !query 62 output @@ -519,7 +519,7 @@ true -- !query 63 -select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` -- !query 63 schema struct -- !query 63 output @@ -527,7 +527,7 @@ true -- !query 64 -select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` -- !query 64 schema struct -- !query 64 output @@ -535,7 +535,7 @@ true -- !query 65 -select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` -- !query 65 schema struct -- !query 65 output @@ -543,176 +543,208 @@ true -- !query 66 -SELECT initcap('hi THOMAS') +select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` -- !query 66 schema -struct +struct -- !query 66 output -Hi Thomas +true -- !query 67 -SELECT lpad('hi', 5, 'xy') +select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` -- !query 67 schema -struct +struct -- !query 67 output -xyxhi +true -- !query 68 -SELECT lpad('hi', 5) +select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` -- !query 68 schema -struct +struct -- !query 68 output - hi +true -- !query 69 -SELECT lpad('hi', -5, 'xy') +select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` -- !query 69 schema -struct +struct -- !query 69 output - +true -- !query 70 -SELECT lpad('hello', 2) +SELECT initcap('hi THOMAS') -- !query 70 schema -struct +struct -- !query 70 output -he +Hi Thomas -- !query 71 -SELECT lpad('hi', 5, '') +SELECT lpad('hi', 5, 'xy') -- !query 71 schema -struct +struct -- !query 71 output -hi +xyxhi -- !query 72 -SELECT rpad('hi', 5, 'xy') +SELECT lpad('hi', 5) -- !query 72 schema -struct +struct -- !query 72 output -hixyx + hi -- !query 73 -SELECT rpad('hi', 5) +SELECT lpad('hi', -5, 'xy') -- !query 73 schema -struct +struct -- !query 73 output -hi + -- !query 74 -SELECT rpad('hi', -5, 'xy') +SELECT lpad('hello', 2) -- !query 74 schema -struct +struct -- !query 74 output - +he -- !query 75 -SELECT rpad('hello', 2) +SELECT lpad('hi', 5, '') -- !query 75 schema -struct +struct -- !query 75 output -he +hi -- !query 76 -SELECT rpad('hi', 5, '') +SELECT rpad('hi', 5, 'xy') -- !query 76 schema -struct +struct -- !query 76 output -hi +hixyx -- !query 77 -SELECT ltrim('zzzytrim', 'xyz') +SELECT rpad('hi', 5) -- !query 77 schema -struct +struct -- !query 77 output -trim +hi -- !query 78 -SELECT translate('', '14', 'ax') +SELECT rpad('hi', -5, 'xy') -- !query 78 schema -struct +struct -- !query 78 output -- !query 79 -SELECT translate('12345', '14', 'ax') +SELECT rpad('hello', 2) -- !query 79 schema -struct +struct -- !query 79 output -a23x5 +he -- !query 80 -SELECT ascii('x') +SELECT rpad('hi', 5, '') -- !query 80 schema -struct +struct -- !query 80 output -120 +hi -- !query 81 -SELECT ascii('') +SELECT ltrim('zzzytrim', 'xyz') -- !query 81 schema -struct +struct -- !query 81 output -0 +trim -- !query 82 -SELECT chr(65) +SELECT translate('', '14', 'ax') -- !query 82 schema -struct +struct -- !query 82 output -A + -- !query 83 -SELECT chr(0) +SELECT translate('12345', '14', 'ax') -- !query 83 schema -struct +struct -- !query 83 output - +a23x5 -- !query 84 -SELECT repeat('Pg', 4) +SELECT ascii('x') -- !query 84 schema -struct +struct -- !query 84 output -PgPgPgPg +120 -- !query 85 -SELECT repeat('Pg', -4) +SELECT ascii('') -- !query 85 schema -struct +struct -- !query 85 output - +0 -- !query 86 -SELECT trim(binary('\\000') from binary('\\000Tom\\000')) +SELECT chr(65) -- !query 86 schema -struct +struct -- !query 86 output -Tom +A -- !query 87 -DROP TABLE toasttest +SELECT chr(0) -- !query 87 schema -struct<> +struct -- !query 87 output + + + +-- !query 88 +SELECT repeat('Pg', 4) +-- !query 88 schema +struct +-- !query 88 output +PgPgPgPg + + +-- !query 89 +SELECT repeat('Pg', -4) +-- !query 89 schema +struct +-- !query 89 output + + + +-- !query 90 +SELECT trim(binary('\\000') from binary('\\000Tom\\000')) +-- !query 90 schema +struct +-- !query 90 output +Tom + + +-- !query 91 +DROP TABLE toasttest +-- !query 91 schema +struct<> +-- !query 91 output From cbad616d4cb0c58993a88df14b5e30778c7f7e85 Mon Sep 17 00:00:00 2001 From: wuyi Date: Fri, 9 Aug 2019 07:49:03 -0500 Subject: [PATCH 029/149] [SPARK-27371][CORE] Support GPU-aware resources scheduling in Standalone ## What changes were proposed in this pull request? In this PR, we implements a complete process of GPU-aware resources scheduling in Standalone. The whole process looks like: Worker sets up isolated resources when it starts up and registers to master along with its resources. And, Master picks up usable workers according to driver/executor's resource requirements to launch driver/executor on them. Then, Worker launches the driver/executor after preparing resources file, which is created under driver/executor's working directory, with specified resource addresses(told by master). When driver/executor finished, their resources could be recycled to worker. Finally, if a worker stops, it should always release its resources firstly. For the case of Workers and Drivers in **client** mode run on the same host, we introduce a config option named `spark.resources.coordinate.enable`(default true) to indicate whether Spark should coordinate resources for user. If `spark.resources.coordinate.enable=false`, user should be responsible for configuring different resources for Workers and Drivers when use resourcesFile or discovery script. If true, Spark would help user to assign different resources for Workers and Drivers. The solution for Spark to coordinate resources among Workers and Drivers is: Generally, use a shared file named *____allocated_resources____.json* to sync allocated resources info among Workers and Drivers on the same host. After a Worker or Driver found all resources using the configured resourcesFile and/or discovery script during launching, it should filter out available resources by excluding resources already allocated in *____allocated_resources____.json* and acquire resources from available resources according to its own requirement. After that, it should write its allocated resources along with its process id (pid) into *____allocated_resources____.json*. Pid (proposed by tgravescs) here used to check whether the allocated resources are still valid in case of Worker or Driver crashes and doesn't release resources properly. And when a Worker or Driver finished, normally, it would always clean up its own allocated resources in *____allocated_resources____.json*. Note that we'll always get a file lock before any access to file *____allocated_resources____.json* and release the lock finally. Futhermore, we appended resources info in `WorkerSchedulerStateResponse` to work around master change behaviour in HA mode. ## How was this patch tested? Added unit tests in WorkerSuite, MasterSuite, SparkContextSuite. Manually tested with client/cluster mode (e.g. multiple workers) in a single node Standalone. Closes #25047 from Ngone51/SPARK-27371. Authored-by: wuyi Signed-off-by: Thomas Graves --- .gitignore | 1 + .../scala/org/apache/spark/SparkContext.scala | 34 +- .../spark/deploy/ApplicationDescription.scala | 5 +- .../org/apache/spark/deploy/Client.scala | 7 +- .../apache/spark/deploy/DeployMessage.scala | 28 +- .../spark/deploy/DriverDescription.scala | 5 +- .../spark/deploy/LocalSparkCluster.scala | 3 +- .../deploy/StandaloneResourceUtils.scala | 348 ++++++++++++++++++ .../spark/deploy/master/ApplicationInfo.scala | 5 +- .../spark/deploy/master/DriverInfo.scala | 8 + .../spark/deploy/master/ExecutorDesc.scala | 6 +- .../apache/spark/deploy/master/Master.scala | 108 ++++-- .../spark/deploy/master/WorkerInfo.scala | 58 ++- .../deploy/rest/StandaloneRestServer.scala | 6 +- .../spark/deploy/worker/DriverRunner.scala | 14 +- .../spark/deploy/worker/ExecutorRunner.scala | 11 +- .../apache/spark/deploy/worker/Worker.scala | 77 ++-- .../CoarseGrainedExecutorBackend.scala | 4 +- .../spark/internal/config/package.scala | 17 + .../spark/resource/ResourceAllocator.scala | 96 +++++ .../apache/spark/resource/ResourceUtils.scala | 42 ++- .../scheduler/ExecutorResourceInfo.scala | 77 +--- .../spark/scheduler/TaskSchedulerImpl.scala | 7 +- .../cluster/StandaloneSchedulerBackend.scala | 6 +- .../scala/org/apache/spark/util/Utils.scala | 42 +++ .../org/apache/spark/SparkConfSuite.scala | 6 +- .../org/apache/spark/SparkContextSuite.scala | 15 +- .../apache/spark/deploy/DeployTestUtils.scala | 4 +- .../spark/deploy/master/MasterSuite.scala | 159 ++++++-- .../master/PersistenceEngineSuite.scala | 3 +- .../spark/deploy/worker/WorkerSuite.scala | 170 ++++++++- .../CoarseGrainedExecutorBackendSuite.scala | 2 +- .../spark/resource/ResourceUtilsSuite.scala | 2 +- .../spark/resource/TestResourceIDs.scala | 4 + docs/configuration.md | 23 +- docs/spark-standalone.md | 31 ++ python/pyspark/tests/test_context.py | 3 +- python/pyspark/tests/test_taskcontext.py | 8 +- 38 files changed, 1217 insertions(+), 228 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala create mode 100644 core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala diff --git a/.gitignore b/.gitignore index 4b1ba1c92bc04..ae20c85ebe351 100644 --- a/.gitignore +++ b/.gitignore @@ -71,6 +71,7 @@ scalastyle-on-compile.generated.xml scalastyle-output.xml scalastyle.txt spark-*-bin-*.tgz +spark-resources/ spark-tests.log src_managed/ streaming-tests.log diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index aa71b21caa30e..396d712bd739c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -42,6 +42,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.deploy.StandaloneResourceUtils._ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging @@ -245,6 +246,15 @@ class SparkContext(config: SparkConf) extends Logging { def isLocal: Boolean = Utils.isLocalMaster(_conf) + private def isClientStandalone: Boolean = { + val isSparkCluster = master match { + case SparkMasterRegex.SPARK_REGEX(_) => true + case SparkMasterRegex.LOCAL_CLUSTER_REGEX(_, _, _) => true + case _ => false + } + deployMode == "client" && isSparkCluster + } + /** * @return true if context is stopped or in the midst of stopping. */ @@ -380,7 +390,18 @@ class SparkContext(config: SparkConf) extends Logging { _driverLogger = DriverLogger(_conf) val resourcesFileOpt = conf.get(DRIVER_RESOURCES_FILE) - _resources = getOrDiscoverAllResources(_conf, SPARK_DRIVER_PREFIX, resourcesFileOpt) + val allResources = getOrDiscoverAllResources(_conf, SPARK_DRIVER_PREFIX, resourcesFileOpt) + _resources = { + // driver submitted in client mode under Standalone may have conflicting resources with + // other drivers/workers on this host. We should sync driver's resources info into + // SPARK_RESOURCES/SPARK_RESOURCES_COORDINATE_DIR/ to avoid collision. + if (isClientStandalone) { + acquireResources(_conf, SPARK_DRIVER_PREFIX, allResources, Utils.getProcessId) + } else { + allResources + } + } + logResourceInfo(SPARK_DRIVER_PREFIX, _resources) // log out spark.app.name in the Spark driver logs logInfo(s"Submitted application: $appName") @@ -1911,8 +1932,10 @@ class SparkContext(config: SparkConf) extends Logging { ShutdownHookManager.removeShutdownHook(_shutdownHookRef) } - Utils.tryLogNonFatalError { - postApplicationEnd() + if (listenerBus != null) { + Utils.tryLogNonFatalError { + postApplicationEnd() + } } Utils.tryLogNonFatalError { _driverLogger.foreach(_.stop()) @@ -1960,6 +1983,9 @@ class SparkContext(config: SparkConf) extends Logging { Utils.tryLogNonFatalError { _progressBar.foreach(_.stop()) } + if (isClientStandalone) { + releaseResources(_conf, SPARK_DRIVER_PREFIX, _resources, Utils.getProcessId) + } _taskScheduler = null // TODO: Cache.stop()? if (_env != null) { @@ -2726,7 +2752,7 @@ object SparkContext extends Logging { // Calculate the max slots each executor can provide based on resources available on each // executor and resources required by each task. - val taskResourceRequirements = parseTaskResourceRequirements(sc.conf) + val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX) val executorResourcesAndAmounts = parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) .map(request => (request.id.resourceName, request.amount)).toMap diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index c5c5c60923f4e..e11f497b4bfd8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy import java.net.URI +import org.apache.spark.resource.ResourceRequirement + private[spark] case class ApplicationDescription( name: String, maxCores: Option[Int], @@ -32,7 +34,8 @@ private[spark] case class ApplicationDescription( // number of executors this application wants to start with, // only used if dynamic allocation is enabled initialExecutorLimit: Option[Int] = None, - user: String = System.getProperty("user.name", "")) { + user: String = System.getProperty("user.name", ""), + resourceReqsPerExecutor: Seq[ResourceRequirement] = Seq.empty) { override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index ea7c902b1b6bb..648a8b1c763db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -29,6 +29,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.Network.RPC_ASK_TIMEOUT +import org.apache.spark.resource.ResourceUtils import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils} @@ -92,13 +93,15 @@ private class ClientEndpoint( val command = new Command(mainClass, Seq("{{WORKER_URL}}", "{{USER_JAR}}", driverArgs.mainClass) ++ driverArgs.driverOptions, sys.env, classPathEntries, libraryPathEntries, javaOpts) - + val driverResourceReqs = ResourceUtils.parseResourceRequirements(conf, + config.SPARK_DRIVER_PREFIX) val driverDescription = new DriverDescription( driverArgs.jarUrl, driverArgs.memory, driverArgs.cores, driverArgs.supervise, - command) + command, + driverResourceReqs) asyncSendToMasterAndForwardReply[SubmitDriverResponse]( RequestSubmitDriver(driverDescription)) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 5723b0f69057b..3f1d1aebdf9d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -24,6 +24,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.RecoveryState.MasterState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.util.Utils @@ -31,7 +32,6 @@ private[deploy] sealed trait DeployMessage extends Serializable /** Contains messages sent between Scheduler endpoint nodes. */ private[deploy] object DeployMessages { - // Worker to Master /** @@ -43,6 +43,7 @@ private[deploy] object DeployMessages { * @param memory the memory size of worker * @param workerWebUiUrl the worker Web UI address * @param masterAddress the master address used by the worker to connect + * @param resources the resources of worker */ case class RegisterWorker( id: String, @@ -52,7 +53,8 @@ private[deploy] object DeployMessages { cores: Int, memory: Int, workerWebUiUrl: String, - masterAddress: RpcAddress) + masterAddress: RpcAddress, + resources: Map[String, ResourceInformation] = Map.empty) extends DeployMessage { Utils.checkHost(host) assert (port > 0) @@ -72,8 +74,18 @@ private[deploy] object DeployMessages { exception: Option[Exception]) extends DeployMessage - case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription], - driverIds: Seq[String]) + case class WorkerExecutorStateResponse( + desc: ExecutorDescription, + resources: Map[String, ResourceInformation]) + + case class WorkerDriverStateResponse( + driverId: String, + resources: Map[String, ResourceInformation]) + + case class WorkerSchedulerStateResponse( + id: String, + execResponses: List[WorkerExecutorStateResponse], + driverResponses: Seq[WorkerDriverStateResponse]) /** * A worker will send this message to the master when it registers with the master. Then the @@ -118,10 +130,14 @@ private[deploy] object DeployMessages { execId: Int, appDesc: ApplicationDescription, cores: Int, - memory: Int) + memory: Int, + resources: Map[String, ResourceInformation] = Map.empty) extends DeployMessage - case class LaunchDriver(driverId: String, driverDesc: DriverDescription) extends DeployMessage + case class LaunchDriver( + driverId: String, + driverDesc: DriverDescription, + resources: Map[String, ResourceInformation] = Map.empty) extends DeployMessage case class KillDriver(driverId: String) extends DeployMessage diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index 1f5626ab5a896..02c166b8785ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,12 +17,15 @@ package org.apache.spark.deploy +import org.apache.spark.resource.ResourceRequirement + private[deploy] case class DriverDescription( jarUrl: String, mem: Int, cores: Int, supervise: Boolean, - command: Command) { + command: Command, + resourceReqs: Seq[ResourceRequirement] = Seq.empty) { override def toString: String = s"DriverDescription (${command.mainClass})" } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index c1866b4c3606e..f1b58eb33a1b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -64,7 +64,8 @@ class LocalSparkCluster( /* Start the Workers */ for (workerNum <- 1 to numWorkers) { val workerEnv = Worker.startRpcEnvAndEndpoint(localHostname, 0, 0, coresPerWorker, - memoryPerWorker, masters, null, Some(workerNum), _conf) + memoryPerWorker, masters, null, Some(workerNum), _conf, + conf.get(config.Worker.SPARK_WORKER_RESOURCE_FILE)) workerRpcEnvs += workerEnv } diff --git a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala new file mode 100644 index 0000000000000..b64a36f532d03 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala @@ -0,0 +1,348 @@ +/* + * 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.deploy + +import java.io.{File, RandomAccessFile} +import java.nio.channels.{FileLock, OverlappingFileLockException} +import java.nio.file.Files + +import scala.collection.mutable +import scala.util.Random +import scala.util.control.NonFatal + +import org.json4s.{DefaultFormats, Extraction} +import org.json4s.jackson.JsonMethods.{compact, parse, render} + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{SPARK_RESOURCES_COORDINATE, SPARK_RESOURCES_DIR} +import org.apache.spark.resource.{ResourceAllocation, ResourceID, ResourceInformation, ResourceRequirement} +import org.apache.spark.resource.ResourceUtils.{parseResourceRequirements, withResourcesJson} +import org.apache.spark.util.Utils + +private[spark] object StandaloneResourceUtils extends Logging { + // These directory/files are used to coordinate the resources between + // the drivers/workers on the host in Spark Standalone. + val SPARK_RESOURCES_COORDINATE_DIR = "spark-resources" + val ALLOCATED_RESOURCES_FILE = "__allocated_resources__.json" + val RESOURCES_LOCK_FILE = "__allocated_resources__.lock" + + /** + * Resource allocation used in Standalone only, which tracks assignments with + * worker/driver(client only) pid. + */ + case class StandaloneResourceAllocation(pid: Int, allocations: Seq[ResourceAllocation]) { + // convert allocations to a resource information map + def toResourceInformationMap: Map[String, ResourceInformation] = { + allocations.map { allocation => + allocation.id.resourceName -> allocation.toResourceInformation + }.toMap + } + } + + /** + * Assigns (if coordinate needed) resources to workers/drivers from the same host to avoid + * address conflict. + * + * This function works in three steps. First, acquiring the lock on RESOURCES_LOCK_FILE + * to achieve synchronization among workers and drivers. Second, getting all allocated + * resources from ALLOCATED_RESOURCES_FILE and assigning isolated resources to the worker + * or driver after differentiating available resources in discovered resources from + * allocated resources. If available resources don't meet worker's or driver's requirement, + * try to update allocated resources by excluding the resource allocation if its related + * process has already terminated and do the assignment again. If still don't meet requirement, + * exception should be thrown. Third, updating ALLOCATED_RESOURCES_FILE with new allocated + * resources along with pid for the worker or driver. Then, return allocated resources + * information after releasing the lock. + * + * @param conf SparkConf + * @param componentName spark.driver / spark.worker + * @param resources the resources found by worker/driver on the host + * @param pid the process id of worker/driver to acquire resources. + * @return allocated resources for the worker/driver or throws exception if can't + * meet worker/driver's requirement + */ + def acquireResources( + conf: SparkConf, + componentName: String, + resources: Map[String, ResourceInformation], + pid: Int) + : Map[String, ResourceInformation] = { + if (!needCoordinate(conf)) { + return resources + } + val resourceRequirements = parseResourceRequirements(conf, componentName) + if (resourceRequirements.isEmpty) { + return Map.empty + } + val lock = acquireLock(conf) + try { + val resourcesFile = new File(getOrCreateResourcesDir(conf), ALLOCATED_RESOURCES_FILE) + // all allocated resources in ALLOCATED_RESOURCES_FILE, can be updated if any allocations' + // related processes detected to be terminated while checking pids below. + var origAllocation = Seq.empty[StandaloneResourceAllocation] + // Map[pid -> Map[resourceName -> Addresses[]]] + var allocated = { + if (resourcesFile.exists()) { + origAllocation = allocatedStandaloneResources(resourcesFile.getPath) + val allocations = origAllocation.map { resource => + val resourceMap = { + resource.allocations.map { allocation => + allocation.id.resourceName -> allocation.addresses.toArray + }.toMap + } + resource.pid -> resourceMap + }.toMap + allocations + } else { + Map.empty[Int, Map[String, Array[String]]] + } + } + + // new allocated resources for worker or driver, + // map from resource name to its allocated addresses. + var newAssignments: Map[String, Array[String]] = null + // Whether we've checked process status and we'll only do the check at most once. + // Do the check iff the available resources can't meet the requirements at the first time. + var checked = false + // Whether we need to keep allocating for the worker/driver and we'll only go through + // the loop at most twice. + var keepAllocating = true + while (keepAllocating) { + keepAllocating = false + // store the pid whose related allocated resources conflict with + // discovered resources passed in. + val pidsToCheck = mutable.Set[Int]() + newAssignments = resourceRequirements.map { req => + val rName = req.resourceName + val amount = req.amount + // initially, we must have available.length >= amount as we've done pre-check previously + var available = resources(rName).addresses + // gets available resource addresses by excluding all + // allocated resource addresses from discovered resources + allocated.foreach { a => + val thePid = a._1 + val resourceMap = a._2 + val assigned = resourceMap.getOrElse(rName, Array.empty) + val retained = available.diff(assigned) + // if len(retained) < len(available) after differ to assigned, then, there must be + // some conflicting resources addresses between available and assigned. So, we should + // store its pid here to check whether it's alive in case we don't find enough + // resources after traversal all allocated resources. + if (retained.length < available.length && !checked) { + pidsToCheck += thePid + } + if (retained.length >= amount) { + available = retained + } else if (checked) { + keepAllocating = false + throw new SparkException(s"No more resources available since they've already" + + s" assigned to other workers/drivers.") + } else { + keepAllocating = true + } + } + val assigned = { + if (keepAllocating) { // can't meet the requirement + // excludes the allocation whose related process has already been terminated. + val (invalid, valid) = allocated.partition { a => + pidsToCheck(a._1) && !(Utils.isTesting || Utils.isProcessRunning(a._1))} + allocated = valid + origAllocation = origAllocation.filter( + allocation => !invalid.contains(allocation.pid)) + checked = true + // note this is a meaningless return value, just to avoid creating any new object + available + } else { + available.take(amount) + } + } + rName -> assigned + }.toMap + } + val newAllocation = { + val allocations = newAssignments.map { case (rName, addresses) => + ResourceAllocation(ResourceID(componentName, rName), addresses) + }.toSeq + StandaloneResourceAllocation(pid, allocations) + } + writeResourceAllocationJson( + componentName, origAllocation ++ Seq(newAllocation), resourcesFile) + newAllocation.toResourceInformationMap + } finally { + releaseLock(lock) + } + } + + /** + * Frees (if coordinate needed) all the resources a worker/driver (pid) has in one shot + * to make those resources be available for other workers/drivers on the same host. + * @param conf SparkConf + * @param componentName spark.driver / spark.worker + * @param toRelease the resources expected to release + * @param pid the process id of worker/driver to release resources. + */ + def releaseResources( + conf: SparkConf, + componentName: String, + toRelease: Map[String, ResourceInformation], + pid: Int) + : Unit = { + if (!needCoordinate(conf)) { + return + } + if (toRelease != null && toRelease.nonEmpty) { + val lock = acquireLock(conf) + try { + val resourcesFile = new File(getOrCreateResourcesDir(conf), ALLOCATED_RESOURCES_FILE) + if (resourcesFile.exists()) { + val (target, others) = + allocatedStandaloneResources(resourcesFile.getPath).partition(_.pid == pid) + if (target.nonEmpty) { + if (others.isEmpty) { + if (!resourcesFile.delete()) { + logError(s"Failed to delete $ALLOCATED_RESOURCES_FILE.") + } + } else { + writeResourceAllocationJson(componentName, others, resourcesFile) + } + logDebug(s"$componentName(pid=$pid) released resources: ${toRelease.mkString("\n")}") + } else { + logWarning(s"$componentName(pid=$pid) has already released its resources.") + } + } + } finally { + releaseLock(lock) + } + } + } + + private def acquireLock(conf: SparkConf): FileLock = { + val resourcesDir = getOrCreateResourcesDir(conf) + val lockFile = new File(resourcesDir, RESOURCES_LOCK_FILE) + val lockFileChannel = new RandomAccessFile(lockFile, "rw").getChannel + var keepTry = true + var lock: FileLock = null + while (keepTry) { + try { + lock = lockFileChannel.lock() + logInfo(s"Acquired lock on $RESOURCES_LOCK_FILE.") + keepTry = false + } catch { + case e: OverlappingFileLockException => + // This exception throws when we're in LocalSparkCluster mode. FileLock is designed + // to be used across JVMs, but our LocalSparkCluster is designed to launch multiple + // workers in the same JVM. As a result, when an worker in LocalSparkCluster try to + // acquire the lock on `resources.lock` which already locked by other worker, we'll + // hit this exception. So, we should manually control it. + keepTry = true + // there may be multiple workers race for the lock, + // so, sleep for a random time to avoid possible conflict + val duration = Random.nextInt(1000) + 1000 + Thread.sleep(duration) + } + } + assert(lock != null, s"Acquired null lock on $RESOURCES_LOCK_FILE.") + lock + } + + private def releaseLock(lock: FileLock): Unit = { + try { + lock.release() + lock.channel().close() + logInfo(s"Released lock on $RESOURCES_LOCK_FILE.") + } catch { + case e: Exception => + logError(s"Error while releasing lock on $RESOURCES_LOCK_FILE.", e) + } + } + + private def getOrCreateResourcesDir(conf: SparkConf): File = { + val coordinateDir = new File(conf.get(SPARK_RESOURCES_DIR).getOrElse { + val sparkHome = if (Utils.isTesting) { + assert(sys.props.contains("spark.test.home") || + sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") + sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) + } else { + sys.env.getOrElse("SPARK_HOME", ".") + } + sparkHome + }) + val resourceDir = new File(coordinateDir, SPARK_RESOURCES_COORDINATE_DIR) + if (!resourceDir.exists()) { + Utils.createDirectory(resourceDir) + } + resourceDir + } + + private def allocatedStandaloneResources(resourcesFile: String) + : Seq[StandaloneResourceAllocation] = { + withResourcesJson[StandaloneResourceAllocation](resourcesFile) { json => + implicit val formats = DefaultFormats + parse(json).extract[Seq[StandaloneResourceAllocation]] + } + } + + /** + * Save the allocated resources of driver(cluster only) or executor into a JSON formatted + * resources file. Used in Standalone only. + * @param componentName spark.driver / spark.executor + * @param resources allocated resources for driver(cluster only) or executor + * @param dir the target directory used to place the resources file + * @return None if resources is empty or Some(file) which represents the resources file + */ + def prepareResourcesFile( + componentName: String, + resources: Map[String, ResourceInformation], + dir: File): Option[File] = { + if (resources.isEmpty) { + return None + } + + val compShortName = componentName.substring(componentName.lastIndexOf(".") + 1) + val tmpFile = Utils.tempFileWith(dir) + val allocations = resources.map { case (rName, rInfo) => + ResourceAllocation(ResourceID(componentName, rName), rInfo.addresses) + }.toSeq + try { + writeResourceAllocationJson(componentName, allocations, tmpFile) + } catch { + case NonFatal(e) => + val errMsg = s"Exception threw while preparing resource file for $compShortName" + logError(errMsg, e) + throw new SparkException(errMsg, e) + } + val resourcesFile = File.createTempFile(s"resource-$compShortName-", ".json", dir) + tmpFile.renameTo(resourcesFile) + Some(resourcesFile) + } + + private def writeResourceAllocationJson[T]( + componentName: String, + allocations: Seq[T], + jsonFile: File): Unit = { + implicit val formats = DefaultFormats + val allocationJson = Extraction.decompose(allocations) + Files.write(jsonFile.toPath, compact(render(allocationJson)).getBytes()) + } + + /** Whether needs to coordinate resources among workers and drivers for user */ + def needCoordinate(conf: SparkConf): Boolean = { + conf.get(SPARK_RESOURCES_COORDINATE) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 53564d0e95152..6c56807458b27 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -23,6 +23,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.deploy.ApplicationDescription +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -82,8 +83,10 @@ private[spark] class ApplicationInfo( private[master] def addExecutor( worker: WorkerInfo, cores: Int, + resources: Map[String, ResourceInformation], useID: Option[Int] = None): ExecutorDesc = { - val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerExecutorMB) + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, + desc.memoryPerExecutorMB, resources) executors(exec.id) = exec coresGranted += cores exec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 8d5edae0501e2..bf68ba8e15af4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.util.Date import org.apache.spark.deploy.DriverDescription +import org.apache.spark.resource.ResourceInformation import org.apache.spark.util.Utils private[deploy] class DriverInfo( @@ -34,6 +35,9 @@ private[deploy] class DriverInfo( @transient var exception: Option[Exception] = None /* Most recent worker assigned to this driver */ @transient var worker: Option[WorkerInfo] = None + // resources(e.f. gpu/fpga) allocated to this driver + // map from resource name to ResourceInformation + private var _resources: Map[String, ResourceInformation] = _ init() @@ -47,4 +51,8 @@ private[deploy] class DriverInfo( worker = None exception = None } + + def withResources(r: Map[String, ResourceInformation]): Unit = _resources = r + + def resources: Map[String, ResourceInformation] = _resources } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index fc62b094def67..a8f8492561115 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -18,13 +18,17 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} +import org.apache.spark.resource.ResourceInformation private[master] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, val cores: Int, - val memory: Int) { + val memory: Int, + // resources(e.f. gpu/fpga) allocated to this executor + // map from resource name to ResourceInformation + val resources: Map[String, ResourceInformation]) { var state = ExecutorState.LAUNCHING diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 3c0a49e4ab205..676551985608b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -25,8 +25,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, - ExecutorState, SparkHadoopUtil} +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ @@ -38,6 +37,7 @@ import org.apache.spark.internal.config.Deploy._ import org.apache.spark.internal.config.UI._ import org.apache.spark.internal.config.Worker._ import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} +import org.apache.spark.resource.{ResourceRequirement, ResourceUtils} import org.apache.spark.rpc._ import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -244,7 +244,8 @@ private[deploy] class Master( System.exit(0) case RegisterWorker( - id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl, masterAddress) => + id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl, + masterAddress, resources) => logInfo("Registering worker %s:%d with %d cores, %s RAM".format( workerHost, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { @@ -252,8 +253,9 @@ private[deploy] class Master( } else if (idToWorker.contains(id)) { workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, true)) } else { + val workerResources = resources.map(r => r._1 -> WorkerResourceInfo(r._1, r._2.addresses)) val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, - workerRef, workerWebUiUrl) + workerRef, workerWebUiUrl, workerResources) if (registerWorker(worker)) { persistenceEngine.addWorker(worker) workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, false)) @@ -361,24 +363,31 @@ private[deploy] class Master( if (canCompleteRecovery) { completeRecovery() } - case WorkerSchedulerStateResponse(workerId, executors, driverIds) => + case WorkerSchedulerStateResponse(workerId, execResponses, driverResponses) => idToWorker.get(workerId) match { case Some(worker) => logInfo("Worker has been re-registered: " + workerId) worker.state = WorkerState.ALIVE - val validExecutors = executors.filter(exec => idToApp.get(exec.appId).isDefined) + val validExecutors = execResponses.filter( + exec => idToApp.get(exec.desc.appId).isDefined) for (exec <- validExecutors) { - val app = idToApp(exec.appId) - val execInfo = app.addExecutor(worker, exec.cores, Some(exec.execId)) + val (execDesc, execResources) = (exec.desc, exec.resources) + val app = idToApp(execDesc.appId) + val execInfo = app.addExecutor( + worker, execDesc.cores, execResources, Some(execDesc.execId)) worker.addExecutor(execInfo) - execInfo.copyState(exec) + worker.recoverResources(execResources) + execInfo.copyState(execDesc) } - for (driverId <- driverIds) { + for (driver <- driverResponses) { + val (driverId, driverResource) = (driver.driverId, driver.resources) drivers.find(_.id == driverId).foreach { driver => driver.worker = Some(worker) driver.state = DriverState.RUNNING + driver.withResources(driverResource) + worker.recoverResources(driverResource) worker.addDriver(driver) } } @@ -614,24 +623,34 @@ private[deploy] class Master( val minCoresPerExecutor = coresPerExecutor.getOrElse(1) val oneExecutorPerWorker = coresPerExecutor.isEmpty val memoryPerExecutor = app.desc.memoryPerExecutorMB + val resourceReqsPerExecutor = app.desc.resourceReqsPerExecutor val numUsable = usableWorkers.length val assignedCores = new Array[Int](numUsable) // Number of cores to give to each worker val assignedExecutors = new Array[Int](numUsable) // Number of new executors on each worker var coresToAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) /** Return whether the specified worker can launch an executor for this app. */ - def canLaunchExecutor(pos: Int): Boolean = { + def canLaunchExecutorForApp(pos: Int): Boolean = { val keepScheduling = coresToAssign >= minCoresPerExecutor val enoughCores = usableWorkers(pos).coresFree - assignedCores(pos) >= minCoresPerExecutor + val assignedExecutorNum = assignedExecutors(pos) // If we allow multiple executors per worker, then we can always launch new executors. // Otherwise, if there is already an executor on this worker, just give it more cores. - val launchingNewExecutor = !oneExecutorPerWorker || assignedExecutors(pos) == 0 + val launchingNewExecutor = !oneExecutorPerWorker || assignedExecutorNum == 0 if (launchingNewExecutor) { - val assignedMemory = assignedExecutors(pos) * memoryPerExecutor + val assignedMemory = assignedExecutorNum * memoryPerExecutor val enoughMemory = usableWorkers(pos).memoryFree - assignedMemory >= memoryPerExecutor + val assignedResources = resourceReqsPerExecutor.map { + req => req.resourceName -> req.amount * assignedExecutorNum + }.toMap + val resourcesFree = usableWorkers(pos).resourcesFree.map { + case (rName, free) => rName -> (free - assignedResources.getOrElse(rName, 0)) + } + val enoughResources = ResourceUtils.resourcesMeetRequirements( + resourcesFree, resourceReqsPerExecutor) val underLimit = assignedExecutors.sum + app.executors.size < app.executorLimit - keepScheduling && enoughCores && enoughMemory && underLimit + keepScheduling && enoughCores && enoughMemory && enoughResources && underLimit } else { // We're adding cores to an existing executor, so no need // to check memory and executor limits @@ -641,11 +660,11 @@ private[deploy] class Master( // Keep launching executors until no more workers can accommodate any // more executors, or if we have reached this application's limits - var freeWorkers = (0 until numUsable).filter(canLaunchExecutor) + var freeWorkers = (0 until numUsable).filter(canLaunchExecutorForApp) while (freeWorkers.nonEmpty) { freeWorkers.foreach { pos => var keepScheduling = true - while (keepScheduling && canLaunchExecutor(pos)) { + while (keepScheduling && canLaunchExecutorForApp(pos)) { coresToAssign -= minCoresPerExecutor assignedCores(pos) += minCoresPerExecutor @@ -666,7 +685,7 @@ private[deploy] class Master( } } } - freeWorkers = freeWorkers.filter(canLaunchExecutor) + freeWorkers = freeWorkers.filter(canLaunchExecutorForApp) } assignedCores } @@ -683,9 +702,11 @@ private[deploy] class Master( if (app.coresLeft >= coresPerExecutor) { // Filter out workers that don't have enough resources to launch an executor val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && - worker.coresFree >= coresPerExecutor) + .filter(canLaunchExecutor(_, app.desc)) .sortBy(_.coresFree).reverse + if (waitingApps.length == 1 && usableWorkers.isEmpty) { + logWarning(s"App ${app.id} requires more resource than any of Workers could have.") + } val assignedCores = scheduleExecutorsOnWorkers(app, usableWorkers, spreadOutApps) // Now that we've decided how many cores to allocate on each worker, let's allocate them @@ -715,12 +736,44 @@ private[deploy] class Master( val numExecutors = coresPerExecutor.map { assignedCores / _ }.getOrElse(1) val coresToAssign = coresPerExecutor.getOrElse(assignedCores) for (i <- 1 to numExecutors) { - val exec = app.addExecutor(worker, coresToAssign) + val allocated = worker.acquireResources(app.desc.resourceReqsPerExecutor) + val exec = app.addExecutor(worker, coresToAssign, allocated) launchExecutor(worker, exec) app.state = ApplicationState.RUNNING } } + private def canLaunch( + worker: WorkerInfo, + memoryReq: Int, + coresReq: Int, + resourceRequirements: Seq[ResourceRequirement]) + : Boolean = { + val enoughMem = worker.memoryFree >= memoryReq + val enoughCores = worker.coresFree >= coresReq + val enoughResources = ResourceUtils.resourcesMeetRequirements( + worker.resourcesFree, resourceRequirements) + enoughMem && enoughCores && enoughResources + } + + /** + * @return whether the worker could launch the driver represented by DriverDescription + */ + private def canLaunchDriver(worker: WorkerInfo, desc: DriverDescription): Boolean = { + canLaunch(worker, desc.mem, desc.cores, desc.resourceReqs) + } + + /** + * @return whether the worker could launch the executor according to application's requirement + */ + private def canLaunchExecutor(worker: WorkerInfo, desc: ApplicationDescription): Boolean = { + canLaunch( + worker, + desc.memoryPerExecutorMB, + desc.coresPerExecutor.getOrElse(1), + desc.resourceReqsPerExecutor) + } + /** * Schedule the currently available resources among waiting apps. This method will be called * every time a new app joins or resource availability changes. @@ -738,17 +791,24 @@ private[deploy] class Master( // start from the last worker that was assigned a driver, and continue onwards until we have // explored all alive workers. var launched = false + var isClusterIdle = true var numWorkersVisited = 0 while (numWorkersVisited < numWorkersAlive && !launched) { val worker = shuffledAliveWorkers(curPos) + isClusterIdle = worker.drivers.isEmpty && worker.executors.isEmpty numWorkersVisited += 1 - if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { + if (canLaunchDriver(worker, driver.desc)) { + val allocated = worker.acquireResources(driver.desc.resourceReqs) + driver.withResources(allocated) launchDriver(worker, driver) waitingDrivers -= driver launched = true } curPos = (curPos + 1) % numWorkersAlive } + if (!launched && isClusterIdle) { + logWarning(s"Driver ${driver.id} requires more resource than any of Workers could have.") + } } startExecutorsOnWorkers() } @@ -756,8 +816,8 @@ private[deploy] class Master( private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc): Unit = { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) - worker.endpoint.send(LaunchExecutor(masterUrl, - exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory)) + worker.endpoint.send(LaunchExecutor(masterUrl, exec.application.id, exec.id, + exec.application.desc, exec.cores, exec.memory, exec.resources)) exec.application.driver.send( ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)) } @@ -1021,7 +1081,7 @@ private[deploy] class Master( logInfo("Launching driver " + driver.id + " on worker " + worker.id) worker.addDriver(driver) driver.worker = Some(worker) - worker.endpoint.send(LaunchDriver(driver.id, driver.desc)) + worker.endpoint.send(LaunchDriver(driver.id, driver.desc, driver.resources)) driver.state = DriverState.RUNNING } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index c87d6e24b78c6..d485db43c5f96 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -19,9 +19,24 @@ package org.apache.spark.deploy.master import scala.collection.mutable +import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, ResourceRequirement} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils +private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String]) + extends ResourceAllocator(name, addresses) { + + def toResourceInformation(): ResourceInformation = { + new ResourceInformation(name, addresses.toArray) + } + + def acquire(amount: Int): ResourceInformation = { + val allocated = availableAddrs.take(amount) + acquire(allocated) + new ResourceInformation(name, allocated.toArray) + } +} + private[spark] class WorkerInfo( val id: String, val host: String, @@ -29,7 +44,9 @@ private[spark] class WorkerInfo( val cores: Int, val memory: Int, val endpoint: RpcEndpointRef, - val webUiAddress: String) + val webUiAddress: String, + val resources: Map[String, WorkerResourceInfo], + val pid: Int = 0) extends Serializable { Utils.checkHost(host) @@ -47,6 +64,11 @@ private[spark] class WorkerInfo( def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed + def resourcesFree: Map[String, Int] = { + resources.map { case (rName, rInfo) => + rName -> rInfo.availableAddrs.length + } + } private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() @@ -78,6 +100,7 @@ private[spark] class WorkerInfo( executors -= exec.fullId coresUsed -= exec.cores memoryUsed -= exec.memory + releaseResources(exec.resources) } } @@ -95,6 +118,7 @@ private[spark] class WorkerInfo( drivers -= driver.id memoryUsed -= driver.desc.mem coresUsed -= driver.desc.cores + releaseResources(driver.resources) } def setState(state: WorkerState.Value): Unit = { @@ -102,4 +126,36 @@ private[spark] class WorkerInfo( } def isAlive(): Boolean = this.state == WorkerState.ALIVE + + /** + * acquire specified amount resources for driver/executor from the worker + * @param resourceReqs the resources requirement from driver/executor + */ + def acquireResources(resourceReqs: Seq[ResourceRequirement]) + : Map[String, ResourceInformation] = { + resourceReqs.map { req => + val rName = req.resourceName + val amount = req.amount + rName -> resources(rName).acquire(amount) + }.toMap + } + + /** + * used during master recovery + */ + def recoverResources(expected: Map[String, ResourceInformation]): Unit = { + expected.foreach { case (rName, rInfo) => + resources(rName).acquire(rInfo.addresses) + } + } + + /** + * release resources to worker from the driver/executor + * @param allocated the resources which allocated to driver/executor previously + */ + private def releaseResources(allocated: Map[String, ResourceInformation]): Unit = { + allocated.foreach { case (rName, rInfo) => + resources(rName).release(rInfo.addresses) + } + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index f912ed64c80bd..c060ef9da8c10 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -25,6 +25,7 @@ import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ import org.apache.spark.internal.config import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.resource.ResourceUtils import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -174,8 +175,11 @@ private[rest] class StandaloneSubmitRequestServlet( val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + val driverResourceReqs = ResourceUtils.parseResourceRequirements(conf, + config.SPARK_DRIVER_PREFIX) new DriverDescription( - appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) + appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command, + driverResourceReqs) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 0c88119441ad3..4934722c0d83e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -28,10 +28,13 @@ import com.google.common.io.Files import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged +import org.apache.spark.deploy.StandaloneResourceUtils.prepareResourcesFile import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{DRIVER_RESOURCES_FILE, SPARK_DRIVER_PREFIX} import org.apache.spark.internal.config.Worker.WORKER_DRIVER_TERMINATE_TIMEOUT +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, Utils} @@ -47,7 +50,8 @@ private[deploy] class DriverRunner( val driverDesc: DriverDescription, val worker: RpcEndpointRef, val workerUrl: String, - val securityManager: SecurityManager) + val securityManager: SecurityManager, + val resources: Map[String, ResourceInformation] = Map.empty) extends Logging { @volatile private var process: Option[Process] = None @@ -171,6 +175,7 @@ private[deploy] class DriverRunner( private[worker] def prepareAndRunDriver(): Int = { val driverDir = createWorkingDirectory() val localJarFilename = downloadUserJar(driverDir) + val resourceFileOpt = prepareResourcesFile(SPARK_DRIVER_PREFIX, resources, driverDir) def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl @@ -178,9 +183,12 @@ private[deploy] class DriverRunner( case other => other } + // config resource file for driver, which would be used to load resources when driver starts up + val javaOpts = driverDesc.command.javaOpts ++ resourceFileOpt.map(f => + Seq(s"-D${DRIVER_RESOURCES_FILE.key}=${f.getAbsolutePath}")).getOrElse(Seq.empty) // TODO: If we add ability to submit multiple jars they should also be added here - val builder = CommandUtils.buildProcessBuilder(driverDesc.command, securityManager, - driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables) + val builder = CommandUtils.buildProcessBuilder(driverDesc.command.copy(javaOpts = javaOpts), + securityManager, driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables) runDriver(builder, driverDir, driverDesc.supervise) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 6f1484cee586e..97939107f3057 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -27,8 +27,11 @@ import com.google.common.io.Files import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.deploy.StandaloneResourceUtils.prepareResourcesFile import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SPARK_EXECUTOR_PREFIX import org.apache.spark.internal.config.UI._ +import org.apache.spark.resource.{ResourceInformation, ResourceUtils} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender @@ -54,7 +57,8 @@ private[deploy] class ExecutorRunner( val workerUrl: String, conf: SparkConf, val appLocalDirs: Seq[String], - @volatile var state: ExecutorState.Value) + @volatile var state: ExecutorState.Value, + val resources: Map[String, ResourceInformation] = Map.empty) extends Logging { private val fullId = appId + "/" + execId @@ -143,11 +147,14 @@ private[deploy] class ExecutorRunner( */ private def fetchAndRunExecutor() { try { + val resourceFileOpt = prepareResourcesFile(SPARK_EXECUTOR_PREFIX, resources, executorDir) // Launch the process + val arguments = appDesc.command.arguments ++ resourceFileOpt.map(f => + Seq("--resourcesFile", f.getAbsolutePath)).getOrElse(Seq.empty) val subsOpts = appDesc.command.javaOpts.map { Utils.substituteAppNExecIds(_, appId, execId.toString) } - val subsCommand = appDesc.command.copy(javaOpts = subsOpts) + val subsCommand = appDesc.command.copy(arguments = arguments, javaOpts = subsOpts) val builder = CommandUtils.buildProcessBuilder(subsCommand, new SecurityManager(conf), memory, sparkHome.getAbsolutePath, substituteVariables) val command = builder.command() 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 ac7a1b91db6b9..899593dff95fb 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 @@ -17,8 +17,7 @@ package org.apache.spark.deploy.worker -import java.io.File -import java.io.IOException +import java.io.{File, IOException} import java.text.SimpleDateFormat import java.util.{Date, Locale, UUID} import java.util.concurrent._ @@ -34,6 +33,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.ExternalShuffleService +import org.apache.spark.deploy.StandaloneResourceUtils._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging} @@ -44,7 +44,7 @@ import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.resource.ResourceInformation import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ -import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} +import org.apache.spark.util.{SignalUtils, SparkUncaughtExceptionHandler, ThreadUtils, Utils} private[deploy] class Worker( override val rpcEnv: RpcEnv, @@ -57,7 +57,8 @@ private[deploy] class Worker( val conf: SparkConf, val securityMgr: SecurityManager, resourceFileOpt: Option[String] = None, - externalShuffleServiceSupplier: Supplier[ExternalShuffleService] = null) + externalShuffleServiceSupplier: Supplier[ExternalShuffleService] = null, + pid: Int = Utils.getProcessId) extends ThreadSafeRpcEndpoint with Logging { private val host = rpcEnv.address.host @@ -180,7 +181,7 @@ private[deploy] class Worker( ) // visible for tests - private[deploy] var resources: Map[String, ResourceInformation] = _ + private[deploy] var resources: Map[String, ResourceInformation] = Map.empty var coresUsed = 0 var memoryUsed = 0 @@ -190,19 +191,8 @@ private[deploy] class Worker( private def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) - try { - // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() - // So attempting to create and then check if directory was created or not. - workDir.mkdirs() - if ( !workDir.exists() || !workDir.isDirectory) { - logError("Failed to create work directory " + workDir) - System.exit(1) - } - assert (workDir.isDirectory) - } catch { - case e: Exception => - logError("Failed to create work directory " + workDir, e) - System.exit(1) + if (!Utils.createDirectory(workDir)) { + System.exit(1) } } @@ -214,6 +204,7 @@ private[deploy] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() startExternalShuffleService() + releaseResourcesOnInterrupt() setupWorkerResources() webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() @@ -227,13 +218,29 @@ private[deploy] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } + /** + * Used to catch the TERM signal from sbin/stop-slave.sh and + * release resources before Worker exits + */ + private def releaseResourcesOnInterrupt(): Unit = { + SignalUtils.register("TERM") { + releaseResources(conf, SPARK_WORKER_PREFIX, resources, pid) + false + } + } + private def setupWorkerResources(): Unit = { try { - resources = getOrDiscoverAllResources(conf, SPARK_WORKER_PREFIX, resourceFileOpt) + val allResources = getOrDiscoverAllResources(conf, SPARK_WORKER_PREFIX, resourceFileOpt) + resources = acquireResources(conf, SPARK_WORKER_PREFIX, allResources, pid) + logResourceInfo(SPARK_WORKER_PREFIX, resources) } catch { case e: Exception => logError("Failed to setup worker resources: ", e) - System.exit(1) + releaseResources(conf, SPARK_WORKER_PREFIX, resources, pid) + if (!Utils.isTesting) { + System.exit(1) + } } } @@ -349,6 +356,7 @@ private[deploy] class Worker( TimeUnit.SECONDS)) } } else { + releaseResources(conf, SPARK_WORKER_PREFIX, resources, pid) logError("All masters are unresponsive! Giving up.") System.exit(1) } @@ -405,7 +413,8 @@ private[deploy] class Worker( cores, memory, workerWebUiUrl, - masterEndpoint.address)) + masterEndpoint.address, + resources)) } private def handleRegisterResponse(msg: RegisterWorkerResponse): Unit = synchronized { @@ -446,6 +455,7 @@ private[deploy] class Worker( case RegisterWorkerFailed(message) => if (!registered) { logError("Worker registration failed: " + message) + releaseResources(conf, SPARK_WORKER_PREFIX, resources, pid) System.exit(1) } @@ -506,15 +516,20 @@ private[deploy] class Worker( logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL) changeMaster(masterRef, masterWebUiUrl, masterRef.address) - val execs = executors.values. - map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state)) - masterRef.send(WorkerSchedulerStateResponse(workerId, execs.toList, drivers.keys.toSeq)) + val executorResponses = executors.values.map { e => + WorkerExecutorStateResponse(new ExecutorDescription( + e.appId, e.execId, e.cores, e.state), e.resources) + } + val driverResponses = drivers.keys.map { id => + WorkerDriverStateResponse(id, drivers(id).resources)} + masterRef.send(WorkerSchedulerStateResponse( + workerId, executorResponses.toList, driverResponses.toSeq)) case ReconnectWorker(masterUrl) => logInfo(s"Master with url $masterUrl requested this worker to reconnect.") registerWithMaster() - case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_) => + case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, resources_) => if (masterUrl != activeMasterUrl) { logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.") } else { @@ -567,7 +582,8 @@ private[deploy] class Worker( workerUri, conf, appLocalDirs, - ExecutorState.LAUNCHING) + ExecutorState.LAUNCHING, + resources_) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -601,7 +617,7 @@ private[deploy] class Worker( } } - case LaunchDriver(driverId, driverDesc) => + case LaunchDriver(driverId, driverDesc, resources_) => logInfo(s"Asked to launch driver $driverId") val driver = new DriverRunner( conf, @@ -611,7 +627,8 @@ private[deploy] class Worker( driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, workerUri, - securityMgr) + securityMgr, + resources_) drivers(driverId) = driver driver.start() @@ -701,6 +718,7 @@ private[deploy] class Worker( } override def onStop() { + releaseResources(conf, SPARK_WORKER_PREFIX, resources, pid) cleanupThreadExecutor.shutdownNow() metricsSystem.report() cancelLastRegistrationRetry() @@ -835,8 +853,9 @@ private[deploy] object Worker extends Logging { val securityMgr = new SecurityManager(conf) val rpcEnv = RpcEnv.create(systemName, host, port, conf, securityMgr) val masterAddresses = masterUrls.map(RpcAddress.fromSparkURL) + val pid = if (Utils.isTesting) workerNumber.get else Utils.getProcessId rpcEnv.setupEndpoint(ENDPOINT_NAME, new Worker(rpcEnv, webUiPort, cores, memory, - masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr, resourceFileOpt)) + masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr, resourceFileOpt, pid = pid)) rpcEnv } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 98e5aa6ec0c7c..a42a928936a8c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -90,11 +90,13 @@ private[spark] class CoarseGrainedExecutorBackend( // visible for testing def parseOrFindResources(resourcesFileOpt: Option[String]): Map[String, ResourceInformation] = { // only parse the resources if a task requires them - val resourceInfo = if (parseTaskResourceRequirements(env.conf).nonEmpty) { + val resourceInfo = if (parseResourceRequirements(env.conf, SPARK_TASK_PREFIX).nonEmpty) { val resources = getOrDiscoverAllResources(env.conf, SPARK_EXECUTOR_PREFIX, resourcesFileOpt) if (resources.isEmpty) { throw new SparkException("User specified resources per task via: " + s"$SPARK_TASK_PREFIX, but can't find any resources available on the executor.") + } else { + logResourceInfo(SPARK_EXECUTOR_PREFIX, resources) } resources } else { 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 e0147218d3eb7..214675b6cfd25 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 @@ -36,6 +36,23 @@ package object config { private[spark] val SPARK_EXECUTOR_PREFIX = "spark.executor" private[spark] val SPARK_TASK_PREFIX = "spark.task" + private[spark] val SPARK_RESOURCES_COORDINATE = + ConfigBuilder("spark.resources.coordinate.enable") + .doc("Whether to coordinate resources automatically among workers/drivers(client only) " + + "in Standalone. If false, the user is responsible for configuring different resources " + + "for workers/drivers that run on the same host.") + .booleanConf + .createWithDefault(true) + + private[spark] val SPARK_RESOURCES_DIR = + ConfigBuilder("spark.resources.dir") + .doc("Directory used to coordinate resources among workers/drivers(client only) in " + + "Standalone. Default is SPARK_HOME. Make sure to use the same directory for worker " + + "and drivers in client mode that run on the same host. Don't clean up this directory " + + "while workers/drivers are still alive to avoid the most likely resources conflict. ") + .stringConf + .createOptional + private[spark] val DRIVER_RESOURCES_FILE = ConfigBuilder("spark.driver.resourcesFile") .internal() diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala new file mode 100644 index 0000000000000..719f34db9e18b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -0,0 +1,96 @@ +/* + * 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.resource + +import scala.collection.mutable + +import org.apache.spark.SparkException +import org.apache.spark.util.collection.OpenHashMap + +/** + * Class used to help executor/worker allocate resources + * Please note that this class is intended to be used in a single thread. + * @param name Resource name, e.g. gpu/fpga + * @param addresses Resource addresses provided by the executor/worker + */ +class ResourceAllocator(name: String, addresses: Seq[String]) extends Serializable { + /** + * Map from an address to its availability, the value `true` means the address is available, + * while value `false` means the address is assigned. + * TODO Use [[OpenHashMap]] instead to gain better performance. + */ + private val addressAvailabilityMap = mutable.HashMap(addresses.map(_ -> true): _*) + + /** + * Sequence of currently available resource addresses. + */ + def availableAddrs: Seq[String] = addressAvailabilityMap.flatMap { case (addr, available) => + if (available) Some(addr) else None + }.toSeq + + /** + * Sequence of currently assigned resource addresses. + * Exposed for testing only. + */ + private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap + .flatMap { case (addr, available) => + if (!available) Some(addr) else None + }.toSeq + + /** + * Acquire a sequence of resource addresses (to a launched task), these addresses must be + * available. When the task finishes, it will return the acquired resource addresses. + * Throw an Exception if an address is not available or doesn't exist. + */ + def acquire(addrs: Seq[String]): Unit = { + addrs.foreach { address => + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to acquire an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) + if (isAvailable) { + addressAvailabilityMap(address) = false + } else { + throw new SparkException(s"Try to acquire an address that is not available. $name " + + s"address $address is not available.") + } + } + } + + /** + * Release a sequence of resource addresses, these addresses must have been assigned. Resource + * addresses are released when a task has finished. + * Throw an Exception if an address is not assigned or doesn't exist. + */ + def release(addrs: Seq[String]): Unit = { + addrs.foreach { address => + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to release an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) + if (!isAvailable) { + addressAvailabilityMap(address) = true + } else { + throw new SparkException(s"Try to release an address that is not assigned. $name " + + s"address $address is not assigned.") + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 69265861a9316..150ba09f77dd9 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -27,7 +27,6 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ import org.apache.spark.util.Utils.executeAndGetOutput /** @@ -48,7 +47,7 @@ private[spark] case class ResourceRequest( discoveryScript: Option[String], vendor: Option[String]) -private[spark] case class TaskResourceRequirement(resourceName: String, amount: Int) +private[spark] case class ResourceRequirement(resourceName: String, amount: Int) /** * Case class representing allocated resource addresses for a specific resource. @@ -62,7 +61,6 @@ private[spark] case class ResourceAllocation(id: ResourceID, addresses: Seq[Stri } private[spark] object ResourceUtils extends Logging { - // config suffixes val DISCOVERY_SCRIPT = "discoveryScript" val VENDOR = "vendor" @@ -94,23 +92,39 @@ private[spark] object ResourceUtils extends Logging { } } - def parseTaskResourceRequirements(sparkConf: SparkConf): Seq[TaskResourceRequirement] = { - parseAllResourceRequests(sparkConf, SPARK_TASK_PREFIX).map { request => - TaskResourceRequirement(request.id.resourceName, request.amount) + def parseResourceRequirements(sparkConf: SparkConf, componentName: String) + : Seq[ResourceRequirement] = { + parseAllResourceRequests(sparkConf, componentName).map { request => + ResourceRequirement(request.id.resourceName, request.amount) + } + } + + def resourcesMeetRequirements( + resourcesFree: Map[String, Int], + resourceRequirements: Seq[ResourceRequirement]) + : Boolean = { + resourceRequirements.forall { req => + resourcesFree.getOrElse(req.resourceName, 0) >= req.amount } } - private def parseAllocatedFromJsonFile(resourcesFile: String): Seq[ResourceAllocation] = { - implicit val formats = DefaultFormats + def withResourcesJson[T](resourcesFile: String)(extract: String => Seq[T]): Seq[T] = { val json = new String(Files.readAllBytes(Paths.get(resourcesFile))) try { - parse(json).extract[Seq[ResourceAllocation]] + extract(json) } catch { case NonFatal(e) => throw new SparkException(s"Error parsing resources file $resourcesFile", e) } } + def parseAllocatedFromJsonFile(resourcesFile: String): Seq[ResourceAllocation] = { + withResourcesJson[ResourceAllocation](resourcesFile) { json => + implicit val formats = DefaultFormats + parse(json).extract[Seq[ResourceAllocation]] + } + } + private def parseAllocatedOrDiscoverResources( sparkConf: SparkConf, componentName: String, @@ -154,10 +168,14 @@ private[spark] object ResourceUtils extends Logging { val allocations = parseAllocatedOrDiscoverResources(sparkConf, componentName, resourcesFileOpt) assertAllResourceAllocationsMeetRequests(allocations, requests) val resourceInfoMap = allocations.map(a => (a.id.resourceName, a.toResourceInformation)).toMap + resourceInfoMap + } + + def logResourceInfo(componentName: String, resources: Map[String, ResourceInformation]) + : Unit = { logInfo("==============================================================") - logInfo(s"Resources for $componentName:\n${resourceInfoMap.mkString("\n")}") + logInfo(s"Resources for $componentName:\n${resources.mkString("\n")}") logInfo("==============================================================") - resourceInfoMap } // visible for test @@ -175,7 +193,7 @@ private[spark] object ResourceUtils extends Logging { "doesn't exist!") } } else { - throw new SparkException(s"User is expecting to use resource: $resourceName but " + + throw new SparkException(s"User is expecting to use resource: $resourceName, but " + "didn't specify a discovery script!") } if (!result.name.equals(resourceName)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index c75931d53b4be..f05281e50b058 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -17,85 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable - -import org.apache.spark.SparkException -import org.apache.spark.util.collection.OpenHashMap +import org.apache.spark.resource.ResourceAllocator /** * Class to hold information about a type of Resource on an Executor. This information is managed * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the * information. - * Please note that this class is intended to be used in a single thread. * @param name Resource name * @param addresses Resource addresses provided by the executor */ -private[spark] class ExecutorResourceInfo( - val name: String, - addresses: Seq[String]) extends Serializable { - - /** - * Map from an address to its availability, the value `true` means the address is available, - * while value `false` means the address is assigned. - * TODO Use [[OpenHashMap]] instead to gain better performance. - */ - private val addressAvailabilityMap = mutable.HashMap(addresses.map(_ -> true): _*) - - /** - * Sequence of currently available resource addresses. - */ - def availableAddrs: Seq[String] = addressAvailabilityMap.flatMap { case (addr, available) => - if (available) Some(addr) else None - }.toSeq - - /** - * Sequence of currently assigned resource addresses. - * Exposed for testing only. - */ - private[scheduler] def assignedAddrs: Seq[String] = addressAvailabilityMap - .flatMap { case (addr, available) => - if (!available) Some(addr) else None - }.toSeq - - /** - * Acquire a sequence of resource addresses (to a launched task), these addresses must be - * available. When the task finishes, it will return the acquired resource addresses. - * Throw an Exception if an address is not available or doesn't exist. - */ - def acquire(addrs: Seq[String]): Unit = { - addrs.foreach { address => - if (!addressAvailabilityMap.contains(address)) { - throw new SparkException(s"Try to acquire an address that doesn't exist. $name address " + - s"$address doesn't exist.") - } - val isAvailable = addressAvailabilityMap(address) - if (isAvailable) { - addressAvailabilityMap(address) = false - } else { - throw new SparkException(s"Try to acquire an address that is not available. $name " + - s"address $address is not available.") - } - } - } - - /** - * Release a sequence of resource addresses, these addresses must have been assigned. Resource - * addresses are released when a task has finished. - * Throw an Exception if an address is not assigned or doesn't exist. - */ - def release(addrs: Seq[String]): Unit = { - addrs.foreach { address => - if (!addressAvailabilityMap.contains(address)) { - throw new SparkException(s"Try to release an address that doesn't exist. $name address " + - s"$address doesn't exist.") - } - val isAvailable = addressAvailabilityMap(address) - if (!isAvailable) { - addressAvailabilityMap(address) = true - } else { - throw new SparkException(s"Try to release an address that is not assigned. $name " + - s"address $address is not assigned.") - } - } - } -} +private[spark] class ExecutorResourceInfo(name: String, addresses: Seq[String]) + extends ResourceAllocator(name, addresses) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2e3e0a2f34079..1496dff31a4dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -94,7 +94,7 @@ private[spark] class TaskSchedulerImpl( val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK) // Resources to request per task - val resourcesReqsPerTask = ResourceUtils.parseTaskResourceRequirements(sc.conf) + val resourcesReqsPerTask = ResourceUtils.parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX) // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. Protected by `this` @@ -383,9 +383,8 @@ private[spark] class TaskSchedulerImpl( * Check whether the resources from the WorkerOffer are enough to run at least one task. */ private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = { - resourcesReqsPerTask.forall { req => - resources.contains(req.resourceName) && resources(req.resourceName).size >= req.amount - } + val resourcesFree = resources.map(r => r._1 -> r._2.length) + ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index e0605fee9cbf2..2025a7dc24821 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,6 +28,7 @@ import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientL import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.resource.ResourceUtils import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -112,8 +113,11 @@ private[spark] class StandaloneSchedulerBackend( } else { None } + val executorResourceReqs = ResourceUtils.parseResourceRequirements(conf, + config.SPARK_EXECUTOR_PREFIX) val appDesc = ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - webUrl, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) + webUrl, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit, + resourceReqsPerExecutor = executorResourceReqs) client = new StandaloneAppClient(sc.env.rpcEnv, masters, appDesc, this, conf) client.start() launcherBackend.setState(SparkAppHandle.State.SUBMITTED) 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 3ad67f44afa68..9c1f21fa236ba 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -269,6 +269,26 @@ private[spark] object Utils extends Logging { file.setExecutable(true, true) } + /** + * Create a directory given the abstract pathname + * @return true, if the directory is successfully created; otherwise, return false. + */ + def createDirectory(dir: File): Boolean = { + try { + // This sporadically fails - not sure why ... !dir.exists() && !dir.mkdirs() + // So attempting to create and then check if directory was created or not. + dir.mkdirs() + if ( !dir.exists() || !dir.isDirectory) { + logError(s"Failed to create directory " + dir) + } + dir.isDirectory + } catch { + case e: Exception => + logError(s"Failed to create directory " + dir, e) + false + } + } + /** * Create a directory inside the given parent directory. The directory is guaranteed to be * newly created, and is not marked for automatic deletion. @@ -2554,6 +2574,28 @@ private[spark] object Utils extends Logging { new File(path.getAbsolutePath + "." + UUID.randomUUID()) } + /** + * Given a process id, return true if the process is still running. + */ + def isProcessRunning(pid: Int): Boolean = { + val process = executeCommand(Seq("kill", "-0", pid.toString)) + process.waitFor(10, TimeUnit.SECONDS) + process.exitValue() == 0 + } + + /** + * Returns the pid of this JVM process. + */ + def getProcessId: Int = { + val PROCESS = "(\\d+)@(.*)".r + val name = getProcessName() + name match { + case PROCESS(pid, _) => pid.toInt + case _ => + throw new SparkException(s"Unexpected process name: $name, expected to be PID@hostname.") + } + } + /** * Returns the name of this JVM process. This is OS dependent but typically (OSX, Linux, Windows), * this is formatted as PID@hostname. diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 202b85dcf5695..9f00131c8dc20 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -440,7 +440,8 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.set(TASK_GPU_ID.amountConf, "2") conf.set(TASK_FPGA_ID.amountConf, "1") var taskResourceRequirement = - parseTaskResourceRequirements(conf).map(req => (req.resourceName, req.amount)).toMap + parseResourceRequirements(conf, SPARK_TASK_PREFIX) + .map(req => (req.resourceName, req.amount)).toMap assert(taskResourceRequirement.size == 2) assert(taskResourceRequirement(GPU) == 2) @@ -450,7 +451,8 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // Ignore invalid prefix conf.set(ResourceID("spark.invalid.prefix", FPGA).amountConf, "1") taskResourceRequirement = - parseTaskResourceRequirements(conf).map(req => (req.resourceName, req.amount)).toMap + parseResourceRequirements(conf, SPARK_TASK_PREFIX) + .map(req => (req.resourceName, req.amount)).toMap assert(taskResourceRequirement.size == 1) assert(taskResourceRequirement.get(FPGA).isEmpty) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index fed3ae35ee0e7..c1402bd2915a8 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -756,7 +756,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val conf = new SparkConf() .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") - conf.set(DRIVER_GPU_ID.amountConf, "1") + conf.set(DRIVER_GPU_ID.amountConf, "2") conf.set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) sc = new SparkContext(conf) @@ -783,7 +783,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu .set(DRIVER_RESOURCES_FILE, resourcesFile) .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") - conf.set(DRIVER_GPU_ID.amountConf, "1") + conf.set(DRIVER_GPU_ID.amountConf, "3") conf.set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) sc = new SparkContext(conf) @@ -850,26 +850,27 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assume(!(Utils.isWindows)) withTempDir { dir => val discoveryScript = createTempScriptWithExpectedOutput(dir, "resourceDiscoveryScript", - """{"name": "gpu","addresses":["0", "1", "2"]}""") + """{"name": "gpu","addresses":["0", "1", "2", "3", "4", "5", "6", "7", "8"]}""") val conf = new SparkConf() .setMaster("local-cluster[3, 3, 1024]") .setAppName("test-cluster") - conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(WORKER_GPU_ID.amountConf, "3") + conf.set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) + conf.set(TASK_GPU_ID.amountConf, "3") conf.set(EXECUTOR_GPU_ID.amountConf, "3") - conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, discoveryScript) sc = new SparkContext(conf) // Ensure all executors has started TestUtils.waitUntilExecutorsUp(sc, 3, 60000) - val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => + val rdd = sc.makeRDD(1 to 10, 3).mapPartitions { it => val context = TaskContext.get() context.resources().get(GPU).get.addresses.iterator } val gpus = rdd.collect() - assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) + assert(gpus.sorted === Seq("0", "1", "2", "3", "4", "5", "6", "7", "8")) eventually(timeout(10.seconds)) { assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) diff --git a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala index 784981ef99cd0..a2c466931f0c1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy import java.io.File -import java.util.Date import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} @@ -50,7 +49,8 @@ private[deploy] object DeployTestUtils { createDriverDesc(), JsonConstants.submitDate) def createWorkerInfo(): WorkerInfo = { - val workerInfo = new WorkerInfo("id", "host", 8080, 4, 1234, null, "http://publicAddress:80") + val workerInfo = new WorkerInfo("id", "host", 8080, 4, 1234, null, + "http://publicAddress:80", Map.empty) workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis workerInfo } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index f19e99894644c..9ce046a2e2f50 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -42,6 +42,8 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Deploy._ import org.apache.spark.internal.config.UI._ import org.apache.spark.internal.config.Worker._ +import org.apache.spark.resource.{ResourceInformation, ResourceRequirement} +import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer @@ -68,17 +70,23 @@ class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extend }) } - val appDesc = DeployTestUtils.createAppDesc() + var appDesc = DeployTestUtils.createAppDesc() val drivers = mutable.HashSet[String]() + val driverResources = new mutable.HashMap[String, Map[String, Set[String]]] + val execResources = new mutable.HashMap[String, Map[String, Set[String]]] override def receive: PartialFunction[Any, Unit] = { case RegisteredWorker(masterRef, _, _, _) => masterRef.send(WorkerLatestState(id, Nil, drivers.toSeq)) - case LaunchDriver(driverId, desc) => + case LaunchExecutor(_, appId, execId, _, _, _, resources_) => + execResources(appId + "/" + execId) = resources_.map(r => (r._1, r._2.addresses.toSet)) + case LaunchDriver(driverId, desc, resources_) => drivers += driverId + driverResources(driverId) = resources_.map(r => (r._1, r._2.addresses.toSet)) master.send(RegisterApplication(appDesc, newDriver(driverId))) case KillDriver(driverId) => master.send(DriverStateChanged(driverId, DriverState.KILLED, None)) drivers -= driverId + driverResources.remove(driverId) driverIdToAppId.get(driverId) match { case Some(appId) => apps.remove(appId) @@ -93,7 +101,7 @@ class MockExecutorLaunchFailWorker(master: RpcEndpointRef, conf: SparkConf = new extends MockWorker(master, conf) { var failedCnt = 0 override def receive: PartialFunction[Any, Unit] = { - case LaunchExecutor(_, appId, execId, _, _, _) => + case LaunchExecutor(_, appId, execId, _, _, _, _) => failedCnt += 1 master.send(ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None)) case otherMsg => super.receive(otherMsg) @@ -167,7 +175,8 @@ class MasterSuite extends SparkFunSuite cores = 0, memory = 0, endpoint = null, - webUiAddress = "http://localhost:80" + webUiAddress = "http://localhost:80", + Map.empty ) val (rpcEnv, _, _) = @@ -248,9 +257,12 @@ class MasterSuite extends SparkFunSuite // Application state should be WAITING when "MasterChangeAcknowledged" event executed. fakeAppInfo.state should be(ApplicationState.WAITING) } - - master.self.send( - WorkerSchedulerStateResponse(fakeWorkerInfo.id, fakeExecutors, Seq(fakeDriverInfo.id))) + val execResponse = fakeExecutors.map(exec => + WorkerExecutorStateResponse(exec, Map.empty[String, ResourceInformation])) + val driverResponse = WorkerDriverStateResponse( + fakeDriverInfo.id, Map.empty[String, ResourceInformation]) + master.self.send(WorkerSchedulerStateResponse( + fakeWorkerInfo.id, execResponse, Seq(driverResponse))) eventually(timeout(5.seconds), interval(100.milliseconds)) { getState(master) should be(RecoveryState.ALIVE) @@ -545,6 +557,16 @@ class MasterSuite extends SparkFunSuite _master } + def makeAliveMaster(conf: SparkConf = new SparkConf): Master = { + val master = makeMaster(conf) + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") + } + master + } + private def makeAppInfo( memoryPerExecutorMb: Int, coresPerExecutor: Option[Int] = None, @@ -563,7 +585,8 @@ class MasterSuite extends SparkFunSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - new WorkerInfo(workerId, "host", 100, cores, memoryMb, endpointRef, "http://localhost:80") + new WorkerInfo(workerId, "host", 100, cores, memoryMb, + endpointRef, "http://localhost:80", Map.empty) } private def scheduleExecutorsOnWorkers( @@ -575,13 +598,7 @@ class MasterSuite extends SparkFunSuite } test("SPARK-13604: Master should ask Worker kill unknown executors and drivers") { - val master = makeMaster() - master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") - } - + val master = makeAliveMaster() val killedExecutors = new ConcurrentLinkedQueue[(String, Int)]() val killedDrivers = new ConcurrentLinkedQueue[String]() val fakeWorker = master.rpcEnv.setupEndpoint("worker", new RpcEndpoint { @@ -614,13 +631,7 @@ class MasterSuite extends SparkFunSuite } test("SPARK-20529: Master should reply the address received from worker") { - val master = makeMaster() - master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") - } - + val master = makeAliveMaster() @volatile var receivedMasterAddress: RpcAddress = null val fakeWorker = master.rpcEnv.setupEndpoint("worker", new RpcEndpoint { override val rpcEnv: RpcEnv = master.rpcEnv @@ -647,13 +658,7 @@ class MasterSuite extends SparkFunSuite } test("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") { - val master = makeMaster() - master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") - } - + val master = makeAliveMaster() var worker: MockExecutorLaunchFailWorker = null try { worker = new MockExecutorLaunchFailWorker(master.self) @@ -697,12 +702,7 @@ class MasterSuite extends SparkFunSuite test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) - val master = makeMaster(conf) - master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") - } + val master = makeAliveMaster(conf) var worker1: MockWorker = null var worker2: MockWorker = null try { @@ -770,6 +770,95 @@ class MasterSuite extends SparkFunSuite } } + test("assign/recycle resources to/from driver") { + val master = makeAliveMaster() + val masterRef = master.self + val resourceReqs = Seq(ResourceRequirement(GPU, 3), ResourceRequirement(FPGA, 3)) + val driver = DeployTestUtils.createDriverDesc().copy(resourceReqs = resourceReqs) + val driverId = masterRef.askSync[SubmitDriverResponse]( + RequestSubmitDriver(driver)).driverId.get + var status = masterRef.askSync[DriverStatusResponse](RequestDriverStatus(driverId)) + assert(status.state === Some(DriverState.SUBMITTED)) + val worker = new MockWorker(masterRef) + worker.rpcEnv.setupEndpoint(s"worker", worker) + val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "2")), + FPGA -> new ResourceInformation(FPGA, Array("f1", "f2", "f3"))) + val regMsg = RegisterWorker(worker.id, "localhost", 7077, worker.self, 10, 1024, + "http://localhost:8080", RpcAddress("localhost", 10000), resources) + masterRef.send(regMsg) + eventually(timeout(10.seconds)) { + status = masterRef.askSync[DriverStatusResponse](RequestDriverStatus(driverId)) + assert(status.state === Some(DriverState.RUNNING)) + assert(worker.drivers.head === driverId) + assert(worker.driverResources(driverId) === Map(GPU -> Set("0", "1", "2"), + FPGA -> Set("f1", "f2", "f3"))) + val workerResources = master.workers.head.resources + assert(workerResources(GPU).availableAddrs.length === 0) + assert(workerResources(GPU).assignedAddrs.toSet === Set("0", "1", "2")) + assert(workerResources(FPGA).availableAddrs.length === 0) + assert(workerResources(FPGA).assignedAddrs.toSet === Set("f1", "f2", "f3")) + } + val driverFinished = DriverStateChanged(driverId, DriverState.FINISHED, None) + masterRef.send(driverFinished) + eventually(timeout(10.seconds)) { + val workerResources = master.workers.head.resources + assert(workerResources(GPU).availableAddrs.length === 3) + assert(workerResources(GPU).assignedAddrs.toSet === Set()) + assert(workerResources(FPGA).availableAddrs.length === 3) + assert(workerResources(FPGA).assignedAddrs.toSet === Set()) + } + } + + test("assign/recycle resources to/from executor") { + + def makeWorkerAndRegister( + master: RpcEndpointRef, + workerResourceReqs: Map[String, Int] = Map.empty) + : MockWorker = { + val worker = new MockWorker(master) + worker.rpcEnv.setupEndpoint(s"worker", worker) + val resources = workerResourceReqs.map { case (rName, amount) => + val shortName = rName.charAt(0) + val addresses = (0 until amount).map(i => s"$shortName$i").toArray + rName -> new ResourceInformation(rName, addresses) + } + val reg = RegisterWorker(worker.id, "localhost", 8077, worker.self, 10, 2048, + "http://localhost:8080", RpcAddress("localhost", 10000), resources) + master.send(reg) + worker + } + + val master = makeAliveMaster() + val masterRef = master.self + val resourceReqs = Seq(ResourceRequirement(GPU, 3), ResourceRequirement(FPGA, 3)) + val worker = makeWorkerAndRegister(masterRef, Map(GPU -> 6, FPGA -> 6)) + worker.appDesc = worker.appDesc.copy(resourceReqsPerExecutor = resourceReqs) + val driver = DeployTestUtils.createDriverDesc().copy(resourceReqs = resourceReqs) + val driverId = masterRef.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)).driverId + val status = masterRef.askSync[DriverStatusResponse](RequestDriverStatus(driverId.get)) + assert(status.state === Some(DriverState.RUNNING)) + val workerResources = master.workers.head.resources + eventually(timeout(10.seconds)) { + assert(workerResources(GPU).availableAddrs.length === 0) + assert(workerResources(FPGA).availableAddrs.length === 0) + assert(worker.driverResources.size === 1) + assert(worker.execResources.size === 1) + val driverResources = worker.driverResources.head._2 + val execResources = worker.execResources.head._2 + val gpuAddrs = driverResources(GPU).union(execResources(GPU)) + val fpgaAddrs = driverResources(FPGA).union(execResources(FPGA)) + assert(gpuAddrs === Set("g0", "g1", "g2", "g3", "g4", "g5")) + assert(fpgaAddrs === Set("f0", "f1", "f2", "f3", "f4", "f5")) + } + val appId = worker.apps.head._1 + masterRef.send(UnregisterApplication(appId)) + masterRef.send(DriverStateChanged(driverId.get, DriverState.FINISHED, None)) + eventually(timeout(10.seconds)) { + assert(workerResources(GPU).availableAddrs.length === 6) + assert(workerResources(FPGA).availableAddrs.length === 6) + } + } + private def getDrivers(master: Master): HashSet[DriverInfo] = { master.invokePrivate(_drivers()) } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 3d8a46bd02e1c..39607621b4c45 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -86,7 +86,8 @@ class PersistenceEngineSuite extends SparkFunSuite { cores = 0, memory = 0, endpoint = workerEndpoint, - webUiAddress = "http://localhost:80") + webUiAddress = "http://localhost:80", + Map.empty) persistenceEngine.addWorker(workerToPersist) 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 37e5fbcca46da..bb541b4cad8bd 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 @@ -23,6 +23,7 @@ import java.util.function.Supplier import scala.concurrent.duration._ +import org.json4s.{DefaultFormats, Extraction} import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.any @@ -32,11 +33,16 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} 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.StandaloneResourceUtils.{ALLOCATED_RESOURCES_FILE, SPARK_RESOURCES_COORDINATE_DIR} import org.apache.spark.deploy.master.DriverState import org.apache.spark.internal.config import org.apache.spark.internal.config.Worker._ +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.util.Utils @@ -51,17 +57,36 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { } def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts) + implicit val formats = DefaultFormats + private var _worker: Worker = _ private def makeWorker( - conf: SparkConf, - shuffleServiceSupplier: Supplier[ExternalShuffleService] = null): Worker = { + conf: SparkConf = new SparkConf(), + shuffleServiceSupplier: Supplier[ExternalShuffleService] = null, + pid: Int = Utils.getProcessId, + local: Boolean = false): Worker = { assert(_worker === null, "Some Worker's RpcEnv is leaked in tests") val securityMgr = new SecurityManager(conf) val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, securityMgr) - _worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "Worker", "/tmp", conf, securityMgr, None, shuffleServiceSupplier) - _worker + val resourcesFile = conf.get(SPARK_WORKER_RESOURCE_FILE) + val localWorker = new Worker(rpcEnv, 50000, 20, 1234 * 5, + Array.fill(1)(RpcAddress("1.2.3.4", 1234)), "Worker", "/tmp", + conf, securityMgr, resourcesFile, shuffleServiceSupplier, pid) + if (local) { + localWorker + } else { + _worker = localWorker + _worker + } + } + + private def assertResourcesFileDeleted(): Unit = { + assert(sys.props.contains("spark.test.home")) + val sparkHome = sys.props.get("spark.test.home") + val resourceFile = new File(sparkHome + "/" + SPARK_RESOURCES_COORDINATE_DIR, + ALLOCATED_RESOURCES_FILE) + assert(!resourceFile.exists()) } before { @@ -218,6 +243,141 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { } } + test("worker could be launched without any resources") { + val worker = makeWorker() + worker.rpcEnv.setupEndpoint("worker", worker) + eventually(timeout(10.seconds)) { + assert(worker.resources === Map.empty) + worker.rpcEnv.shutdown() + worker.rpcEnv.awaitTermination() + } + assertResourcesFileDeleted() + } + + test("worker could load resources from resources file while launching") { + val conf = new SparkConf() + withTempDir { dir => + val gpuArgs = ResourceAllocation(WORKER_GPU_ID, Seq("0", "1")) + val fpgaArgs = + ResourceAllocation(WORKER_FPGA_ID, Seq("f1", "f2", "f3")) + val ja = Extraction.decompose(Seq(gpuArgs, fpgaArgs)) + val f1 = createTempJsonFile(dir, "resources", ja) + conf.set(SPARK_WORKER_RESOURCE_FILE.key, f1) + conf.set(WORKER_GPU_ID.amountConf, "2") + conf.set(WORKER_FPGA_ID.amountConf, "3") + val worker = makeWorker(conf) + worker.rpcEnv.setupEndpoint("worker", worker) + eventually(timeout(10.seconds)) { + assert(worker.resources === Map(GPU -> gpuArgs.toResourceInformation, + FPGA -> fpgaArgs.toResourceInformation)) + worker.rpcEnv.shutdown() + worker.rpcEnv.awaitTermination() + } + assertResourcesFileDeleted() + } + } + + test("worker could load resources from discovery script while launching") { + val conf = new SparkConf() + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + conf.set(WORKER_FPGA_ID.discoveryScriptConf, scriptPath) + conf.set(WORKER_FPGA_ID.amountConf, "3") + val worker = makeWorker(conf) + worker.rpcEnv.setupEndpoint("worker", worker) + eventually(timeout(10.seconds)) { + assert(worker.resources === Map(FPGA -> + new ResourceInformation(FPGA, Array("f1", "f2", "f3")))) + worker.rpcEnv.shutdown() + worker.rpcEnv.awaitTermination() + } + assertResourcesFileDeleted() + } + } + + test("worker could load resources from resources file and discovery script while launching") { + val conf = new SparkConf() + withTempDir { dir => + val gpuArgs = ResourceAllocation(WORKER_GPU_ID, Seq("0", "1")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val resourcesPath = createTempJsonFile(dir, "resources", ja) + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + conf.set(SPARK_WORKER_RESOURCE_FILE.key, resourcesPath) + conf.set(WORKER_FPGA_ID.discoveryScriptConf, scriptPath) + conf.set(WORKER_FPGA_ID.amountConf, "3") + conf.set(WORKER_GPU_ID.amountConf, "2") + val worker = makeWorker(conf) + worker.rpcEnv.setupEndpoint("worker", worker) + eventually(timeout(10.seconds)) { + assert(worker.resources === Map(GPU -> gpuArgs.toResourceInformation, + FPGA -> new ResourceInformation(FPGA, Array("f1", "f2", "f3")))) + worker.rpcEnv.shutdown() + worker.rpcEnv.awaitTermination() + } + assertResourcesFileDeleted() + } + } + + test("Workers run on the same host should avoid resources conflict when coordinate is on") { + val conf = new SparkConf() + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3", "f4", "f5"]}""") + conf.set(WORKER_FPGA_ID.discoveryScriptConf, scriptPath) + conf.set(WORKER_FPGA_ID.amountConf, "2") + val workers = (0 until 3).map(id => makeWorker(conf, pid = id, local = true)) + workers.zipWithIndex.foreach{case (w, i) => w.rpcEnv.setupEndpoint(s"worker$i", w)} + eventually(timeout(20.seconds)) { + val (empty, nonEmpty) = workers.partition(_.resources.isEmpty) + assert(empty.length === 1) + assert(nonEmpty.length === 2) + val totalResources = nonEmpty.flatMap(_.resources(FPGA).addresses).toSet.toSeq.sorted + assert(totalResources === Seq("f1", "f2", "f3", "f4")) + workers.foreach(_.rpcEnv.shutdown()) + workers.foreach(_.rpcEnv.awaitTermination()) + } + assertResourcesFileDeleted() + } + } + + test("Workers run on the same host should load resources naively when coordinate is off") { + val conf = new SparkConf() + // disable coordination + conf.set(config.SPARK_RESOURCES_COORDINATE, false) + withTempDir { dir => + val gpuArgs = ResourceAllocation(WORKER_GPU_ID, Seq("g0", "g1")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val resourcesPath = createTempJsonFile(dir, "resources", ja) + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3", "f4", "f5"]}""") + conf.set(SPARK_WORKER_RESOURCE_FILE.key, resourcesPath) + conf.set(WORKER_GPU_ID.amountConf, "2") + conf.set(WORKER_FPGA_ID.discoveryScriptConf, scriptPath) + conf.set(WORKER_FPGA_ID.amountConf, "2") + val workers = (0 until 3).map(id => makeWorker(conf, pid = id, local = true)) + workers.zipWithIndex.foreach{case (w, i) => w.rpcEnv.setupEndpoint(s"worker$i", w)} + eventually(timeout(20.seconds)) { + val (empty, nonEmpty) = workers.partition(_.resources.isEmpty) + assert(empty.length === 0) + assert(nonEmpty.length === 3) + // Each Worker should get the same resources from resources file and discovery script + // without coordination. Note that, normally, we must config different resources + // for workers run on the same host when coordinate config is off. Test here is used + // to validate the different behaviour comparing to the above test when coordinate config + // is on, so we admit the resources collision here. + nonEmpty.foreach { worker => + assert(worker.resources === Map(GPU -> gpuArgs.toResourceInformation, + FPGA -> new ResourceInformation(FPGA, Array("f1", "f2", "f3", "f4", "f5")))) + } + workers.foreach(_.rpcEnv.shutdown()) + workers.foreach(_.rpcEnv.awaitTermination()) + } + assertResourcesFileDeleted() + } + } + test("cleanup non-shuffle files after executor exits when config " + "spark.storage.cleanupFilesAfterExecutorExit=true") { testCleanupFilesWithConfig(true) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 693b0ee1778b1..64d99a59b9192 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -157,7 +157,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("User is expecting to use resource: gpu but didn't specify a " + + assert(error.contains("User is expecting to use resource: gpu, but didn't specify a " + "discovery script!")) } } diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index 51a92e0a50f2f..c2ecc96db906b 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -253,7 +253,7 @@ class ResourceUtilsSuite extends SparkFunSuite discoverResource(request) }.getMessage() - assert(error.contains("User is expecting to use resource: gpu but " + + assert(error.contains("User is expecting to use resource: gpu, but " + "didn't specify a discovery script!")) } } diff --git a/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala index 6d2c07d89f5b6..c4509e93104d5 100644 --- a/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala +++ b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala @@ -18,14 +18,18 @@ package org.apache.spark.resource import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX, SPARK_TASK_PREFIX} +import org.apache.spark.internal.config.Worker.SPARK_WORKER_PREFIX import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} object TestResourceIDs { val DRIVER_GPU_ID = ResourceID(SPARK_DRIVER_PREFIX, GPU) val EXECUTOR_GPU_ID = ResourceID(SPARK_EXECUTOR_PREFIX, GPU) val TASK_GPU_ID = ResourceID(SPARK_TASK_PREFIX, GPU) + val WORKER_GPU_ID = ResourceID(SPARK_WORKER_PREFIX, GPU) val DRIVER_FPGA_ID = ResourceID(SPARK_DRIVER_PREFIX, FPGA) val EXECUTOR_FPGA_ID = ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) val TASK_FPGA_ID = ResourceID(SPARK_TASK_PREFIX, FPGA) + val WORKER_FPGA_ID = ResourceID(SPARK_WORKER_PREFIX, FPGA) + } diff --git a/docs/configuration.md b/docs/configuration.md index 57a53218f36e8..84545475ae33f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -193,6 +193,25 @@ of the most common options to set are: and spark.driver.memory. + + spark.resources.coordinate.enable + true + + Whether to coordinate resources automatically among workers/drivers(client only) + in Standalone. If false, the user is responsible for configuring different resources + for workers/drivers that run on the same host. + + + + spark.resources.dir + SPARK_HOME + + Directory used to coordinate resources among workers/drivers(client only) in Standalone. + Default is SPARK_HOME. Make sure to use the same directory for worker and drivers in + client mode that run on the same host. Don't clean up this directory while workers/drivers + are still alive to avoid the most likely resources conflict. + + spark.driver.resource.{resourceName}.amount 0 @@ -209,7 +228,9 @@ of the most common options to set are: A script for the driver to run to discover a particular resource type. This should write to STDOUT a JSON string in the format of the ResourceInformation class. This has a - name and an array of addresses. + name and an array of addresses. For a client-submitted driver in Standalone, discovery + script must assign different resource addresses to this driver comparing to workers' and + other dirvers' when spark.resources.coordinate.enable is off. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 2ca3ee6aa7213..bc77469b6664f 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -243,6 +243,37 @@ SPARK_MASTER_OPTS supports the following system properties: receives no heartbeats. + + spark.worker.resource.{resourceName}.amount + (none) + + Amount of a particular resource to use on the worker. + + + + spark.worker.resource.{resourceName}.discoveryScript + (none) + + Path to resource discovery script, which is used to find a particular resource while worker starting up. + And the output of the script should be formatted like the ResourceInformation class. + When spark.resources.coordinate.enable is off, the discovery script must assign different + resources for workers and drivers in client mode that run on the same host to avoid resource conflict. + + + + spark.worker.resourcesFile + (none) + + Path to resources file which is used to find various resources while worker starting up. + The content of resources file should be formatted like + [[{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2"]}]]. + When spark.resources.coordinate.enable is off, resources file must assign different + resources for workers and drivers in client mode that run on the same host to avoid resource conflict. + If a particular resource is not found in the resources file, the discovery script would be used to + find that resource. If the discovery script also does not find the resources, the worker will fail + to start up. + + SPARK_WORKER_OPTS supports the following system properties: diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index bcd5d06c1b67f..3f3150b0bd4ed 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -273,7 +273,8 @@ def setUp(self): self.tempFile.close() os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) - conf = SparkConf().set("spark.driver.resource.gpu.amount", "1") + conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.driver.resource.gpu.amount", "1") conf = conf.set("spark.driver.resource.gpu.discoveryScript", self.tempFile.name) self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 66357b61c79ee..66c5f9f3c2fd9 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -23,7 +23,7 @@ import unittest from pyspark import SparkConf, SparkContext, TaskContext, BarrierTaskContext -from pyspark.testing.utils import PySparkTestCase +from pyspark.testing.utils import PySparkTestCase, SPARK_HOME class TaskContextTests(PySparkTestCase): @@ -194,9 +194,11 @@ def setUp(self): self.tempFile.close() os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) - conf = SparkConf().set("spark.task.resource.gpu.amount", "1") + conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.worker.resource.gpu.discoveryScript", self.tempFile.name) + conf = conf.set("spark.worker.resource.gpu.amount", 1) + conf = conf.set("spark.task.resource.gpu.amount", "1") conf = conf.set("spark.executor.resource.gpu.amount", "1") - conf = conf.set("spark.executor.resource.gpu.discoveryScript", self.tempFile.name) self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) def test_resources(self): From 8b08e14de78382c8a63d0f612d68997b9a4c5765 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 9 Aug 2019 10:04:39 -0500 Subject: [PATCH 030/149] [SPARK-21481][ML][FOLLOWUP] HashingTF Cleanup ## What changes were proposed in this pull request? some cleanup and tiny optimization 1, since the `transformImpl` method in the .mllib side is no longer used in the .ml side, the scope should be limited; 2, in the `hashUDF`, val `numOfFeatures` is never used; 3, in the udf, it is inefficient to involve param getter (`$(numFeatures)`/`$(binary)`) directly or via method `indexOf` ((`$(numFeatures)`) . instead, the getter should be called outside of the udf; ## How was this patch tested? existing suites Closes #25324 from zhengruifeng/hashingtf_cleanup. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ml/feature/HashingTF.scala | 9 +++++---- .../scala/org/apache/spark/mllib/feature/HashingTF.scala | 2 +- .../org/apache/spark/ml/feature/HashingTFSuite.scala | 1 - 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index 0e6c43aec7a7a..b8ce9c3169a68 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -100,19 +100,20 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) + val localNumFeatures = $(numFeatures) + val localBinary = $(binary) + val hashUDF = udf { terms: Seq[_] => - val numOfFeatures = $(numFeatures) - val isBinary = $(binary) val termFrequencies = mutable.HashMap.empty[Int, Double].withDefaultValue(0.0) terms.foreach { term => val i = indexOf(term) - if (isBinary) { + if (localBinary) { termFrequencies(i) = 1.0 } else { termFrequencies(i) += 1.0 } } - Vectors.sparse($(numFeatures), termFrequencies.toSeq) + Vectors.sparse(localNumFeatures, termFrequencies.toSeq) } dataset.withColumn($(outputCol), hashUDF(col($(inputCol))), diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index d3b27e1808e15..90f6f203cb40e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -98,7 +98,7 @@ class HashingTF(val numFeatures: Int) extends Serializable { Vectors.sparse(numFeatures, seq) } - private[spark] def transformImpl(document: Iterable[_]): Seq[(Int, Double)] = { + private def transformImpl(document: Iterable[_]): Seq[(Int, Double)] = { val termFrequencies = mutable.HashMap.empty[Int, Double] val setTF = if (binary) (i: Int) => 1.0 else (i: Int) => termFrequencies.getOrElse(i, 0.0) + 1.0 val hashFunc: Any => Int = getHashFunction diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala index d65646e236f77..be70cf89cdb21 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala @@ -77,7 +77,6 @@ class HashingTFSuite extends MLTest with DefaultReadWriteTest { } test("indexOf method") { - val df = Seq((0, "a a b b c d".split(" ").toSeq)).toDF("id", "words") val n = 100 val hashingTF = new HashingTF() .setInputCol("words") From 924d794a6f5abb972fa07bf63adbb4ad544ef246 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 9 Aug 2019 11:18:50 -0700 Subject: [PATCH 031/149] [SPARK-28656][SQL] Support `millennium`, `century` and `decade` at `extract()` ## What changes were proposed in this pull request? In the PR, I propose new expressions `Millennium`, `Century` and `Decade`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT): 1. `millennium` - the current millennium for given date (or a timestamp implicitly casted to a date). For example, years in the 1900s are in the second millennium. The third millennium started _January 1, 2001_. 2. `century` - the current millennium for given date (or timestamp). The first century starts at 0001-01-01 AD. 3. `decade` - the current decade for given date (or timestamp). Actually, this is the year field divided by 10. Here are examples: ```sql spark-sql> SELECT EXTRACT(MILLENNIUM FROM DATE '1981-01-19'); 2 spark-sql> SELECT EXTRACT(CENTURY FROM DATE '1981-01-19'); 20 spark-sql> SELECT EXTRACT(DECADE FROM DATE '1981-01-19'); 198 ``` ## How was this patch tested? Added new tests to `DateExpressionsSuite` and uncommented existing tests in `pgSQL/date.sql`. Closes #25388 from MaxGekk/extract-ext2. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../expressions/datetimeExpressions.scala | 48 +++ .../sql/catalyst/parser/AstBuilder.scala | 6 + .../sql/catalyst/util/DateTimeUtils.scala | 20 +- .../expressions/DateExpressionsSuite.scala | 38 +++ .../resources/sql-tests/inputs/pgSQL/date.sql | 64 ++-- .../sql-tests/results/pgSQL/date.sql.out | 304 +++++++++++++++--- 6 files changed, 399 insertions(+), 81 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 43cef22da8341..4834907a3691b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1828,3 +1828,51 @@ case class MakeTimestamp( override def prettyName: String = "make_timestamp" } + +case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getMillennium(date.asInstanceOf[Int]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getMillennium($c)") + } +} + +case class Century(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getCentury(date.asInstanceOf[Int]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getCentury($c)") + } +} + +case class Decade(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getDecade(date.asInstanceOf[Int]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getDecade($c)") + } +} 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 49ca09d9ef076..7bbdd4f3c520e 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 @@ -1396,6 +1396,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { ctx.field.getText.toUpperCase(Locale.ROOT) match { + case "MILLENNIUM" => + Millennium(expression(ctx.source)) + case "CENTURY" => + Century(expression(ctx.source)) + case "DECADE" => + Decade(expression(ctx.source)) case "YEAR" => Year(expression(ctx.source)) case "QUARTER" => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 987b56d12c099..9e5aa2d8ea6a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.time._ -import java.time.temporal.{ChronoUnit, IsoFields} +import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -463,6 +463,24 @@ object DateTimeUtils { LocalDate.ofEpochDay(date).getDayOfYear } + private def extractFromYear(date: SQLDate, divider: Int): Int = { + val localDate = daysToLocalDate(date) + val yearOfEra = localDate.get(ChronoField.YEAR_OF_ERA) + var result = yearOfEra / divider + if ((yearOfEra % divider) != 0 || yearOfEra <= 1) result += 1 + if (localDate.get(ChronoField.ERA) == 0) result = -result + result + } + + /** Returns the millennium for the given date. The date is expressed in days since 1.1.1970. */ + def getMillennium(date: SQLDate): Int = extractFromYear(date, 1000) + + /** Returns the century for the given date. The date is expressed in days since 1.1.1970. */ + def getCentury(date: SQLDate): Int = extractFromYear(date, 100) + + /** Returns the decade for the given date. The date is expressed in days since 1.1.1970. */ + def getDecade(date: SQLDate): Int = Math.floorDiv(getYear(date), 10) + /** * Returns the year value for the given date. The date is expressed in days * since 1.1.1970. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 65f150e78bc5f..0f4627ea78649 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -968,4 +968,42 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-07-01 00:00:00")) checkEvaluation(makeTimestampExpr.copy(sec = Literal(60.5)), null) } + + test("millennium") { + val date = MakeDate(Literal(2019), Literal(1), Literal(1)) + checkEvaluation(Millennium(date), 3) + checkEvaluation(Millennium(date.copy(year = Literal(2001))), 3) + checkEvaluation(Millennium(date.copy(year = Literal(2000))), 2) + checkEvaluation(Millennium(date.copy(year = Literal(1001), day = Literal(28))), 2) + checkEvaluation(Millennium(date.copy(year = Literal(1))), 1) + checkEvaluation(Millennium(date.copy(year = Literal(-1))), -1) + checkEvaluation(Millennium(date.copy(year = Literal(-100), month = Literal(12))), -1) + checkEvaluation(Millennium(date.copy(year = Literal(-2019))), -3) + } + + test("century") { + val date = MakeDate(Literal(2019), Literal(1), Literal(1)) + checkEvaluation(Century(date), 21) + checkEvaluation(Century(date.copy(year = Literal(2001))), 21) + checkEvaluation(Century(date.copy(year = Literal(2000))), 20) + checkEvaluation(Century(date.copy(year = Literal(1001), day = Literal(28))), 11) + checkEvaluation(Century(date.copy(year = Literal(1))), 1) + checkEvaluation(Century(date.copy(year = Literal(-1))), -1) + checkEvaluation(Century(date.copy(year = Literal(-100), month = Literal(12))), -2) + checkEvaluation(Century(date.copy(year = Literal(-2019))), -21) + } + + test("decade") { + val date = MakeDate(Literal(2019), Literal(8), Literal(8)) + checkEvaluation(Decade(date), 201) + checkEvaluation(Decade(date.copy(year = Literal(2011))), 201) + checkEvaluation(Decade(date.copy(year = Literal(2010))), 201) + checkEvaluation(Decade(date.copy(year = Literal(2009))), 200) + checkEvaluation(Decade(date.copy(year = Literal(10))), 1) + checkEvaluation(Decade(date.copy(year = Literal(1))), 0) + checkEvaluation(Decade(date.copy(year = Literal(-1))), -1) + checkEvaluation(Decade(date.copy(year = Literal(-10))), -1) + checkEvaluation(Decade(date.copy(year = Literal(-11))), -2) + checkEvaluation(Decade(date.copy(year = Literal(-2019))), -202) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index 771bdc934ea55..f2c9e0b406e69 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -234,44 +234,44 @@ SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; -- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00'); -- 0 -- -- century --- --- SELECT EXTRACT(CENTURY FROM DATE '0101-12-31 BC'); -- -2 --- SELECT EXTRACT(CENTURY FROM DATE '0100-12-31 BC'); -- -1 --- SELECT EXTRACT(CENTURY FROM DATE '0001-12-31 BC'); -- -1 --- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 --- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 --- SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 --- SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 --- SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 --- SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 --- SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true --- + +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; + -- millennium --- --- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-12-31 BC'); -- -1 --- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 --- SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 --- SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 --- SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 --- SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 + +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- next test to be fixed on the turn of the next millennium;-) --- SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 --- +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); + -- decade --- --- SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 --- SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 --- SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 --- SELECT EXTRACT(DECADE FROM DATE '0001-01-01 BC'); -- 0 --- SELECT EXTRACT(DECADE FROM DATE '0002-12-31 BC'); -- -1 --- SELECT EXTRACT(DECADE FROM DATE '0011-01-01 BC'); -- -1 --- SELECT EXTRACT(DECADE FROM DATE '0012-12-31 BC'); -- -2 --- + +SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); +SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); +SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); + -- some other types: -- -- on a timestamp. --- SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; -- true --- SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20 +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- on an interval -- SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); -- 1 -- SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); -- 0 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 4dd2b4724688c..f5586c5a4aa3a 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 63 +-- Number of queries: 89 -- !query 0 @@ -502,128 +502,336 @@ struct -- !query 47 -SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query 47 schema -struct +struct -- !query 47 output -1001-01-01 00:07:02 +-2 -- !query 48 -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query 48 schema -struct +struct -- !query 48 output -1001-01-01 00:07:02 +-1 -- !query 49 -SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 49 schema -struct +struct -- !query 49 output -1901-01-01 00:00:00 +-1 -- !query 50 -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') -- !query 50 schema -struct +struct -- !query 50 output -1901-01-01 00:00:00 +1 -- !query 51 -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') -- !query 51 schema -struct +struct -- !query 51 output -2001-01-01 00:00:00 +1 -- !query 52 -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') -- !query 52 schema -struct +struct -- !query 52 output -0001-01-01 00:07:02 +19 -- !query 53 -SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') -- !query 53 schema -struct +struct -- !query 53 output --0099-01-01 00:07:02 +20 -- !query 54 -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') -- !query 54 schema -struct +struct -- !query 54 output -1990-01-01 00:00:00 +20 -- !query 55 -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') -- !query 55 schema -struct +struct -- !query 55 output -0000-01-01 00:07:02 +21 -- !query 56 -SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True -- !query 56 schema -struct +struct -- !query 56 output --0010-01-01 00:07:02 +true -- !query 57 -select make_date(2013, 7, 15) +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 57 schema -struct +struct -- !query 57 output -2013-07-15 +-1 -- !query 58 -select make_date(-44, 3, 15) +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') -- !query 58 schema -struct +struct -- !query 58 output --0044-03-15 +1 -- !query 59 -select make_date(2013, 2, 30) +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') -- !query 59 schema -struct +struct -- !query 59 output -NULL +1 -- !query 60 -select make_date(2013, 13, 1) +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') -- !query 60 schema -struct +struct -- !query 60 output -NULL +2 -- !query 61 -select make_date(2013, 11, -1) +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') -- !query 61 schema -struct +struct -- !query 61 output -NULL +2 -- !query 62 -DROP TABLE DATE_TBL +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') -- !query 62 schema -struct<> +struct -- !query 62 output +3 + + +-- !query 63 +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) +-- !query 63 schema +struct +-- !query 63 output +3 + + +-- !query 64 +SELECT EXTRACT(DECADE FROM DATE '1994-12-25') +-- !query 64 schema +struct +-- !query 64 output +199 + + +-- !query 65 +SELECT EXTRACT(DECADE FROM DATE '0010-01-01') +-- !query 65 schema +struct +-- !query 65 output +1 + + +-- !query 66 +SELECT EXTRACT(DECADE FROM DATE '0009-12-31') +-- !query 66 schema +struct +-- !query 66 output +0 + + +-- !query 67 +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) +-- !query 67 schema +struct +-- !query 67 output +0 + + +-- !query 68 +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +-- !query 68 schema +struct +-- !query 68 output +-1 + + +-- !query 69 +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) +-- !query 69 schema +struct +-- !query 69 output +-1 + + +-- !query 70 +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) +-- !query 70 schema +struct +-- !query 70 output +-2 + + +-- !query 71 +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True +-- !query 71 schema +struct +-- !query 71 output +true + + +-- !query 72 +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') +-- !query 72 schema +struct +-- !query 72 output +20 + + +-- !query 73 +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') +-- !query 73 schema +struct +-- !query 73 output +1001-01-01 00:07:02 + + +-- !query 74 +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') +-- !query 74 schema +struct +-- !query 74 output +1001-01-01 00:07:02 + + +-- !query 75 +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') +-- !query 75 schema +struct +-- !query 75 output +1901-01-01 00:00:00 + + +-- !query 76 +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') +-- !query 76 schema +struct +-- !query 76 output +1901-01-01 00:00:00 + + +-- !query 77 +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') +-- !query 77 schema +struct +-- !query 77 output +2001-01-01 00:00:00 + + +-- !query 78 +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') +-- !query 78 schema +struct +-- !query 78 output +0001-01-01 00:07:02 + + +-- !query 79 +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +-- !query 79 schema +struct +-- !query 79 output +-0099-01-01 00:07:02 + + +-- !query 80 +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +-- !query 80 schema +struct +-- !query 80 output +1990-01-01 00:00:00 + + +-- !query 81 +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +-- !query 81 schema +struct +-- !query 81 output +0000-01-01 00:07:02 + + +-- !query 82 +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +-- !query 82 schema +struct +-- !query 82 output +-0010-01-01 00:07:02 + + +-- !query 83 +select make_date(2013, 7, 15) +-- !query 83 schema +struct +-- !query 83 output +2013-07-15 + + +-- !query 84 +select make_date(-44, 3, 15) +-- !query 84 schema +struct +-- !query 84 output +-0044-03-15 + + +-- !query 85 +select make_date(2013, 2, 30) +-- !query 85 schema +struct +-- !query 85 output +NULL + + +-- !query 86 +select make_date(2013, 13, 1) +-- !query 86 schema +struct +-- !query 86 output +NULL + + +-- !query 87 +select make_date(2013, 11, -1) +-- !query 87 schema +struct +-- !query 87 output +NULL + + +-- !query 88 +DROP TABLE DATE_TBL +-- !query 88 schema +struct<> +-- !query 88 output From ef80c3226610cc69e04eaf800e00498c7c23c1fb Mon Sep 17 00:00:00 2001 From: Ajith Date: Fri, 9 Aug 2019 15:49:20 -0700 Subject: [PATCH 032/149] [SPARK-28676][CORE] Avoid Excessive logging from ContextCleaner ## What changes were proposed in this pull request? In high workload environments, ContextCleaner seems to have excessive logging at INFO level which do not give much information. In one Particular case we see that ``INFO ContextCleaner: Cleaned accumulator`` message is 25-30% of the generated logs. We can log this information for cleanup in DEBUG level instead. ## How was this patch tested? This do not modify any functionality. This is just changing cleanup log levels to DEBUG for ContextCleaner Closes #25396 from ajithme/logss. Authored-by: Ajith Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ContextCleaner.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index a111a60d1d024..24c83993b1b60 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -210,7 +210,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { logDebug("Cleaning RDD " + rddId) sc.unpersistRDD(rddId, blocking) listeners.asScala.foreach(_.rddCleaned(rddId)) - logInfo("Cleaned RDD " + rddId) + logDebug("Cleaned RDD " + rddId) } catch { case e: Exception => logError("Error cleaning RDD " + rddId, e) } @@ -223,7 +223,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { mapOutputTrackerMaster.unregisterShuffle(shuffleId) blockManagerMaster.removeShuffle(shuffleId, blocking) listeners.asScala.foreach(_.shuffleCleaned(shuffleId)) - logInfo("Cleaned shuffle " + shuffleId) + logDebug("Cleaned shuffle " + shuffleId) } catch { case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) } @@ -247,7 +247,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { logDebug("Cleaning accumulator " + accId) AccumulatorContext.remove(accId) listeners.asScala.foreach(_.accumCleaned(accId)) - logInfo("Cleaned accumulator " + accId) + logDebug("Cleaned accumulator " + accId) } catch { case e: Exception => logError("Error cleaning accumulator " + accId, e) } @@ -262,7 +262,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { logDebug("Cleaning rdd checkpoint data " + rddId) ReliableRDDCheckpointData.cleanCheckpoint(sc, rddId) listeners.asScala.foreach(_.checkpointCleaned(rddId)) - logInfo("Cleaned rdd checkpoint data " + rddId) + logDebug("Cleaned rdd checkpoint data " + rddId) } catch { case e: Exception => logError("Error cleaning rdd checkpoint data " + rddId, e) From 8535df72614800ba789286e569a39ea6e84b3354 Mon Sep 17 00:00:00 2001 From: younggyu chun Date: Sat, 10 Aug 2019 16:47:11 -0500 Subject: [PATCH 033/149] [MINOR] Fix typos in comments and replace an explicit type with <> ## What changes were proposed in this pull request? This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+. ## How was this patch tested? Manually tested. Closes #25338 from younggyuchun/younggyu. Authored-by: younggyu chun Signed-off-by: Sean Owen --- appveyor.yml | 2 +- .../apache/spark/network/ChunkFetchIntegrationSuite.java | 6 +++--- .../org/apache/spark/network/RpcIntegrationSuite.java | 8 ++++---- .../apache/spark/network/TransportClientFactorySuite.java | 2 +- .../network/shuffle/ExternalShuffleIntegrationSuite.java | 6 +++--- .../org/apache/spark/unsafe/types/UTF8StringSuite.java | 2 +- .../spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 4 ++-- .../spark/metrics/source/AccumulatorSourceSuite.scala | 2 +- .../sql/streaming/JavaStructuredSessionization.java | 2 +- .../streaming/kafka010/JavaConsumerStrategySuite.java | 2 +- .../streaming/kafka010/JavaDirectKafkaStreamSuite.java | 2 +- .../apache/spark/launcher/CommandBuilderUtilsSuite.java | 2 +- .../org/apache/spark/ml/stat/JavaSummarizerSuite.java | 2 +- .../spark/mllib/regression/JavaRidgeRegressionSuite.java | 4 ++-- .../spark/sql/execution/python/WindowInPandasExec.scala | 2 +- .../apache/spark/sql/JavaDataFrameReaderWriterSuite.java | 2 +- .../org/apache/spark/streaming/JavaMapWithStateSuite.java | 4 ++-- 17 files changed, 27 insertions(+), 27 deletions(-) diff --git a/appveyor.yml b/appveyor.yml index 8fb090cb0c603..b0e946c543cb3 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -55,7 +55,7 @@ build_script: environment: NOT_CRAN: true - # See SPARK-27848. Currently installing some dependent packagess causes + # See SPARK-27848. Currently installing some dependent packages causes # "(converted from warning) unable to identify current timezone 'C':" for an unknown reason. # This environment variable works around to test SparkR against a higher version. R_REMOTES_NO_ERRORS_FROM_WARNINGS: true diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 5999b6255b37b..a818fe46b9eff 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -151,9 +151,9 @@ private FetchResult fetchChunks(List chunkIndices) throws Exception { clientFactory.createClient(TestUtils.getLocalHost(), server.getPort())) { final Semaphore sem = new Semaphore(0); - res.successChunks = Collections.synchronizedSet(new HashSet()); - res.failedChunks = Collections.synchronizedSet(new HashSet()); - res.buffers = Collections.synchronizedList(new LinkedList()); + res.successChunks = Collections.synchronizedSet(new HashSet<>()); + res.failedChunks = Collections.synchronizedSet(new HashSet<>()); + res.buffers = Collections.synchronizedList(new LinkedList<>()); ChunkReceivedCallback callback = new ChunkReceivedCallback() { @Override 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 117f1e4d00fe3..498dc51cdc81a 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 @@ -175,8 +175,8 @@ private RpcResult sendRPC(String ... commands) throws Exception { final Semaphore sem = new Semaphore(0); final RpcResult res = new RpcResult(); - res.successMessages = Collections.synchronizedSet(new HashSet()); - res.errorMessages = Collections.synchronizedSet(new HashSet()); + res.successMessages = Collections.synchronizedSet(new HashSet<>()); + res.errorMessages = Collections.synchronizedSet(new HashSet<>()); RpcResponseCallback callback = new RpcResponseCallback() { @Override @@ -208,8 +208,8 @@ private RpcResult sendRpcWithStream(String... streams) throws Exception { TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); final Semaphore sem = new Semaphore(0); RpcResult res = new RpcResult(); - res.successMessages = Collections.synchronizedSet(new HashSet()); - res.errorMessages = Collections.synchronizedSet(new HashSet()); + res.successMessages = Collections.synchronizedSet(new HashSet<>()); + res.errorMessages = Collections.synchronizedSet(new HashSet<>()); for (String stream : streams) { int idx = stream.lastIndexOf('/'); diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index b4caa872928db..2aec4a33bbe43 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -84,7 +84,7 @@ private void testClientReuse(int maxConnections, boolean concurrent) try (TransportContext context = new TransportContext(conf, rpcHandler)) { TransportClientFactory factory = context.createClientFactory(); Set clients = Collections.synchronizedSet( - new HashSet()); + new HashSet<>()); AtomicInteger failed = new AtomicInteger(); Thread[] attempts = new Thread[maxConnections * 10]; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 61a58e9e456fd..9d398e372056b 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -170,9 +170,9 @@ private FetchResult fetchBlocks( TransportConf clientConf, int port) throws Exception { final FetchResult res = new FetchResult(); - res.successBlocks = Collections.synchronizedSet(new HashSet()); - res.failedBlocks = Collections.synchronizedSet(new HashSet()); - res.buffers = Collections.synchronizedList(new LinkedList()); + res.successBlocks = Collections.synchronizedSet(new HashSet<>()); + res.failedBlocks = Collections.synchronizedSet(new HashSet<>()); + res.buffers = Collections.synchronizedList(new LinkedList<>()); final Semaphore requestsRemaining = new Semaphore(0); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 0d441fb000331..cd253c0cbc904 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -467,7 +467,7 @@ public void translate() { ))); assertEquals( fromString("translate"), - fromString("translate").translate(new HashMap())); + fromString("translate").translate(new HashMap<>())); assertEquals( fromString("asae"), fromString("translate").translate(ImmutableMap.of( diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 88125a6b93ade..6b83a984f037c 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -533,7 +533,7 @@ public void testPeakMemoryUsed() throws Exception { long newPeakMemory; try { for (int i = 0; i < numRecordsPerPage * 10; i++) { - writer.insertRecordIntoSorter(new Tuple2(1, 1)); + writer.insertRecordIntoSorter(new Tuple2<>(1, 1)); newPeakMemory = writer.getPeakMemoryUsedBytes(); if (i % numRecordsPerPage == 0) { // The first page is allocated in constructor, another page will be allocated after @@ -550,7 +550,7 @@ public void testPeakMemoryUsed() throws Exception { newPeakMemory = writer.getPeakMemoryUsedBytes(); assertEquals(previousPeakMemory, newPeakMemory); for (int i = 0; i < numRecordsPerPage; i++) { - writer.insertRecordIntoSorter(new Tuple2(1, 1)); + writer.insertRecordIntoSorter(new Tuple2<>(1, 1)); } newPeakMemory = writer.getPeakMemoryUsedBytes(); assertEquals(previousPeakMemory, newPeakMemory); diff --git a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala index 45e6e0b4913ed..4262d26036505 100644 --- a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala @@ -66,7 +66,7 @@ class AccumulatorSourceSuite extends SparkFunSuite { assert(gauges.get("my-accumulator-2").getValue() == 456) } - test("the double accumulators value propety is checked when the gauge's value is requested") { + test("the double accumulators value property is checked when the gauge's value is requested") { val acc1 = new DoubleAccumulator() acc1.add(123.123) val acc2 = new DoubleAccumulator() diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java index 943e3d82f30ff..34ee235d8bad8 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java @@ -70,7 +70,7 @@ public static void main(String[] args) throws Exception { new FlatMapFunction() { @Override public Iterator call(LineWithTimestamp lineWithTimestamp) { - ArrayList eventList = new ArrayList(); + ArrayList eventList = new ArrayList<>(); for (String word : lineWithTimestamp.getLine().split(" ")) { eventList.add(new Event(word, lineWithTimestamp.getTimestamp())); } diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index 938cc8ddfb5d9..dc364aca9bd3b 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -42,7 +42,7 @@ public void testConsumerStrategyConstructors() { final Collection parts = Arrays.asList(tp1, tp2); final scala.collection.Iterable sParts = JavaConverters.collectionAsScalaIterableConverter(parts).asScala(); - final Map kafkaParams = new HashMap(); + final Map kafkaParams = new HashMap<>(); kafkaParams.put("bootstrap.servers", "not used"); final scala.collection.Map sKafkaParams = JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java index dc9c13ba863ff..2b8b1852fe687 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java @@ -152,7 +152,7 @@ public String call(ConsumerRecord r) { JavaDStream unifiedStream = stream1.union(stream2); - final Set result = Collections.synchronizedSet(new HashSet()); + final Set result = Collections.synchronizedSet(new HashSet<>()); unifiedStream.foreachRDD(new VoidFunction>() { @Override public void call(JavaRDD rdd) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index 9795041233b62..22d9324ba4b88 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -37,7 +37,7 @@ public void testValidOptionStrings() { testOpt(" a b c \\\\ ", Arrays.asList("a", "b", "c", "\\")); // Following tests ported from UtilsSuite.scala. - testOpt("", new ArrayList()); + testOpt("", new ArrayList<>()); testOpt("a", Arrays.asList("a")); testOpt("aaa", Arrays.asList("aaa")); testOpt("a b c", Arrays.asList("a", "b", "c")); diff --git a/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java index 38ab39aa0f492..163d2137a3a0e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java @@ -40,7 +40,7 @@ public class JavaSummarizerSuite extends SharedSparkSession { @Override public void setUp() throws IOException { super.setUp(); - List points = new ArrayList(); + List points = new ArrayList<>(); points.add(new LabeledPoint(0.0, Vectors.dense(1.0, 2.0))); points.add(new LabeledPoint(0.0, Vectors.dense(3.0, 4.0))); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index fb6c775a49612..5a9389c424b44 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -57,7 +57,7 @@ public void runRidgeRegressionUsingConstructor() { List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); JavaRDD testRDD = jsc.parallelize( - new ArrayList(data.subList(0, numExamples))); + new ArrayList<>(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); @@ -82,7 +82,7 @@ public void runRidgeRegressionUsingStaticMethods() { List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); JavaRDD testRDD = jsc.parallelize( - new ArrayList(data.subList(0, numExamples))); + new ArrayList<>(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala index 01ce07b133ffd..cad89dedb8b07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala @@ -251,7 +251,7 @@ case class WindowInPandasExec( } // Setting the window bounds argOffset for each UDF. For UDFs with bounded window, argOffset - // for the UDF is (lowerBoundOffet, upperBoundOffset, inputOffset1, inputOffset2, ...) + // for the UDF is (lowerBoundOffset, upperBoundOffset, inputOffset1, inputOffset2, ...) // For UDFs with unbounded window, argOffset is (inputOffset1, inputOffset2, ...) pyFuncs.indices.foreach { exprIndex => val frameIndex = expressionIndexToFrameIndex(exprIndex) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java index 7babf7573c075..d54be46c10604 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java @@ -62,7 +62,7 @@ public void testFormatAPI() { @Test public void testOptionsAPI() { - HashMap map = new HashMap(); + HashMap map = new HashMap<>(); map.put("e", "1"); spark .read() diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java index b1367b8f2aed2..2e00713b94860 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java @@ -149,10 +149,10 @@ private void testOperation( inputStream.map(x -> new Tuple2<>(x, 1))).mapWithState(mapWithStateSpec); List> collectedOutputs = - Collections.synchronizedList(new ArrayList>()); + Collections.synchronizedList(new ArrayList<>()); mapWithStateDStream.foreachRDD(rdd -> collectedOutputs.add(Sets.newHashSet(rdd.collect()))); List>> collectedStateSnapshots = - Collections.synchronizedList(new ArrayList>>()); + Collections.synchronizedList(new ArrayList<>()); mapWithStateDStream.stateSnapshots().foreachRDD(rdd -> collectedStateSnapshots.add(Sets.newHashSet(rdd.collect()))); BatchCounter batchCounter = new BatchCounter(ssc.ssc()); From dd5599efaf42de8d5ce7db68e35debc3b40bfb20 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 10 Aug 2019 16:51:12 -0500 Subject: [PATCH 034/149] [SPARK-28677][WEBUI] "Select All" checkbox in StagePage doesn't work properly ## What changes were proposed in this pull request? In StagePage, only the first optional column (Scheduler Delay, in this case) appears even though "Select All" checkbox is checked. ![Screenshot from 2019-08-09 18-46-05](https://user-images.githubusercontent.com/4736016/62771600-8f379e80-bad8-11e9-9faa-6da8d57739d2.png) The cause is that wrong method is used to manipulate multiple columns. columns should have been used but column was used. I've fixed this issue by replacing the `column` with `columns`. ## How was this patch tested? Confirmed behavior of the check-box. ![Screenshot from 2019-08-09 18-54-33](https://user-images.githubusercontent.com/4736016/62771614-98c10680-bad8-11e9-9cc0-5879ac47d1e1.png) Closes #25397 from sarutak/fix-stagepage.js. Authored-by: Kousuke Saruta Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/stagepage.js | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 4fe2cd95c673c..3ef1a76fd7202 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -323,7 +323,7 @@ $(document).ready(function () { "in this task. For SQL jobs, this only tracks all unsafe operators, broadcast joins, and " + "external sort."); $('[data-toggle="tooltip"]').tooltip(); - tasksSummary = $("#parent-container"); + var tasksSummary = $("#parent-container"); getStandAloneAppId(function (appId) { var endPoint = stageEndPoint(appId); @@ -346,7 +346,7 @@ $(document).ready(function () { } // prepare data for executor summary table - stageExecutorSummaryInfoKeys = Object.keys(responseBody.executorSummary); + var stageExecutorSummaryInfoKeys = Object.keys(responseBody.executorSummary); $.getJSON(createRESTEndPointForExecutorsPage(appId), function(executorSummaryResponse, status, jqXHR) { var executorDetailsMap = {}; @@ -877,7 +877,7 @@ $(document).ready(function () { { "visible": false, "targets": 16 }, { "visible": false, "targets": 17 }, { "visible": false, "targets": 18 } - ], + ] }; taskTableSelector = $(taskTable).DataTable(taskConf); $('#active-tasks-table_filter input').unbind(); @@ -897,14 +897,14 @@ $(document).ready(function () { // Get the column var para = $(this).attr('data-column'); if (para == "0") { - var column = taskTableSelector.column(optionalColumns); + var allColumns = taskTableSelector.columns(optionalColumns); if ($(this).is(":checked")) { $(".toggle-vis").prop('checked', true); - column.visible(true); + allColumns.visible(true); createDataTableForTaskSummaryMetricsTable(taskSummaryMetricsTableArray); } else { $(".toggle-vis").prop('checked', false); - column.visible(false); + allColumns.visible(false); var taskSummaryMetricsTableFilteredArray = taskSummaryMetricsTableArray.filter(row => row.checkboxId < 11); createDataTableForTaskSummaryMetricsTable(taskSummaryMetricsTableFilteredArray); From 47af8925b60509d2a2c932e2bcf25394721c6f12 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 10 Aug 2019 16:45:59 -0700 Subject: [PATCH 035/149] [SPARK-28675][SQL] Remove maskCredentials and use redactOptions ## What changes were proposed in this pull request? This PR replaces `CatalogUtils.maskCredentials` with `SQLConf.get.redactOptions` to match other redacts. ## How was this patch tested? unit test and manual tests: Before this PR: ```sql spark-sql> DESC EXTENDED test_spark_28675; id int NULL # Detailed Table Information Database default Table test_spark_28675 Owner root Created Time Fri Aug 09 08:23:17 GMT-07:00 2019 Last Access Wed Dec 31 17:00:00 GMT-07:00 1969 Created By Spark 3.0.0-SNAPSHOT Type MANAGED Provider org.apache.spark.sql.jdbc Location file:/user/hive/warehouse/test_spark_28675 Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Storage Properties [url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675] spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675'; default test_spark_28675 false Database: default Table: test_spark_28675 Owner: root Created Time: Fri Aug 09 08:23:17 GMT-07:00 2019 Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969 Created By: Spark 3.0.0-SNAPSHOT Type: MANAGED Provider: org.apache.spark.sql.jdbc Location: file:/user/hive/warehouse/test_spark_28675 Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Storage Properties: [url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675] Schema: root |-- id: integer (nullable = true) ``` After this PR: ```sql spark-sql> DESC EXTENDED test_spark_28675; id int NULL # Detailed Table Information Database default Table test_spark_28675 Owner root Created Time Fri Aug 09 08:19:49 GMT-07:00 2019 Last Access Wed Dec 31 17:00:00 GMT-07:00 1969 Created By Spark 3.0.0-SNAPSHOT Type MANAGED Provider org.apache.spark.sql.jdbc Location file:/user/hive/warehouse/test_spark_28675 Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Storage Properties [url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675] spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675'; default test_spark_28675 false Database: default Table: test_spark_28675 Owner: root Created Time: Fri Aug 09 08:19:49 GMT-07:00 2019 Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969 Created By: Spark 3.0.0-SNAPSHOT Type: MANAGED Provider: org.apache.spark.sql.jdbc Location: file:/user/hive/warehouse/test_spark_28675 Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Storage Properties: [url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675] Schema: root |-- id: integer (nullable = true) ``` Closes #25395 from wangyum/SPARK-28675. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../catalog/ExternalCatalogUtils.scala | 15 ------- .../sql/catalyst/catalog/interface.scala | 2 +- .../spark/sql/execution/datasources/ddl.scala | 5 ++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 42 +++++++++++++++++++ 4 files changed, 46 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 50f32e81d997d..4cff162c116a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -174,21 +174,6 @@ object ExternalCatalogUtils { } object CatalogUtils { - /** - * Masking credentials in the option lists. For example, in the sql plan explain output - * for JDBC data sources. - */ - def maskCredentials(options: Map[String, String]): Map[String, String] = { - options.map { - case (key, _) if key.toLowerCase(Locale.ROOT) == "password" => (key, "###") - case (key, value) - if key.toLowerCase(Locale.ROOT) == "url" && - value.toLowerCase(Locale.ROOT).contains("password") => - (key, "###") - case o => o - } - } - def normalizePartCols( tableName: String, tableCols: Seq[String], 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 d32d492177049..8d89baf689bf9 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 @@ -74,7 +74,7 @@ case class CatalogStorageFormat( inputFormat.foreach(map.put("InputFormat", _)) outputFormat.foreach(map.put("OutputFormat", _)) if (compressed) map.put("Compressed", "") - CatalogUtils.maskCredentials(properties) match { + SQLConf.get.redactOptions(properties) match { case props if props.isEmpty => // No-op case props => map.put("Storage Properties", props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 042320edea4f8..4022640224424 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -21,10 +21,11 @@ import java.util.Locale import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils} +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 import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -73,7 +74,7 @@ case class CreateTempViewUsing( userSpecifiedSchema.map(_ + " ").getOrElse("") + s"replace:$replace " + s"provider:$provider " + - CatalogUtils.maskCredentials(options) + SQLConf.get.redactOptions(options) } override def run(sparkSession: SparkSession): Seq[Row] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 9f7faa2baf562..158085c35351b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1056,6 +1056,48 @@ class JDBCSuite extends QueryTest } } + test("Replace CatalogUtils.maskCredentials with SQLConf.get.redactOptions") { + val password = "testPass" + val tableName = "tab1" + withTable(tableName) { + sql( + s""" + |CREATE TABLE $tableName + |USING org.apache.spark.sql.jdbc + |OPTIONS ( + | url '$urlWithUserAndPass', + | dbtable 'TEST.PEOPLE', + | user 'testUser', + | password '$password') + """.stripMargin) + + val storageProps = sql(s"DESC FORMATTED $tableName") + .filter("col_name = 'Storage Properties'") + .select("data_type").collect() + assert(storageProps.length === 1) + storageProps.foreach { r => + assert(r.getString(0).contains(s"url=${Utils.REDACTION_REPLACEMENT_TEXT}")) + assert(r.getString(0).contains(s"password=${Utils.REDACTION_REPLACEMENT_TEXT}")) + } + + val information = sql(s"SHOW TABLE EXTENDED LIKE '$tableName'") + .select("information").collect() + assert(information.length === 1) + information.foreach { r => + assert(r.getString(0).contains(s"url=${Utils.REDACTION_REPLACEMENT_TEXT}")) + assert(r.getString(0).contains(s"password=${Utils.REDACTION_REPLACEMENT_TEXT}")) + } + + val createTabStmt = sql(s"SHOW CREATE TABLE $tableName") + .select("createtab_stmt").collect() + assert(createTabStmt.length === 1) + createTabStmt.foreach { r => + assert(r.getString(0).contains(s"`url` '${Utils.REDACTION_REPLACEMENT_TEXT}'")) + assert(r.getString(0).contains(s"`password` '${Utils.REDACTION_REPLACEMENT_TEXT}'")) + } + } + } + test("SPARK 12941: The data type mapping for StringType to Oracle") { val oracleDialect = JdbcDialects.get("jdbc:oracle://127.0.0.1/db") assert(oracleDialect.getJDBCType(StringType). From 58cc0df59e3dd8b8fc46706050ac4ca437a3f18f Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 10 Aug 2019 17:01:15 -0700 Subject: [PATCH 036/149] [SPARK-28685][SQL][TEST] Test HMS 2.0.0+ in VersionsSuite/HiveClientSuites on JDK 11 ## What changes were proposed in this pull request? It seems Datanucleus 3.x can not support JDK 11: ```java [info] Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available. [info] at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215) [info] at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378) [info] at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392) [info] at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087) [info] at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247) ``` Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)). This PR makes it skip `0.12`, `0.13`, `0.14`, `1.0`, `1.1` and `1.2` when testing with JDK 11. Note that, this pr will not fix sql read hive materialized view. It's another issue: ``` 3.0: sql read hive materialized view *** FAILED *** (1 second, 521 milliseconds) 3.1: sql read hive materialized view *** FAILED *** (1 second, 536 milliseconds) ``` ## How was this patch tested? manual tests: ```shell export JAVA_HOME="/usr/lib/jdk-11.0.3" build/sbt "hive/test-only *.VersionsSuite *.HiveClientSuites" -Phive -Phadoop-3.2 ``` Closes #25405 from wangyum/SPARK-28685. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/client/HiveClientVersions.scala | 7 +++++-- .../apache/spark/sql/hive/client/VersionsSuite.scala | 12 +++++++++--- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala index e9eebb4d154fa..d48d70f7d4e77 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -19,10 +19,13 @@ package org.apache.spark.sql.hive.client import scala.collection.immutable.IndexedSeq -import org.apache.spark.SparkFunSuite +import org.apache.commons.lang3.{JavaVersion, SystemUtils} private[client] trait HiveClientVersions { - protected val versions = + protected val versions = if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + IndexedSeq("2.0", "2.1", "2.2", "2.3", "3.0", "3.1") + } else { IndexedSeq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3", "3.0", "3.1") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index feb364ec1947a..34bdee43ec07a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.client import java.io.{ByteArrayOutputStream, File, PrintStream, PrintWriter} import java.net.URI +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -102,8 +103,11 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } - private val versions = + private val versions = if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + Seq("2.0", "2.1", "2.2", "2.3", "3.0", "3.1") + } else { Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3", "3.0", "3.1") + } private var client: HiveClient = null @@ -323,7 +327,8 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: dropTable") { - val versionsWithoutPurge = versions.takeWhile(_ != "0.14") + val versionsWithoutPurge = + if (versions.contains("0.14")) versions.takeWhile(_ != "0.14") else Nil // First try with the purge option set. This should fail if the version is < 0.14, in which // case we check the version and try without it. try { @@ -478,7 +483,8 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: dropPartitions") { val spec = Map("key1" -> "1", "key2" -> "3") - val versionsWithoutPurge = versions.takeWhile(_ != "1.2") + val versionsWithoutPurge = + if (versions.contains("1.2")) versions.takeWhile(_ != "1.2") else Nil // Similar to dropTable; try with purge set, and if it fails, make sure we're running // with a version that is older than the minimum (1.2 in this case). try { From 31ef268baec98e5f9ce7e37ad03dd87a7476938e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 11 Aug 2019 08:13:19 -0500 Subject: [PATCH 037/149] [SPARK-28639][CORE][DOC] Configuration doc for Barrier Execution Mode ## What changes were proposed in this pull request? SPARK-24817 and SPARK-24819 introduced new 3 non-internal properties for barrier-execution mode but they are not documented. So I've added a section into configuration.md for barrier-mode execution. ## How was this patch tested? Built using jekyll and confirm the layout by browser. Closes #25370 from sarutak/barrier-exec-mode-conf-doc. Authored-by: Kousuke Saruta Signed-off-by: Sean Owen --- .../spark/internal/config/package.scala | 2 +- docs/configuration.md | 44 +++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) 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 214675b6cfd25..b898413ac8d76 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 @@ -1058,7 +1058,7 @@ package object config { ConfigBuilder("spark.barrier.sync.timeout") .doc("The timeout in seconds for each barrier() call from a barrier task. If the " + "coordinator didn't receive all the sync messages from barrier tasks within the " + - "configed time, throw a SparkException to fail all the tasks. The default value is set " + + "configured time, throw a SparkException to fail all the tasks. The default value is set " + "to 31536000(3600 * 24 * 365) so the barrier() call shall wait for one year.") .timeConf(TimeUnit.SECONDS) .checkValue(v => v > 0, "The value should be a positive time value.") diff --git a/docs/configuration.md b/docs/configuration.md index 84545475ae33f..aad496dc0acca 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2039,6 +2039,50 @@ Apart from these, the following properties are also available, and may be useful +### Barrier Execution Mode + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.barrier.sync.timeout365d + The timeout in seconds for each barrier() call from a barrier task. If the + coordinator didn't receive all the sync messages from barrier tasks within the + configured time, throw a SparkException to fail all the tasks. The default value is set + to 31536000(3600 * 24 * 365) so the barrier() call shall wait for one year. +
spark.scheduler.barrier.maxConcurrentTasksCheck.interval15s + Time in seconds to wait between a max concurrent tasks check failure and the next + check. A max concurrent tasks check ensures the cluster can launch more concurrent + tasks than required by a barrier stage on job submitted. The check can fail in case + a cluster has just started and not enough executors have registered, so we wait for a + little while and try to perform the check again. If the check fails more than a + configured max failure times for a job then fail current job submission. Note this + config only applies to jobs that contain one or more barrier stages, we won't perform + the check on non-barrier jobs. +
spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures40 + Number of max concurrent tasks check failures allowed before fail a job submission. + A max concurrent tasks check ensures the cluster can launch more concurrent tasks than + required by a barrier stage on job submitted. The check can fail in case a cluster + has just started and not enough executors have registered, so we wait for a little + while and try to perform the check again. If the check fails more than a configured + max failure times for a job then fail current job submission. Note this config only + applies to jobs that contain one or more barrier stages, we won't perform the check on + non-barrier jobs. +
+ ### Dynamic Allocation From 37eedf6149024de8ffbdd45ad43760dbe758a116 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 12 Aug 2019 14:43:32 +0900 Subject: [PATCH 038/149] [SPARK-28652][TESTS][K8S] Add python version check for executor ## What changes were proposed in this pull request? Current two PySpark version tests in PythonTestsSuite, just test against Python version at driver side. Because the test script doesn't run any spark job requiring python worker, it doesn't actually do version check at worker side. This patch adds pieces of code to the test script, to run a simple job to verify Python version. ## How was this patch tested? Unit test. Locally manual test. Closes #25411 from viirya/SPARK-28652. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../deploy/k8s/integrationtest/PythonTestsSuite.scala | 6 ++++-- .../kubernetes/integration-tests/tests/pyfiles.py | 8 ++++++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 904279923334f..3a6e5e7a6255d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -44,7 +44,8 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => mainClass = "", expectedLogOnCompletion = Seq( "Python runtime version check is: True", - "Python environment version check is: True"), + "Python environment version check is: True", + "Python runtime version check for executor is: True"), appArgs = Array("python"), driverPodChecker = doBasicDriverPyPodCheck, executorPodChecker = doBasicExecutorPyPodCheck, @@ -62,7 +63,8 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => mainClass = "", expectedLogOnCompletion = Seq( "Python runtime version check is: True", - "Python environment version check is: True"), + "Python environment version check is: True", + "Python runtime version check for executor is: True"), appArgs = Array("python3"), driverPodChecker = doBasicDriverPyPodCheck, executorPodChecker = doBasicExecutorPyPodCheck, diff --git a/resource-managers/kubernetes/integration-tests/tests/pyfiles.py b/resource-managers/kubernetes/integration-tests/tests/pyfiles.py index 4193654b49a12..ba55b75803276 100644 --- a/resource-managers/kubernetes/integration-tests/tests/pyfiles.py +++ b/resource-managers/kubernetes/integration-tests/tests/pyfiles.py @@ -20,6 +20,7 @@ import sys from pyspark.sql import SparkSession +from pyspark.sql.types import StringType if __name__ == "__main__": @@ -35,4 +36,11 @@ # Begin of Python container checks version_check(sys.argv[1], 2 if sys.argv[1] == "python" else 3) + # Check python executable at executors + spark.udf.register("get_sys_ver", + lambda: "%d.%d" % sys.version_info[:2], StringType()) + [row] = spark.sql("SELECT get_sys_ver()").collect() + driver_version = "%d.%d" % sys.version_info[:2] + print("Python runtime version check for executor is: " + str(row[0] == driver_version)) + spark.stop() From 6c06eea411fb9d6d1f631549c28e397b26d9e5b3 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 12 Aug 2019 02:24:48 -0700 Subject: [PATCH 039/149] [SPARK-28686][SQL][TEST] Move udf_radians from HiveCompatibilitySuite to HiveQuerySuite ## What changes were proposed in this pull request? This PR moves `udf_radians` from `HiveCompatibilitySuite` to `HiveQuerySuite` to make it easy to test with JDK 11 because it returns different value from JDK 9: ```java public class TestRadians { public static void main(String[] args) { System.out.println(java.lang.Math.toRadians(57.2958)); } } ``` ```sh [rootspark-3267648 ~]# javac TestRadians.java [rootspark-3267648 ~]# /usr/lib/jdk-9.0.4+11/bin/java TestRadians 1.0000003575641672 [rootspark-3267648 ~]# /usr/lib/jdk-11.0.3/bin/java TestRadians 1.0000003575641672 [rootspark-3267648 ~]# /usr/lib/jdk8u222-b10/bin/java TestRadians 1.000000357564167 ``` ## How was this patch tested? manual tests Closes #25417 from wangyum/SPARK-28686. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../hive/execution/HiveCompatibilitySuite.scala | 5 +++-- .../sql/hive/execution/HiveQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 12ac7066124ee..e7ff3a5f4be2b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -597,7 +597,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "correlationoptimizer4", "multiMapJoin1", "orc_dictionary_threshold", - "udf_hash" + "udf_hash", + // Moved to HiveQuerySuite + "udf_radians" ) private def commonWhiteList = Seq( @@ -1057,7 +1059,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_positive", "udf_pow", "udf_power", - "udf_radians", "udf_rand", "udf_regexp", "udf_regexp_extract", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6986963ecc089..53798e0ac2727 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -24,6 +24,7 @@ import java.util.{Locale, TimeZone} import scala.util.Try +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.BeforeAndAfter @@ -1207,6 +1208,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } } } + + // This test case is moved from HiveCompatibilitySuite to make it easy to test with JDK 11. + test("udf_radians") { + withSQLConf("hive.fetch.task.conversion" -> "more") { + val result = sql("select radians(57.2958) FROM src tablesample (1 rows)").collect() + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + assertResult(Array(Row(1.0000003575641672))) (result) + } else { + assertResult(Array(Row(1.000000357564167))) (result) + } + + assertResult(Array(Row(2.4999991485811655))) { + sql("select radians(143.2394) FROM src tablesample (1 rows)").collect() + } + } + } } // for SPARK-2180 test From e5f4a106dbed073ed8251c902685957bf43b9911 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 12 Aug 2019 03:37:10 -0700 Subject: [PATCH 040/149] [SPARK-28688][SQL][TEST] Skip `VersionsSuite.read hive materialized view` test for HMS 3.0+ on JDK9+ ## What changes were proposed in this pull request? This PR makes it skip test `read hive materialized view` since Hive 3.0 in `VersionsSuite.scala` on JDK 11 because [HIVE-19383](https://issues.apache.org/jira/browse/HIVE-19383) added [ArrayList$SubList](https://github.com/apache/hive/blob/ae4df627952610dbec029b099f0964908b3a4f25/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java#L383) which is incompatible with JDK 11: ```java java.lang.RuntimeException: java.lang.NoSuchFieldException: parentOffset at org.apache.hadoop.hive.ql.exec.SerializationUtilities$ArrayListSubListSerializer.(SerializationUtilities.java:389) at org.apache.hadoop.hive.ql.exec.SerializationUtilities$1.create(SerializationUtilities.java:235) ... ``` ![image](https://issues.apache.org/jira/secure/attachment/12977250/12977250_screenshot-2.png) ![image](https://issues.apache.org/jira/secure/attachment/12977249/12977249_screenshot-1.png) ## How was this patch tested? manual tests **Test on JDK 11**: ``` ... [info] - 2.3: sql read hive materialized view (1 second, 253 milliseconds) ... [info] - 3.0: sql read hive materialized view !!! CANCELED !!! (31 milliseconds) [info] "[3.0]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624) ... [info] - 3.1: sql read hive materialized view !!! CANCELED !!! (0 milliseconds) [info] "[3.1]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624) ... ``` **Test on JDK 1.8**: ``` ... [info] - 2.3: sql read hive materialized view (1 second, 444 milliseconds) ... [info] - 3.0: sql read hive materialized view (3 seconds, 100 milliseconds) ... [info] - 3.1: sql read hive materialized view (2 seconds, 941 milliseconds) ... ``` Closes #25414 from wangyum/SPARK-28688. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/hive/client/VersionsSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 34bdee43ec07a..da2acdc4aa378 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -620,6 +620,8 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: sql read hive materialized view") { // HIVE-14249 Since Hive 2.3.0, materialized view is supported. if (version == "2.3" || version == "3.0" || version == "3.1") { + // Since Hive 3.0(HIVE-19383), we can not run local MR by `client.runSqlHive` with JDK 11. + assume(version == "2.3" || !SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) // Since HIVE-18394(Hive 3.1), "Create Materialized View" should default to rewritable ones val disableRewrite = if (version == "2.3" || version == "3.0") "" else "DISABLE REWRITE" client.runSqlHive("CREATE TABLE materialized_view_tbl (c1 INT)") From 0f2efe6825a5b50b50bd1aeb8ee970fd190824fb Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 12 Aug 2019 19:15:00 +0800 Subject: [PATCH 041/149] [SPARK-28366][CORE][FOLLOW-UP] Refine logging in driver when loading single large unsplittable file ## What changes were proposed in this pull request? * Add log in `NewHadoopRDD` * Remove some words in logs which related to specific user API. ## How was this patch tested? Manual. Please review https://spark.apache.org/contributing.html before opening a pull request. Closes #25391 from WeichenXu123/log_sf. Authored-by: WeichenXu Signed-off-by: Wenchen Fan --- .../org/apache/spark/rdd/HadoopRDD.scala | 5 ++--- .../org/apache/spark/rdd/NewHadoopRDD.scala | 19 ++++++++++++++++++- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index eea3c697cf219..f3f9be3562922 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -25,7 +25,7 @@ import scala.collection.immutable.Map import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapred._ import org.apache.hadoop.mapred.lib.CombineFileSplit import org.apache.hadoop.mapreduce.TaskType @@ -215,8 +215,7 @@ class HadoopRDD[K, V]( val codecFactory = new CompressionCodecFactory(jobConf) if (Utils.isFileSplittable(path, codecFactory)) { logWarning(s"Loading one large file ${path.toString} with only one partition, " + - s"we can increase partition numbers by the `minPartitions` argument in method " + - "`sc.textFile`") + s"we can increase partition numbers for improving performance.") } else { logWarning(s"Loading one large unsplittable file ${path.toString} with only one " + s"partition, because the file is compressed by unsplittable compression codec.") diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 483de28d92ab7..a7a6cf43b14a0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -26,6 +26,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileInputFormat, FileSplit, InvalidInputException} @@ -38,7 +39,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager} +import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager, Utils} private[spark] class NewHadoopPartition( rddId: Int, @@ -133,6 +134,22 @@ class NewHadoopRDD[K, V]( } else { allRowSplits } + + if (rawSplits.length == 1 && rawSplits(0).isInstanceOf[FileSplit]) { + val fileSplit = rawSplits(0).asInstanceOf[FileSplit] + val path = fileSplit.getPath + if (fileSplit.getLength > conf.get(IO_WARNING_LARGEFILETHRESHOLD)) { + val codecFactory = new CompressionCodecFactory(_conf) + if (Utils.isFileSplittable(path, codecFactory)) { + logWarning(s"Loading one large file ${path.toString} with only one partition, " + + s"we can increase partition numbers for improving performance.") + } else { + logWarning(s"Loading one large unsplittable file ${path.toString} with only one " + + s"partition, because the file is compressed by unsplittable compression codec.") + } + } + } + val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { result(i) = From ae4edd54895c1f5a5737cd2f07d526527e9ee005 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Mon, 12 Aug 2019 08:36:01 -0500 Subject: [PATCH 042/149] [SPARK-28538][UI] Document SQL page MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? 1, add basic doc for each page; 2, doc SQL page with an exmple; ## How was this patch tested? locally built ![图片](https://user-images.githubusercontent.com/7322292/62421626-86f5f280-b6d7-11e9-8057-8be3a4afb611.png) ![图片](https://user-images.githubusercontent.com/7322292/62421634-9d9c4980-b6d7-11e9-8e31-1e6ba9b402e8.png) Closes #25349 from zhengruifeng/doc_ui_sql. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- docs/img/webui-sql-dag.png | Bin 0 -> 343391 bytes docs/img/webui-sql-plan.png | Bin 0 -> 458454 bytes docs/img/webui-sql-tab.png | Bin 0 -> 383887 bytes docs/monitoring.md | 2 +- docs/web-ui.md | 119 ++++++++++++++++++++++++++++++++++++ 5 files changed, 120 insertions(+), 1 deletion(-) create mode 100644 docs/img/webui-sql-dag.png create mode 100644 docs/img/webui-sql-plan.png create mode 100644 docs/img/webui-sql-tab.png create mode 100644 docs/web-ui.md diff --git a/docs/img/webui-sql-dag.png b/docs/img/webui-sql-dag.png new file mode 100644 index 0000000000000000000000000000000000000000..4ca21092e8b397fec9657f64c74b39bb3862b53a GIT binary patch literal 343391 zcmb4}19N3v*RI13I_}uEZQHhOr#rT7+eXK>?R0G0&bRZt=MS8kmD*K1YtNcmbBu9c zcbL4a7%VgFDYOEXmWhled0yeCdhE;V0e! z^Y7wMA&k7ZA$)-V>5iq0K?C8x@^?O*gOnl31+4J@_!X#LKA>T#T|R(esa^C}BJzm8 z0|JOKYB<3JIClf&p;FKrb$_-CxUNDNKN%=jD40E#ZWjtaI?z>GcaxeV_;(M%-p}mH zZU@sy`&(BHB6Xbz$&oz6R^}A8qc7{>ujCHxL3OzNd?{3g29?qS*v=U(@!NjtYB`5W=o@}S@tb<-)F9HI!2fXZX*r*WPA?%do6~x=|?UM#US9uMsQ#2R?4hU15X!ah`g;Lh3)Pz=i47jYk{)w9qt#m=7t=44AhJ!buxL-bNZ?ZrfaLN zD`-)aA0wBfAKbB$vmvvYMnpA{>pqX=T#k(EauhY~ik^PSp9gV`d0j=)3cnbZH5#s~ zUm>W(sI4tf|M?r3t8##-;yBL|1y(FV?H~cv4KSJE`2pqMikGX~t7oqQJZkvVid}fX zX35Uc(wQWb`gJ=QdLz4%&ql+w>pB~r!1CT=3pHga(u+^P2H9_w<9t7S?KqT8y_Rk& zf!UuSfrieeInU6pZ`8qicHp`MRZEw4#(n~OXE|^GGJ%=!W3l&rXaS|$hW8ghPlYrT zAQT5@(oYew=Na#@riZ8gVUi0{334XDvI5fNr?V|q3(p)Nw9Wnun+bulP1ypgCI{~d z{n~3qABqnNl}7|KfL_OMjc7K2b0Yu}M^B6yCZ-tYO#CwrAKDoA7FN-(QlLDpN}T7G z$q|7gI#Yx*E@Q;25Y5f6LyRFY7&F(j6o(V;LExV`a(bw&SpAG=13Ha}TCR#YP8)=c zU`)Z$Y=t@6D>6i#pK~^w-h2I@(f5XMY$w zsM!v!J-FDe1UugJFxvL`jio2rW?;?$-QLrU_xmqDS$Y!oU+<9GeaM2S2CR9J@)9co z?ELKDImNhrR)ldQ#H+tLcji+iCrGDAXvu5|FG+BTej~Mqff6H)rtV8a;}6EW7J?{Z zSP)sjpOBv*m4)0VSBf;@w@}vrwhzAz=xl>P_>? z{FRvuGlVs&IJYYIq7bKGqp-7Z{^a+)zB^{8ZHY!S-sv-zHAPy$g*ro!T7vRu2zAE{8QVdFRswE44o>SoL%dt zZD+l^+`Yu>?^lCYkPpQVn^&^?b`13X^j?E&j4k>mCRqk|n9v`OKePj^1E2%&U{_&Z zVIN?Ju}HB{F_zGASff9D>m0%KLw_QTr8iQ~XqnNPr1GQ!#y{#N>Sm3khFD{~Sf@<2 zS6deMWMkg;Pfya%+BP1VGP>4;n~|DTo6#<1FFl_ypQv!1W9(ysaA|O-hq+?VqPMs| zgUR}x;*XG~S*K%kH+30nL~C^Iu&$i1G_THhfABi+j(bf#BY6sWqP^nYdL11dsBJe- zU(xxXePMrXe9Doo$RWtC@aA##Q2L_PvgA_5qNL(Jd+Dpinjp}X(jE)cy42#kid`Eu z5piX7hI~*Dun40%;_t%_$1vqMQ6vA1f~0}BfuDz$g06sTN3%uAM%hHz#M{AOVB1wE zCSas=blG>Q#z>j2I(n45&t~`89VORP7F2dw2rw@(*Kj@~cAguMup-+CtBZ;oU>^2Y zw>@87X?J*ZcyZ{KC6$fN&}Mp3=+Juac%Fxf3`~P+|D{V7Ok6G9k+mONC~hgWB$1u0 zo%2pg!fYZVxt!>rbSTG>V;N5=Ya>BaL{f4gj+KhZ>hiIWZIL}cR;-g@%hzfC`VWJY z$lc(1>UzpjC=eblvsDgR;m z1epe~hE=`0h(NxiLQX1CH7eh9^u%WT2Pqk=nc0-Qt>on_UlTh?X$+#9Y%y#gHf1kij)vmXJzJbin+P~5IC$tk3iE!Ep zhxB^uslDCnUA)6xrx&NSLyajd?yFDF$M>qAvEUd#S5YF z3^{l|IO=kGt3GYdLrJ4rRdcUe?+9;^{Cm23n!DJ(bX-NDJKyF6d+@|*{L=N8RpVafxl^cf=W3%PzXC1P$&a<;W8JD{M zd|Amc-Jhh_=5LISV`!piqqLqFpH$z$ycOT?ho?{Y^w!WCYizowKZo?Jead)x*yT3^ znI|ny?NC|%G!`~F<7T?nRxNfv^wb#nY`Bfw%1;{f_fy5l$0XunaB6xwZ>Qd?Z{1Yx zZP+la+2*7=?v9O)lC8;!WO3eUJ=>AV*304X{IWls(de#+n10E$;=OX|I`KaW0`2zr z^v>O==sd}P8cc@Uie8$6$`Ydm*WvW$xWBbiYf+ohi)vkUqd8fPI-9Tlvw^Vg>d5qT zc&8QA{KJl}O{hi5TkXmAfM?#r<;(GXd6TxLy8lV&O3yX+diyt9_uGlkOFukMQOt1+ zC7u>{wa@P7zPz2Q-7ucg*RdO=7w3xFW65St<1}{m$F1769@=Zk{S$Wz51xD7d(q8! zR?SeiB~~a_%M<6-!*<4P^N`)2&O5h1uU=zs!vXbtvAoJ&&hIO)6sPl9c^!EK6mmmK zHsJK^GZWH4y60fN+0ajLI6(3x<!t(qTY_&FSeu*M&(F*ZU(o-{0Ky(u z%b`mW3^WV4QhCeVs!H*#IoSG%;q~@MB_uYQ1UtpW#ez%oA?ipW?pKrbCt&^bN4Yp;CV}|>? z)dsoc0zaWX2Cz6~K%@g2raj#Shkl0V8bx;lt?gU5-Y8D&h) zO&e7|Q4O4D|z6 zsd==9+Lo7}DBFJ{)-Huu4Sy5VX`lxdhmA{EP6zMi0>i_hVKO=XvA)RnU;pZ{-y<5) z`S~jweumv;cEi`hVAFc;dlF8#=6UfPa@K9#y=VH)s$1p~KjBJBN`wOIPttADzvuF) z!A<`;JvTeKA{PpiOJY;4*?3yx^KqtOmeshd>v7RAFKb{cs->y)-Cc?xyf*PRoS;h0 z{!I8^p{mBW%~x}jv=)7uB5{kIPm&w{fkmw~Z^^JmxdCai}$J?`XXY(dDLKW5o}6pXYw1 zoYw`t&)XEl`S#_`OBc4*QJHc#ujU0*#6@^7%zD-hyK2UQyqu@O>*>q6$2Gf}jK&kq zPUwED<>7X1Lb^o3tlsCh3*YXJVS9;?+6urN5@;h%!q2&Y}=!vqEtfW znOInYgcQ*ZpA1U}f3I-chduP{`f(ZY4`+XzoB5nb=`qUM!vserTr{m+-llZFH&(k; z1;Fz@r!RMyzUK5UP2ZfkRyebJ)ln*ikx#5d7_)2EFhYv_D9=}a=z8kB(z$X+o09W+ zoev9x>X8IW*FyWH-s*V3ag-^<4ov|+JKayLP~tErPdvMGe^#t|>%Ada@o>cV^%6=H&3-fR3s{4jpTOgB#VCer>tzbw>sTm- zPES^-TT;)ae8R4C?AN8%iW-Whtb`tK=7MD095ZqqGGc<|;0SxK_*64lpIN5E5S8vt z`1ihh%Vgd=pH($`_kiMFhl+9c#Z(*0b4?=l#3^^_x8}CRs^42)PZin+l&N6EBKt~3a|GmN*dTd(*v_Z%Wg@c}SbZr*-n+kY z*x#@FRh&yzQ3?hYeW_w%yzdL_Jom8FD%IoKp3870LgK?yAtzN}<6q?sNNV zisu5D3Y(HeTPV-j(9+(qXAxvst~Rc`pL{-+eQsC}reYT>)W$W;Nxk3Jwm!?`yiP&! zo_+Q7Hp=l3J?3o3Ga#Dq;i=JEvd)Gg3lm1290~Ew4~L>+MYVARGfHmw zTTddg%%#GUxcer`&i=0=Pbc8h^TJAREeM(UPm}y@3E#)bq@DNO9^RKLrwXxA)?guANyo{X$k^AXgiKY=p3`6Sxb5E^WeiM_-^B3h^fnh z^dOKp)m77s^TT0w|8&3ML@N?0=|&t2kkPqwO)^N}ei$JA!gyW=Y?rSTb8Os<^Y2D8 zw6IjKx^ln1Yp8*Ru59L#5!g}w3t$NBZTd3RNOj7k)5V3-tcnIyIVfd=A{A@iue+SD zcfdx8WG&`XelInZY1y&iF_fHZ-?aB56DImgq#5^Kea=T^tUqC&B6S!r9Mc)TeR)K= z5JLDVLC}e5WxN_!0ezKl7ppRhGLHh5U#aAo3KU_w5_o4n9GCV}+p<^hPhI>_ygI<` z=>4Z{K5Ip&zit@!?dt3E>bbE>^4^8D%}qbzy2rm6lyd_Xw15OuluT$O3Sm#IkC=%4 zI+(3vXM`w*IkjpPvg4u3?qgKWThKRtq_B;UF;7KU7|I(mlQ={XCUtKzUm|sunoN{m zmm}_v`kJ6rqoHU^Cb9Q`B^Bi32o+z*=xB-RKF$!3InuUMsbOllFfS+7MJzwEs9dB# zLYTT^COf6!t$A_;()*FmC)q;N3h`ZhN(Y$!Llm=DxZcM04>ozpZu*o(a~7C-*1%Gf zw}!*Ml&6;@3=cSL43B zdxX}y|Jou-i_=#cv1aui!3+%xytM21B!Q-I*i`AxVrWE*bIW!{kmAQ@b^A>zN}3+Z=e1e zGkt$;^Rda!>r9J~&wA5kS;ysw_D}22N#gm4RWxigjITSsFQ?m4vgOMh=Q#;@w-KTB z^3IJ5+ds|i8#I!R)J2ee{xGU=$9jo#g#4>eb| zC8Z6Tnhg(wJUgKr(vIRuv9#xKW?vsqfB14O`=#X`ZcYFJ#Oumq%l`Rm2M`{c)@-G2 zyT9JLH_}#|rg>OW5THrxl;n6+)U}r%H~#1;kk%F7lVG-dq#8xn@r6K;AC+>g@4&%( ztn_OKBF2*65S=!@6)m@KK3sY~wfmeK;k~UN@eVJXX&-S`=bP6WDT-f+vb7<)!5LGk zd=D{(_w^X?W*kweR?1t-WXMgSa68`}t+kKQ8R~B#zh!A`Oh$llYZ(Cos*8LPJk%XXgP29EUD(w7YIxJ{S(}2Ca zbYE^q`n91N!Cx+4ot!BM1no)CWe!Bb5L^fhgWm(@3}oEYI6#SFp}O3MM+x47qpBoO zu8KV-fKD&j?kGKLVCi#TfOjH`jFga?MJdVmz60+Bm-Xz}Qc2UN>Nyi`KpNol*6iaH zN+B{4N!SSp&Os%%KlCXTbq4mR^j@HY5SZQ!<}y2fSrCRSdWY2VS~8r;1VQc%%NnY6 z?4khfunbV8h5;tP8wmS#6XSEth%sLXMc0EPwLZNs7f+4&)e1O3*}7CLCSBJ-bPFt1 z!l?z)v4;U{ohLFZ>I0@51VV1u5y|IM3)SKIY-vFG!Zj>nVL6c!t1NXz;#CNv0n6tS zNpm?I5Z^&YwBmBg7#3n@Mm|u}5_g4>hrXR(H};0(G@mC=xxCYNPs z*~H@0I>yOsRm}v&GF;V3g8d#y`C>wepVCk5%$RPJHQ~@ z@UJV>p~;m4qxBqp>`hy#%?Qql)yozj5R`d{&c%bsDr#b3B$TrIRV@`|vnL2~>!&Qt zZu3xWHI!o#mQ*fQ*{6uCEWBSY(`v{PSo<~hK~#m|Wy;nYa=%`bV^Z5r{MXCZLGTC! z%(_h*it6Rp@&-Z@6Y>nmZ(x8q4=4m7*q9MzTWVM+yJ=zqt<@qicLFXsKPSjYS z?dhL@c$NSx*QF|EG-jjIoLtyji6Jp%{YAG%1LIj`~)j<~$FOg%|B3 z4S%}pj-3FP1@+B^{{-QYL2>d-bY4P9tzO!ddUix12_((bALv6_(Q_M-C$>}|W-jX}_UYeLMp!r+g{r4fDX<=lV!|DlQXGt_FyD^VFjPANY-VIy zwZy_?MUk?gFj8N-v^&63c5jQv!Dn%+C}zVB@@3+8$o7r@)Qg{5y`3Snx-mhRwfVNGE&Gm(fPcLGmYgcjf{?UdEEdJVgl7BP`6GRfRC!uBfBDi@?Pm z6j+!aHF zo!fM3Aw(?_nw$>NBn(f@Veb?!03r%Y)4%dOI^;W0lLP{yHcTIZjsY^TD`_-$!5g}b zkmHMO<7&1|!iG;X$-M?kVJ=T)SXRU=kMuFi%_k?~77oX;kMJge@ooLdmEQqM7U?Hz9+|)r_ks zWvK?FWCgUcw@2JZ9E4Sm76{R}y-{G$kWvDueJ+yx@FXY_mXLb3Uo6k;&)Wh0%Cl8X z%YeA5dvgQGiq+@i3El3UBmcqhDTx405n!17UJzI$mwqN$bP~$aSpHkW-;OuT6SG`LmZtev=pb2`j98Uo8FYyJ04}o|>JV|e7Xo{D z;Wf^Xbzf?_J^4-n!)z-aR#saLhgpzJU^sof*I;~F2VtMv2CNO-fw7^%{kT!#Ixg|r zoRj!YoEWYKTSB2E^9V&*Bjzl3t%;ldsM>mP2pN(0$C+L~b-^HsThqEGw7?xZ63^L`M9gX#d*8!EDMx2*Es{;fic6q#~7au&VVlL}s zukD>5)DY|(nC^{0c^1-Auu6<3v)a#owx>dlcjk*3ZU1Rmo<{7Z10t7s0eI%WsNoWQ z$9YCE>v@T)@f8(>jnC!6N$Xykake6>5A5waN&6M_#X^nx)fJhRh0Cb=4;6YUS%v)w$1AF*(^ZK{swAn1(H+kcs>7bN;=8Eg=XHFF zPFc|!FAI{gTH*VstIXR@f1}A{DYQGsg4)B!&A&MKMIF%@0YtCSq#H1QJ7@5 z!(cCGf=MO9gthoHI|`N9kNsi&hGZ1vXnf6G_HXG_UL_&KIrQd?LB%TWG+ z;ipKMVqaMHai)0Ulv;rB^(jP<$`7|DHLi%ei(x_f+7c7#7fUEig!V})kvxE7$B$+U zUMQ0L+J@;=09iSpGg<>(;zlVfsQJ^0=;C%y@+l93IUVPPh_2+BSZ`z|gm)N(6g7`2 zV<7IA&pgCx#;dX%BsL1K%ERKX^2hDqHJJnxnZ5=!^y z8D1uFSUqkI8Ja?R%oBkTnqhvZtfl3AXilXc`HiSodfWt*adX?XaLQ<%l==q;!#@B3{1ykmWB7LA)N0U5qpEa5U4`b`0p`eg5 zVm$`p{`;lw}HSVS@^^0_k8lhLIDrl0t(6AlEYw95dXlv1w|@~i3~Mxc|9qjbE@ z;|M7f9{Oz|;G?F7*~q5XLKH7~LQFwGkC^JcG*JNOLhm-aNHU*lAEl9o9-KyshQ}a> zYpB}x7O+Php3R)K+^RgvY;zQE22NO9isYpdWsAF_$X&ZFFX6+DjszEF)_D#?>bD(A=uhTmrpf@3;^)Sq&$c8432Yri@ds3m6` zp|P$p(H9L#WfHF@zyZc@P7&oFIl?X+SjHS&WFb9S7w%{ZW`MXbMiT-arxf@8UPx=2 zSRweAjymz&un#1elIovkA&Tj7E`p>zO*)i(DYI~ro?iR&epnc$hNKfOnHBjjb!#at zAkA%Ewu11oXLBs*m-*rpbiAO-gK5o9!Wu<8jE0Z?q8L@|auj%?Kz7~tT{N@9roqn4WT@6$N<4$oSoAbC&+KI-8PbWCG$&~ zP`HlPW}1D(zuxm-GV>@eYTdSx&in>}p5CX>K~phwFl29STTsa1ar`(}6iFstgi6B^ zM{2Le1EVJpwhTnxy%Bj7+0XXGidl8St{0&d1c*bhEAdeON{k;opy`^6*6ziN0j^!>yfdjJtksY zoo8;6T}=x}_RZxX;AKt-kqtSeeINimw}&6F`q|W$WEqkpyEWvqN?@KPpSogt@QeQ za%J>&MS7@REIQB)I6umg7cK`!_V1)B)QP_Nl&gL({5ErqjRtOMsgC#g<`PbY-7`9v zoGyGyL%?dh`tD;{Ftg@eLxv(VVJ4u6Wrl}w%dKh&!FD|1uC!Y5PshP0-LGm!8;>?+ zP316@0AauJI3Qx2yqBWO>XtcEx$Eisp79RvnxPdKYQ>8eGkl~iZX&jz+2i&|h$i0O zrt}R!n)i(gzo?J+2YV%t-9#qn-$bNNQ@n-Hmf!0NajeHOAd(jV#4u%{Q^xIo9J2U{ zzn!D?#A}tA^*@kDUG@#>$~-avslj)XxdQZ zfm(izXDEl>-7dS3LrD3)eU^7b@QZujiCP-nd^7PQ{OEmB zpltS^3>XzWPGXMk2i}q~F*CTgs(Xu~yTMoKB1CcBBsRbmT*gzYzx_LU=?@A+b#{x; zk*%my>i#l^t8$Iq1bY6_pLWY!rfJc=g^)sqH=1JiVE0NHAtBL6t*}+{L@^`*o2mnO zMX6{C5jT@Z6kTV|Dq^{U(FZCTRnrTPXQAM@;j+hA0=Itqkcb|Zh$I%|*o$Xs*7k<*ev~%rNced%HL@7fM=+=zF>CD`~Q*Rx|X|h>rZF1KZ;;K)Ca~+j(n^iHThUFb729to`;k{Njj&ktJFBC%^CrTdl$`A=#f-Qa%QOQjc3X z;c}{nkWg+J6MpPWDQPJbYKfJ$5P#+>9UkAp=4@;p;na6t5k(ItK_TK8k60~CLwHzH z7DAwyVjo&6V#PUOZp3FADWPyd0qo2vE}GBb)=;FH9>H3aYF>6KQH87m8#)q8rm<+K zD#3V%RX$NlNS&tz&@ruX(eM6UfwLT9PnMm(tJZN@h)wZ*y>I;<&AZY!LC6mu<_{dBBO}~36q-y zXXKb*wd^`wByUKV$?nj*`rnx;gG2k=k4t8?CzO!W$jh9|11aP-Ai3_7usfrHyo?SeHSMst|KNLOz6xZ$TqPYfoUb_lC4ItvH;oa%5fu!yL}djEt$H{+mkKSN zhjKNM0c$-0PXHHSVPH@=3ckjnbRVSB?PYbzxeg=~UL+l5={?-v-g^M>;0oNmX&;Ve?5J9f@arV@pl2H zy$LuE5|m#501yd4C}j$|PNhDh|8x|bTKM;T3wi*$@M+z3B4bxh6Gl8a?K;Ht4P4`h zg;F(@S{D0ZZS6tFsQ8M?Jpo+*^;E699IY5)BhN2#;OGiaB~NhPz=vRaoYytF3}BdL z-t?h$#cC-NIDNy+r?}=dTP~MxmDT%9Yb#&->B{qn3xGD|O3023?!U2&eYhoa$-fIb zACJakW^hgmiE=5j!*PK8`^}IRDKi188fyRqD$F{D0`+*Q$Lf6cBY-4M1U+)p=C?m$*20$ppe-n#2y^N7b;kAI%DJ96zj6jU(#xxVTH`A_qYJ> z<2Yk|pOo*p7rp@!nD4PJh5}q}>v@PRcRCz}DjT3wC%gb;)``AvYC%d+nyx3AUF~(7 zkmCuMw>#kPl>h*`j(DG1p8)=b;?i%N1vY1*v@JoFh0N@89e_6x1)#@PG;QC+D1+b= z01rdq={$FSR?BKLr3vIbbsW7ANMqF{_b?1&@2y5OrU#68hx)vv_y7cST_fn$ z)y3+=BPl0RHSCM@v^h?1Z^%~BhX~E0MFNL{`f-p<9?itcEF}h)ByRu~18s-dL)~D$ zN*Bp~nJ_~t_9H3Y0U+Xy>SS8Lp*Sl(4R@SNU(t5C=zQ9Qvue-Fd&If4hsYWf({dX) zFO9MRD8K5C@a$ix>5e}t9smq@{i*fa6T$cPZ@OCgU6ROGWD)NZ;054nEc^zDLL!ZZ z2X@530g*)jJrP$0w1RrK5}Hj>i1Kq6ty^Bm#CU z_)BUdNXMggD!4!=8~@C5CL^&njz#fVM7D!(^Z=C)j^}cS*5)vzR?+^H zkQaczCCCT67$@dRV%)1I87B5{r`8-G_VV;|)SoG7R{i zXN2HYW*@lh;}+bHxV9`hMJINMaBo5bO~A-|djWjMdc#coO|(S-Zq*iGCE|T3lWW8_ zg|bmCGw#7yi{u}d2E7_$>tZN2p(wrwI2UpZW@}tf{+X%|O^wJWDZ|ezg&5W(>uWkI z$eDUQ2C$w0-cQ%_lCS*)xvsU1OVlotb`@MwKzHl;%P0P%>Vz8T#V+sKw-LDXn|YUo ztE(!2D_aajW6&6?sG) zCG6e0KOU77;*5jV$|%xQ@8e*)xIOOy@3tLz&B$n4th=SgXFER8Qq3*ka@)*%!+7@1 ze#&e<{`T$c(~LUF?#@uBG-Mp^Gh30rf5RMEfV01)!mC=2Aew1KKe%d1o5C>}6rYWN zz+%c#Z7Z4H@u-a;iUQ3kR{&=9x;ET_q#_?BU%!qd#=OzNDk?y3j-;D%sG}%ZwZ}PG z8THTiizG2;U065F3^X>HO8k4xhAsC8wf>V+&YQMWZRg|aI0@^Tuv*(&S-00pKkks` z@*cf^lln+}6n4T=2T0eFEoTex+eVvlJUPT$Seh%f$Y8YJ&p@44$0G9U5Z-%&7JO2o zlb>{Pky1B8wc+j3Y6v-H`E2X?N|I0kSokAM0FT1-ttL)_ksQYXa--s)$uBe2K#(QR zPhSXbUt%6Dn?{tZ;|4(#Jr2()r|dm5$1o#G&d=vPaAtHN18Y<-Q{3Er)J!Vv9caho zaXJ75HJsaSw$D3k=GyiJU68SOEk#EB`Ibqr`Y-^nU$HL(6B3-@e`uSHIiLqA6Fh4S ze4#eCh%nb@PUe$L0TX!qf|)EYE#)MEJAf%A6G|;NdMy$j%}Q>q5>)?HR|NnE8N;9v zV3&~)q~R`_s$310^()A(;Km%|S-Tiyx9Rlt9RTx`wU8|Kl3*mziXwwh^aL624bR<-O%!5&}6W%>qKlGsoCL@(SrW1kbU6Ii~XHSc)Ka zvA=cTn!B_NaTd=2v1gcf11gv9OVPq$WE>8eo!`9JjP1TSS z2*5vlLZpc%@1eY}y9Hr%NJI~=IqGphzga_N~Gj0+ZEgbaW zrK3kxj`6gmmKfe6Wan5SG8DPEa%ty$EO#8TEE4JiAJ6%v1_L!zfLZbD_y$Bq_K9;6 zCp1B_Bm1&Kv`|2BqG%8sI^tjFgd0Qa7*v2!dKw??I9k1}AtSNPPn29a!6K z%@&3|g{+(WN=O;A>B)xN-XzE;LFjE9a)D!ZN2sN^aRfEYP%=r^n8UXjw!22@j_1m5 z{aBm&kA^&XeJm!UDBaDvPOg>?U}H!ZQ&cLBUueZoh76jl}QdQ9&C){YDlju6Zh@n3r;abK26 zXF_aIps3@5k|sxSCT*t?^jxCcR)AkYK-a;T;L11d*za^vR>jaVQSUfLAk{csKIQ%O zoGifsH^DjfSEap8L+myrj|I;U`9aE7p^Z8$VT7g4c5zB1^FFkB`~q4&A&C@5O$i;w zb&=vPrxkv;J}9!#aCPUO+K_q($=mpH(Ekcl_E96A)|2&U#{%yYv#!i)TRdocg01o?IaQ+u>yVX$!@0Bn!kH^SZGw9CvmZandd zb%#46VWU|n95>o`7RX}I0%2UGv<^KL2%h@|Vl=(jcR5`?+n{)m^>qADq;TCL1}ym9 zxjg=fp(i&s7dlB6!goFK?)trJz)bl60_h7$$ZSznMg8Nh8h*dPv_?vkbt06X*}g3* zLv<`UkdWewOTr`f1WtiP0%KEb6mtuW5ZJ^~J({unIrr3=Dx$JeR;_IL2+M!%5z_Fs4~W-mM*IDvenUI|x4;0-GZj!}X2;UTGN+$eA>x2$SmTdC1v#2Rqdk zY(syABKvOLNICMrxleVG~1u?#_J8r4@q z7gNMD(8UdB1{LhzD+XYJUhkLGXxjOd*?9wt!LXr$Hl)1qM8Q2#m3~kF_B?W)|2s&pRUyBN^z_V<2S>BK&1Z?NrYFqfr!y z(m=qWy|B#APDuP%nY43J>Tw#7Mb$R;wOS$>29>|SErB~#%QYuw@uZj;66YG`Ala1> zJ9*!TVlxYW+RWJR$BFeVsR0t&#Eebr=Yaai1?I@@{(EJLL^gr)c$zh4eGprkRar88 zeShJ0o(!8HFm6pl%m_DnHO3HN4Jn<^FcNRNDzayf`&qFJ^ukpo&vDNo#{?fPUgnW%rLS-(G8yj977QkmN2$^XJME`|fy7_(Qox1;X7wSL$eL+)9@ z4X^8G2bZ4&DTdnp4vW-{mJ8s=zbHlp_l$ ziW`>_BUh9ziD1c6V&;f4vWtSnpqfRV{=*W}8lwLYC6ccNf)S$z|J!4gi67L2oTj{f z&D^fcd$@c4&vma=@Go!x31O9ERJpBxoKS3(nuua)`VXn))BcBlIlmv4Ep@Pa@_v6H zuqJs-$;)b`j-D7GO};Bd2tJ>*F3aUK`BpxZWeJ{$S4|P0X|M?rbcJV7zOd*RIpAQ; z5l&({BJC>k!LXoJw2UKV^Eh~<7Bes7Q6Act8p7$|a89l($>=Q|?Yk`5OaMgGgZ~&v z0?P1693|r0D=1Ku1gVIEBDml}f6(t5)yk$IbWZHQmZhpZl>>s4>aiWFgwte=gOb$Z zDuGrTt7k5h674Splr|ctqD54`!f6HJxQ2=<>S2%zrxwS6+5|=FWXtfPz^uG-zEFg| zA_&FIXzhn)*-%Mqm+>+6#4OmodzLCi6#qqf{{vKwl8_NAyteaq;@#NRmB4c3Jhk`9 z9_A^jLN=39xe=<)v+ioxzZ$$Cs;)cii3yFTGrS`|7}abx8*ky@a4}WGlEY-I4hCl6 zSjMb}TXGefcgi7M_3HGLROr4aZ?aTomN&tyoKYdXRo*p)uH>`+fC77R5|L+h@}mJ( z<=X?7u?|Y2+?H#69#=CGF@<5>81K{u7b&^mgj2#ug%eblmrT1*XkJln7_Z$Isp(u9 zWv|1qsJZhLBt$ooDgvq2kDw{t1<6y^9vUuS)2)9&hqar0>`X4glu0_DXlzDf?_@P_^YiiVBjj=_qFtB!)5h)`Kz-F85-LYYMk(vpz4q#N;Sj> zmm3IVx~HlV{%YIOMp3GmuDM0C9LfiOom9vd$s{-&B5w{{N3WwS2)pOBIiUZL(bd_Z z&)g^8TPy8zR(G11As|c$zOKmW?hZ68lW?Z0pz(hA08~0?{?Q zv3*g@Mh=ETGIX+|@>Nj{e7MI)_m%S=Dg^!$oL*QwUZY*=2k>d`v+zdtjg+XOHd{cM zmO>DcXDgFZH+Cj2b9za*~&AUWct3~K?W139|KPKZ9SNWimjRxb}A zJ?KZ&vW!l27`%fk-UIh|0#$-AI`SgySR*-BgaQqu3qWBh#<-IN!b-?x{RYO&i#ad> zI5_zEdJKS`RL6Gb%jYXUwn%DmG|%U|AC3YdU#jeXzc}}0!&tcEr~XdJ90yP3!qdq{ z2`X;QaHB( z3XQfXcFh95bV9hRlK#N{@&6#nnyZ8p|cSNI>)i z**0Q<&F%x-Qb#L4q7?%GYWA{;+M$5Q@u$~2yUg$mUQYa-Z7}S9YhgzDJ zfU6g!hHVp3N5^)oxz)IUs^y1E&76Rh?+kv7iZRqN`)wGFjpTv3Ar)1U905|qKvncF z_uG;`QSyIEDKipn^cORY$PLP3qL@9RdjyghNZnJ)a_WZps{p(Yeb_(7&as_eovNUb z&b_`Bi`9Q4R!gqgrDC#qRy40p)e5L9j{o1GNmr0ZvI`W$C zJ+#F+gum}d_|1IJR0vck4v?9HMN#_DNu>X(mi0c0_wnfbyt+5DsKVO@q2V#h?j0M` zRg45T>+h(imlX~FriPJ^cs@j)3&SKuVv`4;5K7WJwb{fnI_(gKE3M)f9=(U$Z+F|o z(rR8bBCVP5V8t}Gr)5!CKChwQ;1@SbO|BRKi*fz61g&LH;Z+h2 zFSPW`$SSH=Uy5=gYji}KdQB5y0vr6AEKN55Dv1iSzr>>5UPbbdHlz#6*lp&%0Bw7_ zr8+%SU@T|VT>T3JaVHG>I_nAiL}4f%SoF@BXtkAjz*FSwC2NSGVJu{ULEFQ}?pTl4!a!>zCC8{X#u+4(l$ZJsWQ8?KusYkUgaV7FF(DK+H+J_LII+V5(iFKLbq5*S`3caYS5!O563wW#&tEU<;zYR}ci9Ud6iW_qGE zp+c^mA=JzNF$Tbj_Vf!p36Q!{AmpO12)sllGNYK3Q>YN9`yC(vrKu>Si+CvD6hOnf zlEYdq_+30kgq*p)FKDHth(OHs?vZ8(SpuzOF-LR&L<;$j0^M=y^wIx#10XI`*7S<0 zP*pnCQgD&0ViPMFj|2wwakWT=tP(C+jy;UOpjLm6?CC{neowu8SQdN1G{4f7{R7O? z+??irM&_lxFNEhYJC9YPmuy`~Dgk&YYW-eAY__~dTs3&$@6(2QZ z_8xHwxlS;l-onhdFyr?l96j)J;cz~p6n);v3*;i9dhKH)MF}WPJfYn@t9YCraK9&V z8e-~s2akqy%upkH1(K92M2}Lo=cEnFKn#Lo(gmkgqpQ&facJz|xccb+a#DNjBL(}| zs!2q|Bz^;6$%1UYBiR_Y5ZqO@Iuyyz4Liw2j{mZs98d5^iTLS4XO@VQpQQDqd9TWI z1k?yqpy_M=+nkLc5B%gk@%&W(kZ%FBvl!a)9B6UO^wP&VBxx(FFSC2SF6C&_ya1U| zXgu#hZVruA=aI&X6NU(gGF($M*lJUahgVj@buH&x&U8cy6LI&I5h#UyAeD|UaU9toEb1*Bz1%!I&+~|0qj0#21>rGo`E_I8bvD&J z^2T8EZKNn>=`KZDI>taGaf=+|fecl~K`|RwFy42((eQ&uWO@SX+R+Y0{`HfnWU^cL-^S;e;fZulZUc{lFd^ z(iW-JKS)PhA$Qbu{n!kX#^4o_EvI-NI#Yg?<|X>FMy;*+QDtqQd($=P19&~WuNs3o zh*7}{f54uY79p}sZ3qrIgR5+pF0L@G&v8YJ-F>`&Li9fjsj!PLb$K3cK{(%5Rb7}M zZ&KdD^e;9A#q}n>+2nlVab{$HB=QGuVDR0o5G3|UQ)OamcQd#<>)vYX-l{BxI!t3~ zx}k?mWLP{VBB)NvAS5EV-umS5_f51*=U+|f-arHA+Gt-jR1Jw|)PV3*RgI0Jo#hQK z6@fB#s_m~>NhS<;CO>I!77~Dvxh0MJf!lH%(6Ha{@K@W9g2g~4i*}rA7}zslM$!QK zJ%Dw*yD$g)?HIht?a&VZ;?Tg|N&XbZs9E{ASe|3lt;@F<#SLip`ch3xaHuL(j2Hv% z)=M1ey2Ng<3mpd`GortYIF$Lz@gX+-y!<{U1H{)hRyq5_tjkFT$@E4?!ZEx~|;=cB?Y8vr0`sQ=Vof*YU14cXy4O z9b=H=E7!&`FVczxd{%nH4VtUxGn+Kvpot-)*=+poXbF+){K;pP$f5J}9MVam#+GS| zn*0~5+sDwa%?y_zZ^HJ2w)jeqJsvBk_;m@BJa;2UGmkZmn;JI4+x*Q+Ib#o_@df!( zC&Fa6Q<@AxzO{gw87!c@_k!K8$wX%kWV4z?zwRT-S77Qs!+4?|TS2hSZMy{R^}RqB znIiR-G8sdRZ&*1#y7UFHSky`_b?~Wtv4{mZdA7jTKW7zt2%Z>Y77~F@=WK`?Db4y_ z@mb7mybtC6C3Y<->+Pg@iR0Khvx{r_;l+@y5{! z^xED5+sDDAwCm%d_uVGZ#|`nK`@i@bT(s()Xw%t5DViUo3r`Fli;JJkEw4*8_LCVn zvqyWPs{N<5xHg%*E$1iK$E_O|N~;wwgHG~?nh(F!L$(>$?t4~FeJlJi?^Xl3J&EkNFLID*A^cK9ch#sFy%f}(%_sX1{^b@I1A zgf+ZiiWE3c`-~7(P|J#9YOqI-r2(6h#v?uGGx0F?pl30hCh&94(YmN?E!_wePO+&M zHN9Ie4DT-~hU&0#SSAc?^zhOvLxrrOo+tW*uKn$e%N_7RY1}QdtsPJ0m!D4*iL1Lt zW3=qnFUdIETNTi7lIj)OA8^XAYng?gFQ?k?AK~24N9TeB{dL`stRO_Y!XyKD(`YrL{aTyj=mQDne-H5ndKv&bC{aEo z3(Y$i8+4Fy$EwwOfz~nSwUYqq|FD?@rc75LI;550w7+&XWGnb80QC$cQQ~5}?M`^+LncUAB)O{aXqO9}Q zmg4$G*3a+!=(dk|wylDj?a=rN+hE!*iXLe7i~t?v<{sDh%U5DJxDW6$9aVCqf4)Hi(-iNui??}|!OL%ddD8nBBKrL_ zgF@HMBghDtKoRJ(v4Egk?0Frgeb!#KwN$F%X?X)whd__AW)CTmxXu{qLZx065Xd06 zS_Gef*5}S+en&Ry{R{jr=c`E7;9<}>W!5IVQ9FW01Y)Lf;<*9X!V~QNsptXHCTni6 zB`tG(*YzRKrE3RFHXKew3k%#2DLQI9KkL(~5s^8q+myvL+Y{3ccH8;)hfViPnlgZQ zewhFDbckic2Ic}kz$NvaP-zy>E58xM!h?ZMyUGf)wC*c_EGmtZFez!ZE<`%_KEc+> zQEfDTU+Fqu<~NUC0+L0ZcKNo4bon&*$#aS@_zt8hTKlD}L_s>y;bR?;&w}<&H=N%} zZh;eW*Qyw@(>bivp9IGzJOBoz!2=hb653&W2{uFP?jC>$ggJB>=B1e|5Pr(HBRbH>xL0{`JyY||7-Hh1 z`Wb=x=cJ}fupc$+0L_HKEt}yZP0A4)MeHAY9K1yf(%X4Fy z!EO69gNaAw96Gco8YMeQxqZKvY4zfz%bkD{+e7=MJ!O#^o&}YFtab;s8V@LE)h0%qZ`ZhhM2sb6L zH~KA9P{_*09}lKH04d9P&32GEuYzFKZ;pCF(=CUHwmX|iJH1d4KM0il1{{%{FA`Sk zamBoHDK~Uxt3d8$Sy;?u8&N0;jNKteaZ;iS_V0I;NHOBB7D3t`=2l9SZCk4E8}GYB zA4f}3o^MT21nzgOksT`^%KHmDF^FkgWexx6C)q;Nt5A;vL;GK=W;4OE3FxCF?HlVK zD79~12wTsuS|~PDO@zj8Jl>81)X=5A2{+(S$wsC-W8>ST(ww!;R$P8g1xG&j zg+z6pMvM}k_+uEj5t`X%*-({@eYRNWUsJ=+3$b zkU>;zumbig1wQbA_bq5%o7Tx0)VF0ej&CH#8Qx1?;kH%hF|$Z(w&N4Ikc zhnhDaKiC$rY9Lrdw>`^%Ecw01SD)q)AIv-!Go>qpBEw~MbtAz?hbOX&Cq>#R#Yiyt ziC9Qnew6vPp^@mhHuLSSDwun2&Ss7)(&L#%D;v&yv?F3?_m_HEHC5{x2!1*)-XrDi zT#Y4NsVhDN4alHT2#de$D_`XM@p9#RmvO_UeLoYSch&HS+cFXv5!x>{6gNn*M$Nty zPTn>x40bvsv6R@BW*q~YF0VxQ9`H-2izXzR$uzcKFZbH%AHW&;iV3ZCd9m2(Q7vo? zX(>F)6w9n0L{fJ@s{LuSd^DuhdijSL{xGgJnZV3h1ad%P-yjf3ECctH>96rFB=Z4_ z^9G)c+!wY{0x$=^4V+iE*n6Py)++)GL540OzsNNmc!VLCuzyY6FcW=j+{t=ElB>N$ zA2y9_+a2upz+fdl?PEs))QS-Qu!JHSA9_CzV8}|VCFPI}5TpY&Y2;fk%9}O#zF={G z9%lHEqcUK&t|=w62GNwAtrQ>?$-V;LoGO3z%+HR0CW(`HDk+!)PnTqevnSZwoC&%DoRJdf2oxt}ME zwXO8_y4pv>f5)<6J4K8X(PGIESWRB9Z2osfZ}|#=HDb~g(fNY@NJHI)l1Ty}e5<0( zgtYhMn@M?T)I`NWCi6M&i|YWIEbm!#LCsCzo%%VqYNEtK?Ymmt4~WLCa%F0Z(!fF+&_TcjkIrt-mKl*@#r;|=|kjx$3B^UhDW5UomyQwwsqlBMPoI`&tKQbRx^bUkAJ`@`zw;8yNe7LtRpTsUD_tOBIu z=6!{8egYr+0d{-WcHI}iTsEIrLt|@jl`P{a&b=K3O*lj`={8|M?n(|-epxd+(Sx6` z@$is-ce{!2c((XiZ4o?N&fxyyY90x=7cBuMLQYduy`x?nOKw=91zv?up|I@czAUDvL}57*E=}73a+c4Ed5YGCOF(L=FCE0fe)Fjn|EW)K>q!hi(|4KK(1(f{$ zCSH=^w4cH%mV{lGK`ux7f%vn0?e@($zHlM0Cg!%L4m^xHM$*Z>UE>j z@)18z)*rv}hT~*sSG2Qh{uBGEfXQi4O*EnN2{t~_3L)tCD4-Ey1q&&Y#X^6xTd1Kb zUxgG4P^ls^>gE|rzDw{G=O9NX?uv%yjg$$7+AS5nf;}Y@2emP41Mr*YG8KZTmG)CZ zVQfxY=qqw#;Vp!dT4_78y+iU61&!k#nOU|jIC7(bUBPAWFi@+Ao|Jq3)EV@9P|BMd z?6pcfqx~`&dul}3N4nw>eZG07K42%4E)n|=f|3we!Ohja9O?e{sC@Me4T`VZ-=Z|} zJBe|zlvzOwl|o7FkDRI6PJS$Cl-Z&9Kbt*}RFx}W3;T$Le@5J-qaaPKdcN9ar(YOd zYm`-YP52BPPklb88%=bM+uyEZ@SPWk0|_Ov5`|c~_5(#D0DnQnW~s+t>a2Qj1GY~lp zybeVzeITwZR1fGFn(JQo(rtH|KNvE#ag(Kz7L}0a=v(?F!8KhB7S2js(bQHIzDk=r zo+|7uLO;@z6BU9VqQ3*+;OgtFy90lf@{!4{XDdXbqz6&|oD;{;D&*64`P;x+*vzUR zaUA{P5iD-*O%k$v_>Dlx(o;|0;N>PYR71V5t<}UNBLK1nm=#E>_zumLE11#{uJ|= z9>EbJYR|(cz?|V=R6&|d&XDO)ZE@%2tn-ilfpVuyt9>>U8}MpExBk`1bc`JSPb}Z{ zR4^x=EBWi1on)7NBLJr8YH^yVWd1XFC4I8E$Mc8Y>71BFM-1`Al&i0F{cn+ zeY%u4Fxs>Es{!!2_hGh9^w6~ePtA00qswIlyppQGA=MQUC-js`8ULS(fa<36z&}W4 zkzM>@e8s13{>k(;=oSY28VW*+C{(;2iBd#!{?$h<-_&`*BW*1^+{i13dn^S^;INH@31 zHQ_@VlrYfe7w#IV*kWN}T|V=E9smsL8_pxO4gRaVfH&_yNKa)m_*%UYs-(wB1Pc!L z#?uGC9WA55BtG``#=0I&IU1gvoVp1_=0j{TtKd5l(JzCo{^y~5;hrjQJ%lr!<_ex7 z^ZC_?ni<&r~l~)bp*?%P+ooKGqgr-T%SJ&5OVzD_#E*My{eD8SZAfo|W0~$uY0$!6YeG!X# zgew9QIrd-z!qLwzu$<=Ug_dI~n+l<3bz0b0_9Y93=zq;~)Pp>oSu-9Q$dgJ#{7>?m zdhF6TYYFbQw~YZtIe5ge=qguNpt>tIc-Rg5y4x`xfABSTfNp^vyGi&CGJwon-{;Rx ztA7;FxJ_}}T)XS!S%97K3ksb-7dUpuE<0|&&CsRQ?!7h{x3jHtRB;`>W$4cuvr$pg zngDo`u8V68)te+zpT9OBi349dF#sIZn<_M=e{?gA`w*BseR->R0)7)+NnzoLa+fDp zVlD+mp~xVVK1mO`JOFCezC+bvrBXKOziSLLQrRdt35kBkY@GSyaB=c=HDdedO{GTV zWC4fZW%<})C_SBbBDaDbie|!6t@-)Hw$gGgWcquuBMC|e{Y3+6roNV%#@fpl=m9*; z7%-}lPZkzP2*;rCzLTi%FtuhHFbAiWOiN^CW-&{*JPpHoF zyCze$djB0uLIAeTjm4OXf!6I^P+7lfL|Q(H1lKBqx;|+rcHcO#eaKZVo8u3yks30j z_&ABAU+>qaMId5--0z2U=5@*w`?J|yr4qH(^yLz3ZVMa>UAOeELAv}VMvbBLF*hz0 zq-7!^;GCRK+jX3vNopI$ZRs)P6YHkOLDp(xGlt^0qQ1ND>s$qUfBSGbXf7>0$4JXa zPQheP#=%L&$025v{Y~@#MywZnRWkgdVb)1`Nw>;_u_{8hhoCmbN4VPxTGfH8!+`-5 zIAU@Y%0?rG;UXCm1m7m`ziqg`4Sy!7u{O*%{Mgj@FSgBEUnk1IZ93PH-SBBm3;FN6 z#~!(*K%@$X@sx+N5?Tn0S;CtYr(4U}KV)$IJ*p4xGZ!s*l)5xH$ z(7sQGu(>YInf`f|;7#jT>&RWT4(4T*Clsl#7om0+<8+=BE@-f6WWD_YMG|_zW7iX) zp57j{Kb`=K9WRPbwC>;{0|gYVo(P&}?sMl7#h{Nucb|K6I&VPXg2=yHz4N2c(y8@q zVpOCdXU2+;xMZULHyJmfvv%Kr+Wmnqq-&POw}jH3nrSK|tSwP{_`esS)q}&-d15EPvlIq@AQ3W9*~o99D~44hjZoHOtfn{^s|q;^2|3nVu^Bti>IPF)C0O z3MHFZNjQL^b(k}eplfg@sk|dTl`qSgy$;uGhWsXmvCW}eMOECebUX}IEr;rrRGTAl zHAu&VhhL}X+#ACCJl=a*cj@Kd?ZlegWOkXBJKn5qcOiRD#<;7e%<+0lmrfup(b;4q z&E?L>!csrZO7aG#S&_Z%I=JQQSIU|a$OO|MC`A6?unnG=2gBO`TxJdq zuMM748n^$kzIl@-isL!mV3pk5Vox{mo-WVq!9BS7)H&N;;Z)yZOm%QlXj2qFfme%) zF}`T|nU*2-gyjYjS8TPpdC_|L5i8HR&NH_WV5u3SNXKDg?C0E)={4Y*o144)d&Mwy zIHu8E%*n0x9Jmsydk}#65G~#$_7La|f&|I9>BkBEhKg!cizs?@FcUd#sHK;3YZ#CR z2U%$oHGUJkTiDe(P5v*OSX~P>W6ADa2AGnqB7~TriSxpog(93u^iqk8`cjVyRVDGu_N+sm!hgP`H8CRA~Cs734(*J{Z~4+7R?7<5h^lponB?7?vO z2SUyJUJ5x4Xe}KnKHYSbm#2!(@>=&f>Je|O7fOUaR7+$1a;kVQHU1d8Iq+|&%~iZf z3luigB=|5+V%{A+Fb-Kle~Uw6VR;llsAk{^mgpd;T9uW1GvvxzC*go~*_ko(b}c=w_ohL8AePX)LW@T;?tl;%nRvQC z_2n zxJKiW+))p{&r8#9>;%@Vh0r6W%k`VxvLXqybpkETQ}*?hvB^AeQR^Z1@YwR3gJVu7 zFi=>-2`9=e%>}V)_`B+l!jaAtYB^=Z9JCAA`_ToN!8Qen(Lkocfz5i84XR-kpCwpk@)G!$iJZk&!)YubTYUsdSg>q`&D$61If!t12a!D!gItV9gYyyRe@ZqS zg*IOg_skj@Yw22633V)R8*{s*U3>b9QQz^I#|6<_A9-6$ zi-V7em%azM%@vv=PF658k;Di;E1A=X1j}#x??OoxGliZQ5X|nm0;)c4B$G=`a7?-S zv$^CTp9M;jlIU~b(S$sBbf^2(KQwf#jmp$75{wb+wVY%pCEfGfJ@oYMNnuX3*7eJg z@I%b&-7RBqAO#a3UD~Tv>nts(eIW^K;~BMRe@8`_<_IFY zW;)&O(Ypv{196bl;|Sx~zvZ?levZxU)!5mWIN+?sW)1}f%?wU_!>pj%j5VAt%CLT5 zZai5%L(ncmup-}D2+vftV~v}gI9_>fO)GC_c37e@H2^xKT)y{iz%KhLnt~0wQkJcO zR^HZw8&Xn3a9hc$cEHMQK(rgx(3V3J(xhgjG<4C|XVsi+6bet&;PS8}*E_*dIh-Z$ z&&KBFQ23rL1LC*0=!rbPSJ74oF$u$W$e0Rst-`gce@sGAmNjo5sr+>7E@sLgbfg&l z?Z#L=CdqsyOe{w8GfPsItSPrmFakSjW^hJ|k*9vZJiTlNV>>)N(XnL$ODOfHV$)V8 ze1Vm){-_blp&vQz^|a9vqKbRf#y$UqQewU+kV8sc1UI~L4h!*9b8VJuq_AdeUn!-@ zu8TiDItsYI`DLx0;y0iUzI;u^Ku)O1R9~ax4Eq{cNn5%Vax*~ASMcLs2K_eVbs#E1 zL!P6vz1_GS&BAbe&*5Q20@jo>yS0V`Jf0k62Q-=kjJ-@E#tZANeyOowLs(azU}FNq zsKH+ehrF_DtRs4PX)KydwGI!9Htv7c+dRe7)po=`I|f=@=iuBJb(K_hwG4xgXXSHu z$>6{`L<$#^q(|13jpABC53*z%Y2nL|0xn0n8#!jVhbkflOYg5{ge|1E2p`fx{~V5O z*Ej)t4yxSWEz+Ig!SQMKNvD`w%=VLw(>OaU^bXe_s}8@JqzI2y(pd5Q1k}3i zKO3ht8|^QzTG!qK<@l8NSSBRz5qYO2eTLIH#LsTEZpG3EmWF^QR@R7Fzi;}TuftIQ zyFC=H^P1Prk3swL{8lLfJ<2}?+`TY?jp-bXwA#uQFUwJ0E(giR7xww37Y&mD{oT>R zK?gBU^Yo8X#uW^I$hW<|yOZSj!;cRvip-Zn-t)HA4CR$i z+}zPd)n+3L4-eJCwg8b4@l4D0c9EyHHRDd@%=bWGd`4cZbjyKbg?(%5nuJN<{)=Wo z>^RJ7?P(kRN2RA#((Ao=XD}lC=}bH;gUzd9tHQ^_y2tU2>FGG;9;igHRr1k?I_%DV zokQ%^Akg3~fy6bpT?&*uM>S11dEYJ?Ww`dV%W{x_iOGO1tw$~Ujpg9=lLY_dcp1Rk zm37sRr`7mMm-u#| zW%_1kzDA^ab!b{~?WwFRg)3Gm-T5MQ^C`s||GuI7uB67NCmn}thi34yX>jAP6;Jgd z?^jVO7N<$W0VYTFf)*?1OFLSp>x^@!IsCZgr2|`RXj5C{)LMTc>qbJqM1^zrVC2O? z_l}}9F-sz0Q*(_38R3t(#lVLXy72F=%|ttmlS3g6qm9@|al=h#sx;oC@gEvPrCjtR zE%e?RGBgOnI9i7K4plYkQK+oW0Ik&FCX^zad@=;S51eczaH*n;6@K^SUpS|czI#xD+>b$IV&y{R?-byGhpLX*C zKAyTqm_!ODUze2u4YTXko;{zQhK!z(?05GS?&aNgo9ol7y7$eiZNIc~$?YJm7pb{A{@Fh2X|vq98Qff=&h@@|R9JX| za8+r_U?#FHdfG5H)}MuDkb?VA$?G2r6G`b{0v1SyM(r4wr`dkr(h%=JgDQZj{{F;1 z^L}se{pyJU!D(fGQ`@Fb5wU3lmcq+b?|ET%_3~R_V|vDe)A4b-1>l;@9|dpHcp_yTHHA76A)LrP@!w5?z088`y(cKg2!O8J-$misSD~PD9foB`Vq=nuxq?4Eg zln_6%EFTIa1_}n~HDgAo>RlhmL7LE=^q60$4=MjJ2XqKE$6*~U-V*ooYkNuZa{t?U zv5&*gJ11pj4bR0c_nmZrjD->INLbn3WadxDH|5-4NCceurnPuh z$BMa5?cf|DXV(p$r8BgOtzCL&6Ewg=D)(U&Sp=tf;w?t6k(`d4@G(Afm$v{b26fgT4 zif3i&r8HVKenCTz``k|nziU>+U!P9;@oZ zc~89rCVO~SL3t{igg8TUn|iPWgEJ{9B*&6)Bf)rHC&Vpoz7Oo0tay_5Ck@;XSiEF$ zfAK0MI&~(@R*#nHzdCFu-RA8CouJX*16zO<-tOQ1rV>AT)WkU96!jlDLq&CUpZ^xv z5*|Z%rvGC!lOxn!&AFbV$p{Fe>9sqgq!@?0~KYMTuMk3&L+fu(8qN+bB!{srUpM%{!n6PwqZH$Lc zv`S(-jPLNg@OT=AJD!L404)^46t6eHt$sR7o&~tO`=Hz{$D$f}JzSE_c6{jV)ZL%5 z0^E1!WcRd$kZG*^%J(fj(3{QvT*@8no@O(Lx1jTfnfL( zrO*7X9|)V+cM6nr4u~r8SHG+gri5g{@a^_HYhX8%vtM+9THm!e&k2SZq7@S52gY=z zIGFHh1J>V~RFtH*drFe&U4!Qeq0y&0ad_kLwRTs3Hi|;UR91^n^zW?U8P00}qcdd) zVS_fS{+UV) z#+|jGN|W9b#Y1gJ$4qY|X6iiJ6z1oG2P{eKpUl(y9fuXI`99fOJdc~+>j;a^&37@= zXdM}ho+I!w04wx;jPs?qT4%N~ckyvE<3>v+)0+Tqo@>%GM*Mb~v)8RnaBd8=$}_L| zC6wNYlck^e{Ov;a#+Mt-1YmO|%0$5Ao8JaEx`ig)ETf6%e3a-gZCnOJ%DV&gCoAje zCZ9X+jSb~m4c6D(L7`Ed*e~}1*OAnU-xA?fw$0kYrl!Z{+`=e%o06CDWHQTxY)|`VeLk{JAEc}u zg7Of7$Gbi7N@zpQnJ|$age=^c;-N4DVCNZ~d8Bx)FB3^Wi(@?f)RyJz%^VO!{+DSp z2p0B3XMiaD_ob92neRHL%^ZOovOFly5?xc7cc}eAE-6mkHu8E-GI7Y1^RU>9l|izg z2lGnFH^-k%P)P&+7t}1rw=WdB!Pu|9kk+>^=bVf$W%c$; zYRP3v+~&<$owjuHwiCmvg4Pgr?rfI(Ji;O8u^ip6bbBeE_xcCPy?Ag91#y1UDlr45gqG_K*TEY%?S#UZh+69tVWI`P z*F4=1$CEN*GdFME!o99mXt{Y+?EbbpPa-bs#HPtz-eKl1JQ*wX*xbIX<1ONpNe?KOIG(72 ziu5d7t`;hd!#wr2HeUV%3+UzHa_*>K<^2{GORiRDaGWu}iAA{N{K8QVk3T_fxq{=w z7P#5!9wpfxK9wrN<0cz+u&w7kWc}>5|A?ObTX_87oJ(zI#w>GXEiqlW${d=ms1tcX zusNzu`&)1SZk|&?(O#a_2XD;{oY=8GA20_wU9on6k2LA8jjzsI4v4rh7Oe1@tSy?0 z5ZF;uOo7^3BuMApZ{+mk(CntCBYI@9Kb_$a;c;T<Mf_2 zwn?;BZ_a>c`_J5Zxj9|AxVZQT3(THYvp}(uJ`gF8(DOL|C1Ud1vPQ$xQ$QY^7he1~ zPM!*_#>bK6m-iU!M5(l2&n3WzsvD3}`|y+exw)~IAy{e7^-LVKZr)G8f+UMdW z2i@rtMI^?7H|9!(hiIy6=d&qRmY$(qm z=t1aWv$zmou)=_zIT8PxB5pGG0l0Dh)1wl46xzZ)oAq(^2I?VB{0FzN+M3kHmaGla z!=;^=<@bzK*QYgjhMOD^WBHjwePFiO#DXR3d={d8b`cQnnMD>QeEQJy0$1SIV=Bsp411hzU-ay*bRyDWr`w=Cjzb zd79tLs;s5Cva^`Naza}!BY9#%{UQfr=^aO_`SCL|cxjFaG-R%06rWlgmS<1d{t-D~ zf@Bj!T~ilD6Y*DG$}O`68S|oVLYt4j;=qtmuJjNbaoyFr3QE0cK7UWLef^0jrG!y zOS)ZLY`nXz?<1(FYpgvtv|irhJXEl49PsSm-FH(cJDuB3`bP!7FE9XJEN+f$gCgcM z`jrmzl^rFdO4S@F50{}|&ep4BO$@3b%N`Jw9f~)yO!=j^-`lOc_lGG4BvZ6NFx_H4 zZikKOIJGc`7~D>)DKa|`Gn%y{RjY4;1H_L#H;D{Ll~Y)}?10rg4Ipm0PIJ)wvSl+v z;W@=irgnwnJPwDwcD8%mo9!(+uE#$stC!wG zrwC>0b$iPIr0-mf?iv;BQHwSj-izRYn3FI2yldX0=b4|c?Lsu2kGQOz>+%+k?w0hK z7OeeG*|E69G_w&X@4xqo-v%*FOJeVlxW$$e-ziYKjj;oRs&z)A>p4E=%;M9osLVdl zMx`62>PGw9##7OP!0NL2DA;M`)n$QJ9BR@9)*dg5WH@plOItb~)$z0Hh$+npqS3?S z4BA+$oP<&kGg5|ng6ropX;cp<(oB(YhpImo*^fH z{%~H7`3mHb9#$+BP8=G_5*T{}A`(M^njawBNP1l20KO!bvrk(?Yp!IjDPF|ytj2I# zw1&4W^D%ogd$5uscc2oH_z4DrKDr_@$#$rrH#fe)>8!1ddCU~$ zZCo8iOH9QP<#Nq-OR)j zp<#RXG_jCjQOzd$->ZKM5N{7-vDgG6_>=KX~^anmG3ra(uhs_P9cbbmwYMRDKGUnbK)Bgi4R! zfKpp_6ZHQN$v`&0?AiTbyNne17Ro#I0a3xfuQO%bf8VzK%WD>qQb##gfiRXUMchf> z4}!3k5O^!gfmVlNf5Rb%5_w|-gtaJeH^9CRgq<>TwhFk}Y`&ntG;mFaCkP2TF=>Pl zz+%V59jMQ-1tf@}BpC$g-@=uNLV^v{7G9ssqyvlXBs~&Hk>rQMoSPJoC|K-v*&ZTj z4nSD~e)0t}*bY<5JDVE~2eNV4!hm?O%>%q(z|X;Ogyyrw>I3GfvJ?-iu5REPwK}Z1 zmQbbm_+(EV%2a|3NHCG|i(ObC55WUOp&AKk)s;wqv56T=$Y*sq;@oL9lc|oE^jrvE zAdb~%wOJFA=~PuFRK{uL05Xr=bsyiRoZsC z%pR9qg!q{F6;e|wrTPOtzYHq029gU$$mYO^6*}N(null+HO~;LhAc^`>C_J^v!#rl zG@&JAL+vEwO0JTQgsB8liCI{Pgb_nVF-OI!H7qta^AAp8u!QYHhK}B~b@M6p8)fd? zvS|JiygFQVMZoToO-43be5LA%R*O$9xKa>W7&6lC>=d%0FmBkfF+i))BD9jQ0*t15 z?L<-fFaNHO#|%y2J|0`;1S%7q3P%T)i@F!#fxxivO5hdzD}5O=9l`wEgS&TEs8lhf za^+I1i+<|_AOk*Y;Su(X*w~{{Bg5`+lT>`bzjN#6xQYoCQ&WT1B%9reG#S<#oCBbJ zavXseJrdXhfZ@{o*kMw+O93Qvv4AQ7WY`=QpI^Z=!-sI?A#X7(vo*eu&M?O0a|`~3-)aQ4AHg++Pw>ekM2CNmzuo-5p8 ze4!X1H5&_s4h$1iS!^xY#k`*yU>I`bC;#)O)&MJ%nP|LK88*jh!;>){IY{Hf$Uwux)7<}-qY zrW3}T2(XbT*sZW+@PywlU;4&31QJjC?wb%51><#1@0R$`bc_34|I$T<0S9huOYd|t60ARPW zpB$7yRa)T`5({}N96x_VZi7sr@TTi;xc;BFwK}WCUEMnyrG-XOQH^uYf8m{vZ0@+c zg9je|S8w0iwGz5c&F5ZztwEDB{iWXLAMUel&UBdSDyKJoe(>n{#3YzZ+1f+O3bkRq zZ2+PeWfiFpMHNjqm~0zW7f2E|lxc-*MTak!jp7lyA(#ka0ycR6?YHGEnNM=I+Vx)@ z@ShqDo4`ju-S6&g(uUxwHt=sQ3;$0B)o|j)dr`ye7ql>(Q zUI&uzkY6}z!;Z)p`&JM_CLvZvJGPOF^(x-dPUKF%XV1BugCiT5L~w(TGDzYdKIntV z@4RhzWZKc|nYTN3x+G-t4;%97gx6j-Qd5nrLud88w$HOKQMKG%+wZ@lmywri6qlZJ z^|k%_Jyz&RU>uFJ`I&SuSW#mtg4Rp1;^r%sz zKKXnE7PSy(Q9#g}f@wv2%*fJ=nsBD5M80g(nH2b6C_wLuuKU4pM)?55x9C@Z7aZqW zQt-xu|2psLuEUou={Mlj$?px^xpA{I;NP@#$)`_0`p|nHj##+-!^J<;%%~q{81KCH zO2bndk6*O-jW0(o{BqO}D_0w8`HZQ1zg;+g&SY!2 zIKLpL+ciCgf464%idD@{KV{h9_fTFNKX&xC9b1RapF3>9!b5q5%oczNu#!WWBN00|8ENs*Z}z>HfR*WPs7 zXG>QPS+=2SqJ7L4ABV$Uj@L;_uQ_PZ%8?6~-O#7s(AQtH73ar$3*UO+z8;T1HR0PI zZhGX=_xe5R%P(?-2<;Eqojw~sx6c;GA}SUKMf`*8ILvqvL?{G(fIfl3FfwIWaaeAV zGufB~P*V6`R2Pxj*ldl@YW<(-b4M*+(XQvsgWh<~;VZ;|Y;j5W%}H~{%v{iaxd=VXF{g;Md49a3&Be{`Zm0u%_h0oJ-S{M?{T$1|NMb<>MUEhuu1&} z!#;Yi*@fq|X??a2@dzrt;b39T{>`&zJTpYpB@O@pKmbWZK~!|O&uXuKO5>_6TP>V7 z`-~Q6TJ0`o)@a`C*|K%s^r;t|cizN_6XW6&dfs%?wr$&H&YWDmTFpzZ=-{NbVoGw0b6UaYRY^^@)vS@9pO18+ zZo@`(>YswHh$CpUZgutt1K+`>(&UNXbn0?NVoGIJ%r5KNW6re6ci(5Ztqcf&(bOgLXKmQDF|AJR8*c6e zKP)Dp5a`HqsxbFJXxsDJ*h$a`HK>!ZD{~*dsKq?g?{R_lVDOx?TMaEP%+5TxZOaC> z>Ri#aTXD#^vTL^?ue`kc+r<}l=zt8w&){J(#*7)$pibRQn>K9UvE_mbFKl+^Ib%nU z*|&H1Wu32Ra9T6F(-}CDyI|qMrOUsqkXotRbvJVLaasba3jw6ovOd5-(zJOq7juNt zqQ-UW&dWRyK&s+&^}6G3lnVoX->HpGF_J6p+P1Y;)wC`1=e_#rXXwn-ZFp++W=+4H zJF~}m9etSSb~vyjxcu9N1=-ngad9+9qf;Aoy}J9*f^%AX!oia5TbTirpyv>> zSFL?Y$^Kn%rLc%3A~KnPf5v}a>w8;IqvXiC6^owkeQV#_t_%B)e7Ah=!#8$MDm`K^ z$bO*LjjO(0*r;Beb!%25)kmzYWV=erO*z!EXK_x}#$~fdeen96X|r#*sTV?C5tobp zN|?J<%a$2wRjX95m2>z=LY;cfI1e1DBi`ewSFhkmHUfSb4AAlj_@<8?_0qk!AK19o zpOrb_j@usYb>rUkD=m2kpS$szgL}4wgT9ZRe{|FbZ#S-=aqz&N-0Z_pTiQrwBrwCs z>)A!gk(4FtmM)kv;`5lp36|V1seCIqcgut~YXXS~PD)GeXsj+Vx0; zYSpb@?_gFYauJRjMiPQ8s!dDh4eERUxZxvGJ)SS0?)&idSAR2PP?b36`+a)P`FfrP~cRaL~J)ivvW8@SP>WAlfORKw$fpjs2hM=i9UW^b0!IsF5M7aY$K2 z4s6}D=7&$8fBv4g`Xge=&dRBnR6%fBl;O^*9-*r)Zx1Ksb}tf^R1mFfy+C0v87c{t&-*igKh3x9)9Zbd!Om`=xYtm zZL?tVIFDQO0}G3b(P6NJ3;ZRS6;f-2ENp{DDgkQbh7-2399ZJai4;-jgs{vckMI+B<>v7uy zLl&%lf9Aq3Cr%qS=>5ci*Bv(EL%vm0M}GO@<9$CJ<7(6j^CRGuzo^9S_qr{mfx=9q zay37;qXJ=ALTbRdU4}KGX7hG=wu-3@&wwLsd`{audzGYG>`SyBKD^hTR{OkOcYW~W zV~^Z=!}hhy9CE-7hNp!#%vIc?`+^R*y+Nz<2haZQ?L|90ZsXwx@1vehe>mO<9oW3_ zvEFxdz5o7p7hUPMC%P&o6~M(CK^H5p0h=%17nfR%6F=o>6+;+Lngc4_5 zdebu-o!2qXk(%SEkkRz)OxW14zsvo7r;nd_eYY#;jvMU?m2#9ZtETKy6eZ0t!e~n= z+;ei;XOrg*UbVf&jeTCZ?#e*sE`qs&frPw$FW!Gkr8*66d;EEhUyDmfV0$yFLO-j$W@vstgY zx;xW_9-Z2M_WsL;Hxt1W!u0fvnwbZ8i@7eO@*;dAIZ0905AxWex;OtE8vrZQK$lDYAupB?WmU zdv?^US(6IEoKjKI-$2JkbKs)gVs&B+i$AmqxQsSS*ywoC1z*mYb=jqtzVqbc3+Bxi zO+?lwLjbpslu9ri;1mUC*jvHvyrz2>(q6S{l`ZJ^7a#f8y?1*&uKusT?i3|RpTX!+V@Bui*}QGlqUAHE zw`$kkRWZFr+Y5%i|E{Ajch&sa+25^d->DO48u$g~1Pc&!GbS(>R|*?k5vDtx%<=I@ z%h~GqZqb4RyLWc#-1+qL+wIx4lkb~7KS7M@Su6-)jlFx={O7g?mM@yMV$Kvt(c#^j zH%^;0;j->mu@`UUyh(50->diIkJYVJD{t@iBbj?k!=X-BcN;t8ll%kQb9Qf^JnG9! zueu8LRwZ75ACxVWkPnPvQ;k4X*zRC+Q6S`y;-Y%zD9t;#X!g{~X;p6RbvrnpwQJXw zmCHtc_7OEbPNFSc|g9j2P^yRPu>gVXMVS6=Hb<=FX`Nc zLDxh`lW^Q@7spuV?x_eDh_O(MwzRy!N82t_isk9(eSrzW3bLrS-Xn+w;(?1FC1# z!i0q=da(V3?k+&dNnL3F0tN7z4lkk%xM38!Eq3Pb%UAp`^^=d-x1QX*dEb7|2@`5Z zUxPAOQs59PCVl_?_h+=bps1+mqi3HUkeh3yrFQ6iMb8^c+w|Rv>pMyz+h2SArB_}>zH9=bNF*zB0jtvoyvW8!#3qhXQGTKBR9=*+ z3yO-~eEH=!v$Bn(qziAp?er#1`@j6EE1{y(ZeweuEGmM>UjNV`Lxw*0lo97?d2xq3 zdfmxbwP4XA<8bbr@e}7vngu6HiQb+vJuR)~L;c@+^N|O}aF9Zk%5VSYKWJ*RF~=nv zlGwNp|LiCYh!PFDm=;$Y%H(JT`oaMy2=`A3KN&jg^}df9afuDjY2CG3_eD!rEdO>% zpMO2*vs%%U5q>)um_L91*Z+Q#iewmLY1awzIQlkpSteP6vm` zD}_bd(q$|D{o(r*Hl^9Ak3RV{W6QeLtBmw&Y>7f*15)F-#su9aYzsw{6A@SR{?Gbr zZov7oD0@7mD|oGYnC$Qnu>qJV9FlCW4zp01pO?$IFphY(lfd2qa&%=`-(z{rX>+>b z6F_FMSb@kQ96ie~jcWcG_MmU>gs;9BGVH@IM`2Rbg_({+SxJ?X#6TTqv4^cLc5w$~ z`!`9#!?XP|Y%Z>!u;2Igfc{k~r(S>4tzKca?Qp`P#}X)E+Yu@=C|ojefX*aH+sXt-*_=V? z1%mm!e(bSOp6skF#3sV-he9O|G;u_09^;G-@dz=>G9IAnE^)cDSkQT=Y%-P*x#)=3 z=ZjBHaM`TBg8UnL-gxU>z1w%X!Ygwsk!{CNTf{-|>Pkkyy=kiidYaCp!2d)6LG4%{ zgS{0#RJb?U-3=mG3~wmlPD%o30wSLlhL)YyoY5lF^Mq9JyI2+sB@v8#`mNUZv>Gho z(b1)ED|{DaVgvw~5@B6XfJCFbm^6_t&R_tvg9c2l7M5 z4;?*r($pE9uE4^zB`!7HZwDPkw5qm%6UTx0LdF~B`>bNZ$s??{*gPqfu_*0DM@0dL z64;QU*sEgE)Ie>S@>{K?AdASZ$xhDKWk`_wkR_o~Wgl90&?Fdc+N#ZI&6^`82kHHO zp#_-g4ck4*Rj7xTL=krgg%I*sv0NfPkW;Lt*rBJC&EZK(k1tW0P7%ZeMltK+sZ@pS@1<75?PAfea@+PDP~j%H?ERF{*Mj9jM}@eM zKj>o&p!5z0#DNZEEe%bT5q|{3_BgRyKvk2{)8z)sD%fSEgo4Wynt_b1o2pU8+Ba&A zGR~2R7%-6GPX;Ype1a?9;$?$vC^UKEgbJz27k2Cfs|P9g$%mwz{=np8FrA`+$#I(> z!XH2Z-LE3tk?LOXZn|fZI@pD@634v`sn+)d)P#Xn;sTHgRXO&^5Jq;Q&~F1wx~2C< z0+0?z)UxT$#lh%;WkiRk1ptIILu!E?D+H@Pl`s4=2fBzTRd9_E5+1G`bveHmUXMKC z7FAxH{31-fI0edb3l|KCQDBj>ip>-10&huzveK%sNsx(bmAsZvlo;$L1@UC=K%H#@ zlyJksnNvcw06>;DFS`V7*y4b(hcgyz71Je>KXGMX5X_U(lA3_2l7pkwsH}8R22Spk zm*mD3f(3kG2LcJA2%f3|MAA-YWSY#{!GdBRZAI%bL z(4j_A;frNH!2Clf$bYOs`K z-5Mc@@&p-i*!>V_0Cot5+(hA&N)yZynKQZaUg9hHEue&UI7thO4Hif&c8Ulhu#_mq zp;Sf4l*ENkrxp@MiXq8T0jUmEqqyn@E>b{B!bwI^M|7!?1X9`>QGoL?)Ho12BbR5( zydb-51_FbXBV|S@qRI{ag7yMc;2oF?P*DScNBjg6oe!8i*{) zQCLzuiJiGPc@j7NA5nGH#RMgQ3hB4yo$A1Nh>TNE42OJ!2aI{B2ZHmXZvvZw2DYHG zMbrR|@;F|i7{@EpVI#OI38B=SowCDC%SfT}kmwSWszo_O#a|N~Hd2WNL?#Pd z3IG9pBoBkko#e{_oKjR-(iD85LaC=BU9Fd>PeBvJ4o z3!OA6QX_eZmrz61Rx$Ar;e(1yj#3w{F`}qsMUg|*7;ha{v%RqV!kWXItAa_1NLl@) z_)=*exmAghHbaQWvDEOQ@**|;QbMWn2mx_YDfaAy?d$}QX4dtHkVO_r6p3^@st70h z#U#ba{ssI|^%tZ#+Qq;KZ{E_Y1!QEmPNc&}I=Fg@M&*MPFwt>!SY8Q2X(xQ6dMlm# zCt0eD2&i7Fw?_w2EL)iucTzCr0aX6HQ*U(`0`Me1m9=^knY{B#Qc(T`j)lilr>wxI zEC(r}1lMUq>Kl7a9b-LHQi#1!bWoWPMXX%mcq;+4uXw03M!;I0f9fP`S$HWePf%O? zNx^iSXc(2Ectt(ro`OkR=mMxXTvTBqu?Z8SgGBf_7o(57RPC)Rt8-MPie_4Fc>B4O zPA)%_<373C`wy=z9ZX!bt7?$aez8ugH34*(h{#f+(JSG=)YeOhAc(AhD1u^aN+bl# zp!8vtRtz;HrUE{RMnbx;t1?8(qcV_WB}8OyA3+Avm$y2EaM2<;)fEtlf%Z`n8PP@I zwSLDsCHYW(ojWhIlcK2*QY0B+Bw?J2DhJ!EdyP7=mjHGw=}349ryS3!<}7b&4=GiQ zu;dRHud+dMqXQgu)t-uA;?HCGdrd>_qlZHXj3<~$NtKT>o2y2PNX?_k>$uUUqyMU+ zs_>Fb`Ak(f3MwIVg(Ae(+2JMGtESiOSe7ucm@_bEksvs(KPpmL*puwe1n!fp!T-$a z{s>3P_g;lkl6MqDfG~QvuvpS<0b<_iZXGS54#5XnmaGaL>BBmhba`281kI{mA z6-HNGm$`h(Dov^tO(>d|PC#X*qN=1dp`t;e?XH6AfvP-dl}IKQR@1Wl``D|K?#2Y} zldj4C=n9v0&lq;Zve3O$IyBu)1yn%9o#2&{x59MGUpi&EelI~mH4z2K8le&;3+SVB zq2NV{7*rI2ceq1h>kds!ZiKG~CL?eYDUJe5m4YN6aRS36q%lFo)!esUGmH9@iwqG| z2Y5$Vt&$=IRVzv*399wI2y?HNQzd0gH&FwbOo-Fcb0UclT2LD&kx3LWcDYi?iwNve z6OpnAy~IhJD4~QhF;}hzlCO5<1K@+WSxM9-1<6*WCbZ{HMbV-g6<=~Bj1(5BJMkon z&{a_@l7XHXsH53=fta0kg*IwnBZw+qtSs^D{iU(e5CwNS&H z(AjjV8<=D;De$LJK;w}>gX*3dRMaA)>fv~n;fF2+&k>kXaivegi)nT&ieD0vh_R5| z6IkQ5dZ&^lB?TMgq(5{~+lwNcPC^He`-me#`Aj0dYRyfUSp^~z;Kx0uEFqcNGZtwN zKur|^%p#J)C$8#V1`N`RyjFgZ+#-J>*_A7>4oe-hOd~2Hm!Tn=I6+jxGQgDkMF2OF zvTl@Ub)-~6qUZpliWDtYv|Osm%F0(pBh{j@(2)>7em*K+l{+Csq)r*Q6Y095n5IUg z#%dCyW9qlM*ky${iSJB~`y_7sKd$Pro)$||LzI3j-`_-u0$+Nmyc6`3w`gBPBJiRN z6ZCGECFeS} zvWTR^TS-E6J;aAlk!%zns*n^WHX@G}0D`L)5lTdFLggtb0^~a3C|N{|tJ8`kDM?cX zDII^MY{Ha_9~UyhCKRfUfHy=_yALDOP>ilFBr%DrQ&v=#e036%u1=O>a4VR|KUF48 zA5|}iCc7gwYwB_M4bUp=4P7oo7}ObYyE6p#)XLtU$kD&u*5<$i6>#X3Lh=Js?=A-CL}4x zh=#%qZ^S7g=vxwD3qJP*j-?z8r^Cus$`kc1f3I`W3CE%nGFk?S8>2v3?Nm}+VSTTk)h7|*>&yI4;yV+# zPq@a{s1WHM^cfA=x+@-CnOLVd+S`2kODQ12A>nvv(#;^ zf*4Mw-(6V^ZK}C=i|0{md)V>o}%>ny%9Saca)3i>mwlQ9WNxft9#d!3J!f z>GM}o0Mw?LqAD4pa1wLKvf#nX zoYsl~BKM4-c8qOj@lVtk5Mr9wV?qJlI%++Z7NmKkEp(g7y$mAqU#-b$K|D{UGb!-v z6fimNU#HJ;_*R3q6pZd4MF4q>tr7j`UW2)LJysNumQo9e(pb@Z`Kz9J73IS9dlksM zm=yR`3YZ-Cuaf7_e_5W7K03E(cP__A6F`|@li^m>!J?g?JDy@(Hjm0f{+LXhP~n(6 z#bxWZ2B}40A}Gz)P?|D=*Sft;@IK~kbwU)t1nv`}naTY&GziE- zm`uY>cYIKQkyWolint`=q#gB$8}o|3GXI(s_(cksqLN=E&9QwUd&jWVj1n1oyGjH~ z_vXiOUmU{v6Yr$%w6vDh93^vEH`_+y} zIY)x#0mErv;G3VRsBC^PzmozcaQ{w1{nf$bj4qDL7P~zW2V7&^h`kqB#RSkf+yri# z_9G;4AW~Enu)J)fjIoO*V*dS$DWHx3lZg+Nv^k+E6M0;5ei zwZNPp{l)EbqGo4`N{(^M{Sr^iv0QX%Ibaa%RyrXRl#{ZpHqP;bISc!IoZ4se1_Dl( zom2kA#<9ZgSc2NJMC8PiV_eIVH+zmllS47pu`V`sVm${6(^yWFf_!49LyyD^v1*W( z*_G)`3j8JoOyK@a5*?cWFxOg6LiXD5JU*P|Dhpb0+;n-ToSMkqZjQtjGountEp!Pp zCqfnV*PML|HIAVZNf(k3hafViJIb_(X%nqT%PG|62uDF}i`5?r=_7iNVQI~*PZSE6 zzPvr^P8OMi{E~uVKyfESjsF?rJv*eS*%g)v)0rMNzRtuo6v*lnG#R z+=`mV?83%LPT~#ur9u6SxM9QPFkD#mqp3JgQ;ATJb14BALo!XP2jOE@U^C+rf&wOR zAE(U(yN!$S+WHX)C@D3vvU0O>b5hdMvXA5m1CD@OSsjY#nMEt4H+S$PD`4jJ7*T*D z&=76g><*tVV0Aj2PG@0pL6X~3Ewy6BcsbXMmL&+y#EJ%1RMifa#HpZ&VLd;_&UxHEn@LG7SngRSN)6Yj&tj?CQp%}-181rq}yw-I&` zkb_VZvrQDicVYn&qAQI2gqB5S8s^MH)-Z8)>>r~7ZUyE80SoFWz=)h9PH?nW!r`DT z9I^$20rFC*iHQ?ZlBj|=L6xL67Aaj1A6(%z!4J4)Q8q&ly!%U%HHt#jjA5XEiR%7 zY*17R6`Lyk86`2({5w&=1n%RvH9;%U?S)4R$rRh?^EsWO4or!GDy~*>!cp4+O0Z19 zJy*i3d*F{fIEF)kg#cgx(TT)GdjXJx#RKmE3)Bj6#J5;QZyZljBPv*Oq;Q zOG$|oQY-8S0;=ZXVLFonzeWKQxPOg0$MBu%)sjCj1vc>mxssBMP=Ff+jPS8A%EHLv zDX0f#Y27j_0x&CB9{|1_KmqzE@YymEx7%383|fMISaCr#;*x~l?+>swM9%udhl2(| zc8#P!vqFERRBpFg*%KtN3(t)@1WTf@=l&p=&J91ZwfKWhxa?rS5e_?T99Crmb}7DF zq`1OzqD>(X3N7kc{Jz>N^-?O=nWz zcTm8TasLh?9it%hWC1KtGRVr6QCwWi4bE=0@Q70f^sygD_)(xcB*2Phf>G*MTsBO! z$S{z%ScxcrB^)joVFe(-O~3*tstmv${zyzn9L80K0h`-y^99in7ssdo3l8Y>$7Z#9 z5tBF=JS09GHzR~i@ysBx9R=lZ7*CWa><))WIVdMAc))PEtll6%9N;Y~j0^xU?je7G z!Gv*uk%H75R;2gHR#cXNG~)(!4Lr1a-zArJJoAF{-@A8rl`54Qp5D|EpX`-AqQX#%$b<61Uvs3SA2Tf$;9kTvc1kXA%veQg zQcH#b?x?atZEc!ds`$%jD1*gM0R1ofASfGS9*rpN#dxKFTMBGClLEg*0TZ}?i$cff zJsnt3C@QelrGU*uL;y&^rh$Q=*!Zzuz481Lp7?~BD}LZvxXb`s;T7JCOcj&=Z3yZM23<(PgWhQq#veI!=#+-_ znxEeWQH@;=@`lTn4-W)qMfBC@o*6UfV+Fk;*J#`2s#o9n2zZ7UMKnU;C>te!k9Ht? z5gs*&cpT+9oIswMDX4MB;ztA{UZgE$(;%n;QF$Jtw*0wlDLl=GlQ9@FD)k|mLqJZ` znH2af3YftCTNFA*?=^7i%(>Sqdpyb-U)W+t))BHGnFgm2T04vYvSYtm2n3d~StO{Z zRt>>qWFHo%3;4r09UQm|==b^5eh&rNfq+A_O2R%Xauyp}AV8m;G_96lqsHXq5#NsV>Ib9B$15TZgqO1}| zgziO zp#!5KyB2>*%ZjuoGT0o$Qkhww5EL+h`}l1p;0DJPBo*Rl4A>&pvYnwxHpfJv$ zvdAo8w+%OJadG}vqepx*e*EE_+(r%R-}KL0&p5jk%&RrzpEz#ZHzP;n=4RJF?ev>& zxizg?)rsGXpEGT$&2G8q(oS8vclRW?{jxzjE|@d;HD6S5EPmb^noT3S^{h02T@ z*;)H1eKT&>j9Ca@F749ssw=y?6BA*FQ4L(TdgVWFn3-SXZF@nxd+xl8#Wz4TBx;Cg z=>;+m9{%#fw-zj1=t+pb`s$un_vm@(z-}~A94^OdkRE7_4DD1LXRxFzL z#pj>x-LfmCO66;A{^vPuS}&e6XWW>tii=8{w>b0K8*i#wE$!Qd3zjWf0(NiMys@CP zu=5p{UD4y8>{LS6L}W6;r_17}cHp?o4x*2&d>%tFHv%9^#d0fG-O#i$*ljwK0>6g> zCUE~ADjkC`x(~yV(XaJ98Xaby5LE-*fJMWuuz-!IZnjAW~Z{vowUkv$l(Zcy2tF^Q!fBkpM z4s2fi=!-AQ`XcLkaPp!;f%9*}fe?_g8ij{5ujxupFEFF)b;7hc!(vKL=_vq6()EG}a9bjQ>?SF=$$yxhjupUn!dqI9kGERU|Xt6Nm!kCypmju}|KjqWsMMX(R&< zI5u|8s0O6}BQp^Vxj<;uw@a4Io?-ZX!^eKTXyMXs-MhynC4TUqH`lHFcHX#=aMFXn z8Z&PoToNvGJnCMw?F;>naQm2V|^vXdT3$$ z&ZsXwv-nCXS52QWW$N=!KAm2v^5n^rYuBweV#>5SO`2_6v*y~%J3ae&UvJTou(#;7 zM;->X?|Z1v)LAn|Gw{6lVwcXHznwpS^|Ei@c= zykWGZZMLo3c350-u;q=Lol_KYv_7}p*imDP3)XF2_XCTI4tHGR=B^Ww$S2v zvz~wa(Ic5ziscLB6&1481}VC%nK6sU9;qyzDdB=>;00t0q}5osVE%JYJYID0kZhz0 zmy{Nj!uN+Qj>M#j9WLpFf$-B?p3Ou7%-^_iqilxp2hkbazII(WH#aLkyGf%)Mu-p3 zwr|hA!a%TIv$ID|oi%sb?7sKjGjZ7GhQH|MJMJ*7r9f`Si!YT0O2zN*-LdQ0M;_d` za=A1ZY1_g%*;y95B{`!;{d$dDNr|VNdU|P2VQKDG1_=VV98UDtaiV)J+^p0>%SI@* z=}Zco&=gP%+X-ERldbS9REh?IAS>b$6!l~mcsS&;hTK4Iz?+nm#EzPf*LOHK2S7h? zV4p26&Yha#j*AO>OMHd-#RY{%BM7{ zCshiE-+cKQ)*FX>_Oab%xuVB4&ChKs40_lY@Z6K#dj2z~sL=28BL5(0TBWr2-hGEn zjUhCpmF*G07aa#+8r4I_36M}FDGQ@HIXSYx7Ye0RsPNH;9|P(r9SVPLx8xq&_vMhm zXP$k=XPQvmDX<9Qm0X)gR?T% ztzFf!`I!xC)t~(JSN6EX)N1MJ8TC^epO(FK>stf*-;jUI6M6_$qQEZqiUn6U=EbDI ziADhvxKFSalWmi-i_zv|FQU!H#u$I0&mRunb^VQk%NFOqUVQPLpVlRRFZqn?E+i#!n$-ut%KS*wIRjc=V`f0B}IR5it z<3ArNrmd{jCT&_j{_L|Bhv%L5KIzlzmUZhl-rMs!K)1`yw>;4I(N~{%Y`}j8GE%5a zSvz(6C?paiM_j;4E=)mCIJEjvZbZWdrAfO3-9d&Q7F8nxpA)RL&ElFA_<0J5TEfr2 z`Qv=qH(h8L3$`3to8wQdR>{e>3sI{ro4eBeab?T>e_YllR!WM8mLiMNh@uIiV@)V@ zv{$U(*7kyWb&65omfb3-mWVik-H8$gwi2MS!7#I-9PMdv_VD3D-r|xQP87>*lxv&u64<^PZIGhNzLq40^fxb0X|LwLCu{3CPB9IY#(z1cv zi&a9FD1AP5Y!($2#m6T);%H37&<=r9Hr@vVhjX$hLvl(LIn^8%UBuC#zK;~JITi2* zat~+6yWFX%sZvUm_QIje0|&8jm7bn%cRGA*CX^*mh6yL!z0X^en3U{_j|09#B_+j$ zMOLRPA*lk>0OZ%~%*0q0IwYu>JFM1F0A+M1@)Pvzm@>!!#GW14@#uE|>D0%jY+yy` z9D5lJ^q2gliHmKv*LnIy+d#RwBQvGJX zqSam@y}H`ZBc~t+EjDL`6j{8($_McnEjVPy$I2cX+&Pe3Boc^FiJR{ z*k-rKCnb0i$hVZGK#`}QjZHFalX0Y^X8@@Itoi}Uh;w*Fnb~<`WltxaDZXk(LdY+6 zOwm7Q>VPagEhB@R#R4K#vpYb4P`e~x#HFTI2vwvPdYK8JhZt{*bH}6SM=(s_viFGb z3?4uhObNp<<8aEZAC_t{8_UilR9!>^M_C9^{E21}POv67i)&KgCloM&`zIthzCGYA zfmgBir}zn3L1ddFWfomv+5RnNUSY)bghoUhO2R9%9L0fPZHo97yVj5c$VfoUtHy9m zxK$LR3C+5ou>CS52-|NHq?f>K%#<8Cf1ax~Ao-!FO$uUA%4YEa$Q|Mg%y-~pL%>2ArBcX(c;ir6i)j;;k$5Sl3i!sUWUxN2 zyw%DoK7!XI%AJZL;mAS7V(OqD^|VIc|F2g?z(^vxK#>QPW#m4R@6YBHp+@<1em0!x zX;MJ4z{svK&gGSYzwz>Qqtk7e(qb~BHAx=k(8qu$&2!tmGdv}T^9d;xht1=G#M>8(zWZD`PZaCi~=Ud9V3N#^J^6N6+i(!fAY5MyZYbxT#eB`TR2|*^lf<@)JXc%xcL7x z;!lJ9c|K8!{ld8P=VffB{I{Th$#I|X%@~D%U#LhqjxQQgP&I<5Kj8tRVI;K}=4y=N z)z5I0k!(@f=IzlutY``T*u^wDe*EI7XDOjBNeo_p34&F;vT;J4h$((LqEtZ%*|iVh z%KW4EDuqAhQl5H8rC5&66iLbeagGMM#;TMf_14@D>*-1F>~s zRAr^!QV~g$T{e`JtMIDqwvk+@0~J(3$xi((S4UOTC_#i2LXb%2u^ZCYUOFS~ukR&y z{zt1I#ZZA`aV3DtUWmq9E}m6K<&pe?5^;JC8I9Zv-=0>p$Myh1UK>t**05F#P@0!Tp) z`-(b(1?1pLc1(S9aHqZZc5U0Xx3=weyR~iGwr$(?*52CP+P3}s?DL&>=Kb$Z=APgr zIXTIdbDeRZKxCwq02eColf)u!`e@v!1!~I3T@@)Z>KsN~4`V|3_K7Eay4MEiT=;au z@?Qh-u*Xy9;WV|veAFzsH1vkt(xFHVQv*jXl|bUCh=g~C=>6rTSggsNPUe9Ju>wV@ zAP#J%P_2=K06HRGY<nehbk9g1V3!xHh?@nY zXoMo%z`a**-^z$GTWXgL3I${{Vb^4oqwR0Ve^`Yd(z!bnjIuveegudrd_sOC+Ehe< z4fr9>T0bfMx zl@U1ZOe@k@idyJei0t+cjTT8ZMa9JEiq>q;j&$+x;bx3{>J0JpfN+6o8gO7aKMcoA z6?iC}CU*J3Qu#iGk=EF7NMX6&yp;(m!f}NAOn&h~Q}ag_Zydf%l08ozJ5MErHAz+spcqZ>$Xs#5fa zXd;9Y;A$j)t4U}ro4ERhVj`mAe2kzi*~5OTX=6+;vNJLlkSZYcFCYjL1=v&p`<={A z7TYqmOU}xfn zJBh#{MT0d}#-G5*_5SO72fAetVe&ARNP$hM%oPcwRAmEE_;eaP!%9N3DB>fa!w#s4 z-73Q&bQ?%ogABV|;u86)w||UFK&!k%f*@LM`;{bx&j!x%A1gnsyN)>tZ1R9;HH7w< z&Q7V$D$jpS4|cOr-|$c=)|!3C&YH#>$pdrqsY-(h?tBXVxS>T0OhL5<@dFVILXia} zPqzo@t~TTZgC!0Q@Hw2|Ln#~NtUZ1E0~2uz^R2=n%()Oe9Glg)@$TD$K3&aIY|2xg(d zmoE7uGAmNs-Y`GVf491b;=kOY zI)E|IcQY|2ijVI7Q%;zs5vnhdPN3anOcJXC*0d6Bz!9_~9z$yREEhM2vn6z~8(msz zVkm_Wy+Bten0Ikbh-I7$ZA~N0)plPyHy_?e&vn&lx7C7AjGZ{=ACTuq-REZBi5dYC zxq`Yp2Mt2?l`sz(?`>r|!=%mdqDsdGeGm1Na%=#N*ouNe`7BA9K0q6aql&)eI>4c4 zw^|<%-dH8Bh^(a`n3~NN^jqL+Jh+{#5CxMz8J9e4m!_LZ**bu%Pe(q;m=}txp&AFq zQT&j_=Pi5XEc@lfojzOM?{VH2D~Uy5mgCb6}Tn9>Cb+d z*)#z&Wl!3q1K?$@xh0VzCoP7)Qy0R_EEu^!jRe$r3jS(3q+Fq!u z6?dhMTo1a^_~C;s2SB(L`|n1|&6&rvL68aznJE+`S+elccNm{^4_xTr_#R@YkR7T- z9?_oze4|gN9|alqPq0o{aK_=NRWy7k2uW~AoQwaqknCNAbmUKjWN0q@CnEMna9dIo zdoDh{OsL@a;2$*;f_s?ZE9nZY2)N6=pdj1*ec2f-jGYhjT6W#Hi=psTSlTp7B*RF5 z_N33MYL$tQSs{aei*XXb`M(SGx*b2&_|$bR>nTSgB2A_w+ZaT_0>=w0xNB6_C#@EP)3zHV3redwn(w1n9vCh`1ZlV5V``Dodc$M%(rKkux7|RerTL*h2$BU0=&BhU zEcP>YZPeJ*dqBDG>7r@oYb~TK&hXsBE%sSaii+WfysVFOs&iZU&$h$dqA$vIJs}1M zZL!IM>tw>lgino*7RxfSCgXC%z7Q6+=;0_#zvoT2jEsyt?>p#GgvA=Im0f%9=S@Tw zvuW$)YHa9(!z1s9ISIK;)?MG7J~z$;$CsPk?wVBHmg3@r{WyUy*V{eK7G`A{Rj@9J zID*Z5p`y($SNN}s1$r&^yuUB3%A`EX6bTJRqen+axvqaLl5NKm$!XPUH@e)e^heLeK#|-<@yDz_vP^jsbTHckMa*qcQm3ORJ{kT{nmm>OKXrw0OXZT_&Ug}q$(2C5QWeI}&=#v@+0DCR^Z&;Mkj36%vA?iI z$axbc7Ip8_69kEnJy#eaONkXUFao%hl8D1Pjdc>~#9_0vuOHm#a0)}f=k|GXu3Rfu zB8^Rdxfw(xriL_fOg(zP?u{!OMU|)lFcRFW)@a(Ox7i+yC7@Es?s)%!fX5YF^?83< zsZxt$p-Kk@-n%)9v0Sn*{At)!quz0*sA{@WyV>tfZr{iRki&ip*QOMMNkD35*449; zKVx1(Q~EyIU0idmPXGPBI&%m&jDauSPOzUUlIOt69$2lvX;*Bme=1wJp6G?*S4&7o z_ZV^b2@GU!9!Jx+$g|Jw*?r5NwjJN&3r`&%Atv40^I+qx2!s~A9#X}4fltwv`*2>K z4gD1E=Ml^EmD@j~$U&Mq25$i{dS{WREb^@M& z^UnGVzq6ZiJ*T_-_R{6dcDuL1VC6Vr_rt+TUe7rKdQB$BFZ@%rQ@R60p4eIFBI$Mrofjy4{% z{rTa?1sVjxm%Ax77eTb?cK(Z+ueGcdE)epQ2)Zn8*R^V?)c6%sqBd#=@JQs5>$L`> zu_%S7j@GIgRsp;3E67fljW*wob*c=u!z>d($TE6%c7`+L!IgEg*jq#Z6wJ$}L5^DEWuMF2*c0e*$lGVKWf-imBu1(aO zF9$7$+p16gitb8Ki|;VLtrogQNay>i>ay#SLDOIg1ylh)IWR;O0S_7xo;JbJ>|v+8 zlSZANZ0+~j@_OW6hx zj~6d*(n+|$%X9mAM)&tho!)xW!Ydy^_iik|!qsZa`Psc@KgEjeY`TW8r}#*CgON;N zzbU;IzvCadlFz8#4@33tbvEN0BK5Je6k0zv+Y5m&4_GgoNJO)lR|C0vNBBr%k>uzL z60D2uUcdV%{7`T9boAe}UC$@WwSe`~)a82FXg+ROydK9p&$w={M1urFC}p_(PjR+COBYU^Vh|;`!tUw)OY#)>wN=#;dm-j zEZI!gbN2$4n}p?R>ork>#gv`1CS!^MHgIgsz$Jm5#~RPZ^m^gAK*$chyV9x+KpQyO z!5f$z4?@i%*1VeMd;1#2*u!}&c@>ufoT&&@W>_FKJPUnoq5+jhGtEt%7$$2I6@cWAVvUC$@R_<%gAl7a3} zA+S|j)ry_RUPM_`>!tPDZsK2TwvZ(WN3rEuoC{S{T4epxYgzBuHpXeXnOjK!eE|PR zjqvr-$I9IHve|>b!7qcOtEPUCZUJ>;CzXoR-a+jdSeSM*DtB5L4M>YNTI2kDz&z*;-3ji zId+_uivr{fHFwiBS?XYD4lM>I=>{E&LxU;$MaO(6W1^OwOY`jaBSF4?J ztlVI(8oVvU2KNE=7}NYBy}{@bSyUl|jzPAnT1`&VPsC7d8tctmc5993zRRDS5%3tx z=Mt%xOc%b4cQ6zvKrKnxT;aOnJfSR9hAuFkq1JNit<-SuEQlAo!yT8548m|%v36UH z=mAgHYiqj$Pxoara2VhiGuhuCuywMzTnlQ+y|%l7_YB?b9_RN8RtrxPu#6Yav!EIo z@5tHj33*N^xRg$g59BmkLJ+b=Cg)F0NAy`=n>cRX1DC&TTSKsUNIW^4U%ngM2<%nS zkGFe2x9I8UV7t(O!G$`(iz8T-6dMdgVua-A8ir6%<)TTbma&S&`*r(4*wMs*fvE$* zPk%o*CSQ8Jzkg13JU5!7eDp?MG@9|d%>Y2Y36>eIV~!c#`$-nE)n||~7)kLn*Lp5{ zL4?&lA3+TKNHB3E7W6D+7jSAg8FmdmE9`;;i`H$Iw@*O`FIvy;L=Nm=NRWV>^;S^b zi}CO`E!>;}6R1%2sJ+Ls1`Z9ZW}sHyul6YVp?Kh_XlxK$UQLxqhCuB0le>~Xs{qH0 zA2rZZ>#aFlc3YF+u%%O30xnx1OlcrZPUrO&#l!e{aZ`cBOQ+-24K{yN#{~V9wY8Ny zeIzbBJLE6H>U0KAl$&9h&}+JoGI!#Ybv&yu!`E zXO`u(3c^aFh`7L!m@t2o&fYdEDn{O!Tkj5p;B!0s)8O=2y{iscyUP8uqP&bEdC8tR z(_=Hjjv!}`#~y-+fSzBNSqx+}P2q#LAG6q*A#veD05-t@O=f1xx$DW|k`fBu<6RZK zR@>=DNfr;i(_&TK0js6A$^u3PTcgBpuA`8)5jqXd>&z;MqLEgZ65(DuHSR+b#H7Xm4}H$vNK5nOl9COdb{uOZ)F9h z?&Uux`~IRwzQDr-<}_;A2Vyoh93`cx!^m=F=X5$F93LyZIt!0af28I5A@DwO>#QmP zJ(;H=L+51kV9mY@X%SG`-nmvMxiKcfNY8;wP>;^Ygt%yW8-{ z?LH^q!Q`C7yVv~(KBN|?1gQ|5RDEgR7YR*Z$QpCECqM!dpxcE~@AD87%v&)ds@JS4Yp+tKW<M0(foi%WA*h9zv7lb$H}H z9tP#C)NIu|oHx?jwEv!9eR?)O%}LZ~H&AKop4>BqNX>-&=9p*T1LjweM6SzOVQRB^ zPs_vZFr5p`!By>f`xq0abgtMe&!DC6TF)>Oc-`?a%U;HXc;q>+rXeUlCI4F@( z^?sjnqjWv7Kh2k6r`JZJq0e}JT#>onRQiIeEn?Dn~@waMD@IFEVa z^ID!9PY7s(WNTAOk_2EcJg3XW7h5X^gcHv*wOw~keajANon~a<_QQST{@KwF9=rz7 z>+^XeOO0~eSfLRR;W0QMVBwuQYaZ|n-B*Zy9Y>m?8%_+p{vTuA25>*K2~4M`;JG$; zpHi?x=y2;n^^nULi@~8!VE3%J)~j4e+2+)cHevs(`ZN-xVd(&Ros z6_BxZ%}IQ@zvsDLjeYO$&=QShrQO#l*)3IMNkl;);5c7zU-P#Zjb4<>t>-QVh0Zr` zbid`yaGyusGoCp0mmFuiVAQAO^_H1+?VtKX==!+)F^UsQ*ltqWrvB}jGgnnc%5t9k zQ{mHg*PDWM?f5H^w$9yiY}QQA$7q>jdmswwOP4n`|Kvup_8DR|`>VoCxBYR%3Nj}) zl+Se$8lT{ErFVaeb*DvQd%-k@aHT~pFMVda;e9->`)y*}N4}qBVk8Fly2p}fo!N|S z4F7Ftp{KcT5K+K;QE7_6=Rt{&c7$0)MKzXw;-+`wbWHZ#>xqj)&x^hYubD5ur8I|@ z-y}yXi{+bXxu=!u#t4B-mGI+bgb^#Kv~hJey{A!s?#f<1h?}2Zba%Grc{!0Kaqvp7 z{)k3VWw_O;wx2DJ^~BV9yVGiV@W*JIjH=u9LbXSOwgFI}I*6f4GJAetRcr)?kpHqMET5-Vp=^0g?$3i7N3q!2I`{Q|ZmOYLF&+;4EP7DvlJ z?TDVmpbp7l!r)>dE$0XIWZ7Wh$BDF?$#gByEfyFmEWUK&ZmFpS6x0=3We7_NDD-R= z>eBL!NM|Tb^nA{SgwrF}C5I*BxKhQ4%Scan2e78m zEwC{Q`4pB!pUEuN^eeRQM~VV?9s7y+Xsa;xp>RYl&&))B(~7e2+7jYWPT7+vk7n;G zN+}7|WNS9_9NnR(C4+nWx(_g`qA4;MaD0SgprMtPmCday!8Hxhgh}JaDz_Q^jResX zpMuap5I|f)5Xb=SUL{T=6*!~%+rHbH8)Nm&R5Uy_sF9;DvmxMplyaQemV`Mg{G`HB z7UBlclF-g2W<@_tFSz3X&ITU-hARd{LO%g#KRTss>n}}fZ=SAUZqrdA9Fd=R@Qfgt z*aPMOPXgI*^r`kdzvL1Si;Ntv&n)fjcuJrzRu| z&GJ>sLC}n>NEP9HyhVH=U@ni8uPc=Tv}I=)bZr(RWHu8J)^Z;?y-J3(gF|rv7ussL zH&cT72+Y_*q)NkQNL}duw_;jKk_ZUvUb#xXhRljZMf1U4Dd`@Rmrwbaka6bOE#&-W zF47>vUu-v1W0M^)*d?aQ|l-6(tph`EYi|0gC3YkbXItNSBF41(X^~pn=7v@Mfk>&;j2@S=IIZI8l6{1kr^* z_3DlCHVO2k$j)S%2-Fv%P6a{SDCmEn(`7T9ZQQK_{HisR-;s7HhGrGYQ&drtbq~r- z(S<+j#m-=nR5kK>ub^l$#TD~AMPdZP*=OR-$nYpghi2O{Bsu9gbmyrnoKtE-UW$Vp zEs1r3lk(Ih-KK6R)*Rn^rvknQ0v0i!^{&?E!jeI~$Cqcq0z#sR!Tf+?$&5*fQNjp- z?!#e>1F<{Dg_cdT&*rty{b){N!8`Pm=7BWPYz)zhm>)r7f$(-FHWKdoU?{?fY`^34 zMLm*H#AJkBtfbO9;n6Y+YosIck@-Y?G+^PO`^;cgCt3p zVCaKYOKr)U=4E!8cY-ug=JP$l?`VW8kXGb|E2EUbl~hZl4GdT*DSn?R6QtPS|Kfk0 z6l)X=@(tA3iSq(8HfYAuRz`Or`mIdOpm6XL>6tD)IA3d2uJsQt^bfByaSM6jQs6P0 zHzpB}Mi{2$sUs5IAq_e9<|<>h5T$6S1C(4UxI!2>J%y_|qYUnObd0~g|U~Rdb`L_e`(xb@%A~HT4BEdZY z9B_k7VDx?@&PpfJUdlTIGBXp*-1IIC?xaosoSQ{KjKVs>^_}{i6#XUgBIq-s-=e-0 zIONimPhhPXk-^KAe}}TF%{i!m4ORGmP}mprNKY67DO(-VkE&CNsz(RaVyo54E?JKJ z6ea#U&<6HI{?~$yL?}zCgOX_dj8X*!8>O&WGikO77f$@WFf>0S5Q63hw3KnnB1r-r zSV5!ZvcQ6&4vMyI@1$>Wh=dghNsDJV7E4N(ncG)_1V>>2Qqd!~^Bnb<8V%1X-pkBC z%T(b0P%R4mMMJOg_3P)O^lfFuj{@e^eq&A7DNV5u1PM8`HSbMpI*NEfSCMVn%Sti< zpKmZ&kC339TzjOF_5is??t}Gke?7j>w)t~HwNCxtg(nHTwh;r;vi=aZxI`tUG)Wkm z&~x!YwxzBUL;>FBK?O*NSlGboRnXw!mSqy}FoNjts3eu4WC`--4V4(-( zDIwzK!i%!Wa1fi4r^deMhd;XTc1{%zki|emphKv{#ZIA$P|)U2g>c!jAW-YBgry#4 zvK8AJijebV6pFJ#f1#X|tYOZjLgZ$3S_P))Wu0R&Xe7+~!vvEe&Z&TXX6A@?;X86q z7-4~Q+*TeQ1E|6SMJde>TjJOuzaSy4cfJ5-?=L%GsuLb?Z-`DX?&yiF*}u8}){qi3zV9AU9wk#T=E0=(3o?(GWn~H1Hm;bw!3?y3IMv z@E}H`yNSf`a;Al-6-$idK@+JBxwZz>106HxP>^eM0|qWK6^2|5=38o*2N_frY0wo< znY0Ldi4kh_rRed6Sg75F3T8w0mHK?l=e+1rIH;sSG8LJN6(>?2n~PD8#L0dvZaTKQ ze7U7)8*u6A{mqpuC4T|ye75^Z%pB++Ip?g z!{n1_(1baU@;g(28!|_QD zr&vSqP)EN8F{S{uSI8ec)I(gVWOOJTT^R_4E%Ff>iRZ)sZBb4A$}?FpjM${$JB$34 zG%7qk5m|~>|%V;uTd_y5a_PL`mCWIq3c~QxzOHIRCGF-Sb4v{P=g(H!!hKAj4;e zP^_O&&=1>W^^q-&Ds;R7ze8O%*m4(^VE&p!OW_ThU=8!6GHh-%&n2*dVkHfM=o)*E z@AQWUrIh^thv-Z&4BrA=G2kgq+;sgvV0klhdoM>?ZJ`)-Q!}zNR5Ub+o|^27=^avm zU=hLo;^X>H9$>~~$dm_GL58Lk9E!E>Liq=_1u-E%`GcN0-Aj;?(e()rsUl{j_n3gI zxW8!Ar1GVK1@kA!{1GCXxFKXZIo9NK`+1qE$f+Vi+>+}K1&&ZI9hh^$@-wuPCBuL` zo%U{pQ%gBC;)CQNX>tsl_s4eJ!JLEtfZEz+NP~up)rPMqh-!tBV|-Qy5rQ|x25J03 z`wS&z@PtSRhc0+vr*h1uptu@usXMgTjej4tic zw_-gcK7Bw>5CKQ;hM!j?4_do-n^`gR>uAW*xDnFdnpzHS%u#~p`C{D9@+5D~e&;0V zJ%z-dBIBnCPP;(C@`B8;M&*-e5s(k-Cf<}SFKcoH@IO68{ZCJo*;hE0y#7f2dz?}eQ~A>rL7)5gYjo~_x#|(i=vx$&6$-y0C#@;# zxfE@<^g-QPd>&Ftc2mVR5zBupFA0P58o7QHDQ6&XkGO= z$%~N_D*3mWrQ9{B*cB>!CRUoW|NKpuv-zeM@o())N4HkJM&PY(U%gVAHYTm%mFoiJ z*yYK0T{*9b+fy}>)OgBE_D{6&^_#WM0tjP|5%|`C3SY+mWl}ax$h+Hf>K8rr?^fxY z?M?#Ywj&;wS+z|a`A-E5CIYG;OpR-)WS>yAQCp%kTlhS81V~sfZQIpGAOQ5Ka6GS% zj7O^F2++i(w_CN=5v^&di;9VfCv9s_8eG5(sIl`HlZ3#Ke=RwdcQ1159I}f2J`I#9G@wsRPDj z4N#(+i5CewA3Nl~UpRsOO?#jq06*-1>2m9T@@(Wo&LYSH{g;L3|4%6XLQgjTORKO0 zO4M1)4VR1i*GG0pgg)RI;Hz~&1s{cY`f<2syjQWKRBU_B=gNnpMe+Z& zdMYbQX+dxfW_zWF^;xiKQr4H67Wg&kqR3AIS)gIm{~97oXkB8P%F-g15wFg4!Rl|N zze+7ji8QzgqXnJiJm4Av7M$gDE+|{M3N!aq0T{CQf2|8eY*FSI@2d)#g1(U>YN%f& zm*O3P767w%lT~J)T4Vfe{3ME+qS7C}1HH@V@aN0SX@}|G5Qg0j)>O-(y0UbT7-$Bv z3zU&s)~_ViB?UH7=qn~7DNB9^Ic@&4GS5Z<15>rlB2Y}v{$gFvebtyu9B&jc^&_Bz zmPS@BoLp(MvzZwuCbGk-mOi{q|2Ijymc~U9pnWNz5pf6hHo2R%6XU=|2g@p?g4sic z1#Q54LpmDUI!o_O`-S9)bAiNP$P zy0{mv!!Z=(83`yf#%pm!0#qKoUQZ!_I->r`KzF7|x<#D1bpgbX2xl_3Bj8jWObK_^ zl}EU?eH{w{W=hBfiNb!}Ff1Cmw^J-+sq{;rVvUUE}hFfQ--H6!(bnw+7-Mh~nv1 z8cju~rRe@jEHZB+#c<`!i87OysFCB7n||B0^S#GqrP2;2x4L&iR=%+-dYW;^ab2#Q z0QSKJ?148{jz9@q$S5?ZI(cmRg6FL^LN;wIXW?Psv$yuTIo2dnah_l{(1Gl*`ze_9 zX~nfa0h+oz^xulW;eQZWI|!GtvYE979p(OkAf;%{VeRQWXe!~w!cuq}E>g8+bRw+T zV&`|{am@8ww=Bv+#_CJi(?+-wUh-<05#1;cu-YyFD)HIiJDgSxl-w59Wq)iYlv!uT zpmO7s>vAKYMu9W83xi>0tZL;h7`5qqnI2Yxl?B_)b1ohUskf-L-6q|Pu#)zlnyFRz zLTbVNgb+}mJBk_J_)*iJFhY8T&;=?%xL$CDEa$^!InuD}%B@MTUGdz38H!xXr)wqD z&ANGX{Nv3*tIH=mw?OBz;GdYozOYqjVUX&9wecjS?jS;HLM}7iokQ`roESUHf33SR z6|36nM!AlfE{{hg_U-qzRZVttjb;uFyjc#jbv3X63Le~zcG+LTc9Zr8=xMFT_iE3&yG}L7@fqkhAFZw zpYbP_z%*nhj%+A*YqFi#Zd7j_{Z%~q0pVcoRUso^E>|D9wZhkoZ;T>?rGs!atWgX6KBa>kct7MTyM8-;R z(NENtk_SJT7;+OYC0l6c*s-O8t{{)gRV!!%$BfIs>rPZ3*NOx>4yt``7vB2rKG`3kM)%tZ~nStrCcnVL+-HOAzWwaBGcQ~WaLH|C#R&OXi@Imag!j2&74Mx7a_Qj zlz2=(?dm)1)B|6UL1BvngK@e(wFkEoZ&Ac`wNrJQy{D4mu%4_=(h44@Vmrf1#9+x7 z!$>RNiAA1!>Y<)kZs`a$8KT@$)JcogVuDZ#3*)h?Ny(y3qLP_v-m292{;VC|RYPxIhl2E81q~gY9j* zpM%IGd!eysH3e*Re;L4pTYP4~w`tQou#f|VL#zCtmKrp{vdBFcF{}8M8Q|_X7FB^R zE4wEH;0^b_!ft^J7cUt?jn{3rs!oOX*4nYMrEcJIz^RPBP#B&tGZ*J8UN2;d*vwWf z&-@9!M7BF)S<3vuO+5a?vZ;~+L<^ZCu8%&`@D|I89G?JZGzA|48jMxoLJ44g=K4py3OVqjeXB@xxzjVv65xtqPLs6_6bloVa*Q(g^A0Ao_t=s^(Hfg zGSt|c;Kvc-(0hfMM0g ze}j_@3SJCd;Y$cuwR!_LZymvwdK~_yxi{Nv(zHZ3nC9wBa%Vld?JV5I(y6KMzl;=Z z8Ts_%-OdvW&3P?z5M;F6OOOJWn57JoM0PUn#NRF7PlxEy=VTG+ZFi`VWpsR2pk0=K zWZ@zTxTuHjHV`HkLJ_f{K6Z}@+%2Ek2afY5=!3Rjq7QT?Jqk;PEWcHrFbl=bIG1Bv z!6U!JPFS7*8zI}?S3%j$PthLMEUc57dbdHrU2Imw^`+zoO&Z`XHmpWPFT;g6{}o!% zH*}KpgQT)zxH8$8q6DSBY4STYM=bXho4Y`MGM~LFcqjzj#Xc6!uKIHvPC)h?ZlT*& zjSl76-5pHxFTv0{+Ho|SGA|eB!|LI;9Rk^wYi^M4?}Mz7zHkmMyg4+J)r6D`4GcIX z`PE`2ZXP^i1Qm~+S|}}yI`hPxIe+)XODZb5+O(f)6xd4C;EK6epCWKGPPgGhMraiS zcLy_>6D;}_cj;k2gsEuMKb=?9MaZUQbui$erPH&ANGp`wn6hR2sp?pdkA_RFbo|y6{F8D7TS$!M>%;^fgz?l^jMJ(@357%QeS}X2}$d3vIN{XW{(R5S!$` zlf^mVAhKZl+`)e?0%oSeuxJ+6!)1dd^Kgel(Ql^Wc28a@(VQEHfo(g0D(l&dlaI2MMASMNCW1qPhY|ZnczgF=qCaS zQynAcC`cTU8pI=Km!(pY5})<5@YN3i5A>{nFDE0K({`BFQBI5~q9iyh;}?!V0avG2 zL%%KOVOSqX#+vkC9!n0)`=wU6KFbHHKVe8Nn`}|Ms8U?9C2b9d1(%`fxd2M8@T%bO zmcdT*cQL(PS)h11ezTHIqr%1ICKdyiOuOujc+#F%o`MVr!q|pDPdLda)A5F?L53d) z(p`L6E7M(R>VDRX%!q!Y#7Jtd53imMZBd`?F_S!L?HdsVva&nYOQ7dwi|j_8F;&&( z<}b9u6mG~4kaBzWocSbX946)kOls<4ZPxkSi<0B$z+so$(f%M+nQgA~KxRXd0p|od zDOhHNa3NC}=r5A;o*X^dkB2J8Uh~b$%hxnSaaT%AQm<$&WKs8&{e+#ZOI#lUJ5%}E zKUom)qBvt=1P3y#=U0-Pp|}W=!76d1>eR|5xHpCJu_NpdXhu?;%&cjRNlIE|iNacF zzk{ml+QDAp73ld|RCtDnCbcN%_gTQJD8}-2zB09uixh-Ys#zfZc|vj?Xi>~w7Qt1||8-re)-WaQ3(@2kQGIl3ztcfp0_OwX zeaTBgLCR)oycC!K&+Cna5`SSo%cb9w9i;oGP9=aG4XXMv^^~pY0Q|R?xE;_7upob% zXy=0G6K|Em^$b5w*cWcRPM7y77o{s_m(l3nZ?8U?H3NzI%u+1zVEH31YI=eg{biV9 zkIj55=l%^lGld+^%lo*!T}RsmLa3eWSVgKm2XQ^0-HfI#=Mz8gbqzh)g1Vh0Q{Cj; zLF|ZEsvkHm;)S~9kw(3ApI*&)w|Z&QRBON{E={?YDetmHgHUD(Ouq#szkQ1=~& zE%w+gNTz?sUc8MQRN|4yEO6H|)ssQUi9-ha_XaXh1>Sv+WO2yorO76!f^|%(W@WaK zVrRk1qw)m;14ssU+E!m-!3iMxxi!Ty!%T8u2#A)PTf>&BGR$?C-ba5j#n)i2Q>nUl z$kIlu3}NSC?SVK*LX0D8Em@hS6tz#d#U<`MA!5v*>|-LF24PUcigxwvGnWSKkUDqe29?AaVV#*mBQLa|yNC;8+4w;i z9`-a7LXz{$bM)AOw>#fZ*GtCa>ap$TrE>jKp(MF{>k}#hzd(|rFYXn$ev*d~$fc}+ z8cO0f!9#cg6w}$QQtAVf0~K(a(IJ(OwrwyD6>wbz;yqXO@t#uijgo+l9OoDsp}zw2 zfgJ(;WlEX~B?Sl*S4R4VnT9Fdd5vg|p zAze$~>e=1?f-7dGZy6Qqv|9l##lwIu!}ABg6S;87ojGntl0Mcx{XRS=|Cd5RO%8nd68Me<9xPI zb{#(Y)nRsw;EYM$ulRjGed_Bt!=dZ@m83uc7G(1KJ=^v0apirU1*vv4?tN!d$u1e~ z!e+g-WfaqVwfWVquG?~@RwvK<;dXIGz$oM7)a*&%y8FH-r|o9bP5gST6WQ~siWp~`Fuf?yUj*$Q z0h3wA(56Q=(%xU0o6^~u^+mursIa#22vRJ8_Sg&>H<#9FxG^C{?|)NbIisAq3Dgeu z6$E?^=0*z-3o>2Kdid`2k_6nn?$VmR97it~D145d-_Hr=+y^^dyxh=<9&NSTY@gA z+}al!oz8t8CY&F(J|pY+ik`>aw%n&CsTYuDtd{*aJjoJb+f^|~rC%k!{YW%`@Ceo?L<(K{WMymm5X zRfqL)Ac82H^C1@V&|DbEr!ezvdfzDa`z=SROl*IX-aGxw#Bl6HXaMu}kmWz4*27 z?Fe~(-2N{leXSs3*lzepR3jgs$@3Sini7I4AIBRmdrl`FYBaI{JTl=^;5(*k%IOjO z!$Z5@cM~})dVZtu93S_)VY1w%I)%pwoIY>KQro`Ik3rAyF@S)H+D(QFZ!R*J41%N% zme;xaBR%(fO{7|F*2HX)t}QXLIk!u4ZQHpSS6S>|BGn0N%nk$TZEm((?)@Q{x{jwM zHzPE>q+1;HP7QUrU(y)=HzWf0%bJsCB%qG&Vy0xVA zL^kfpLO>I@H|T15pIh&Qre-CK>C9_Ns&8}e*FAh^@Dc|77YZX`kuU>uo>ZB-nZG3c?7e z82;XGHjl?DzP3MadpRwhoLuq!9Eqc)r$4MB%g+{)39PmWG{AZ(xw(%kO@|3vP`F7ezdc^iwviF6oMT!}jFS^fBzQR7feZOr*#}L z-$N1MqTNCPRqU0DGQ^N=G~;};ICWJ5GhW8|+955=Wr1>|s7jms{V>TAPw77qQ?>GS z8Ox(KnPp`x3!T62deL+E#B zzxBI~QvfSGOyJG9n_WJ7n814{K%eVAU|377MuYkJ9T3i}pyAPE$@Xt4pO=%%`GVW- ziXNAD0QG_y6U@!)f@UiphDMuL-PZQh=ZGY`IhD%$t&Rc$mrj*hU22NdhKprgE`_dJ znS4e!3^)F5EM<-giktK4+Wq}_Qa!s6EWgv+L+{sP>^%N@^ePTaN|>=%7zuzeW)wAA zOtnTMW*WaIdo2j$hlAM>TdV!<9)k||eFn(pgFSIJZvhhXF}lX*x1TOBkTF#gBUaTA zj9Ws_)a4-H26^Re!qaq|Q?aZ+9vf+{Ky#!*RJ{t!-7P&YSuCl+v$9%_tNMhy+JD`X zS)Wx%`Vxl9-@NY}9O>i|Boymg%pZRF7mpGiXM^uOvqFS05Y*$=(`eRxo1MV*yo~R2 z*Eal#3(Ct_xBm7|g{-g74NUQm_a$V~tIksjKhxGq2aoQa=5pM);s>aC3zpZ{=@X@` z2SNF%Oc%@Jp~Bcg9k28Quj`y2W!a7}f5Q}oe!y7yd5#2<%eY-_`KYj_=H^a6=uv5F z6Tm?*Gh;C7B=$PBQ3xbC%I0`Jk~m029nse?9{-LLcr4IYG2 z240j@&Y{inxT}aft+ndSds%f$HJ-_F;94*$B8guaOecVG3tT)YMlFR6Neax*cPZ9Dz2Nrct=4AZRZXe#IE8}W8 zU&eo3?0s|zxUO|~m9N?4X-mCa(`ol{>%zF~&dAuk+j+U&e}Ce+uhP(4tV&zIPh`gs zpf4V|gJ^jX61nu5cfPBrxLagtp=yW=iwBK3HUzVi>A?`hr3LC?G^!hBX7G_*Re7az zx)PDJZK?3wPWvgB8GV_(a2SLb#eLXvC+gRvH&4>y{p>+$So z*KR^dN≷w_G;6%ZbRhv-^TB_U(WjgznA50%=;chMt@*XjcY1WF;#k76PRhz;OUU zbj>i#=xriUtZR@9{tSwa?P!|=N*s^kgO}@%+9h(zoL?_>tV<#Xs(wrk5Y}97sllVR*h3r`7aw;TX4U|z1l{7P7D~+;*!_j z1Q^IEO^3Cvp2p%$NMdfTuC43%Vl9nMqXf&bM+ZmwcXK1)fcx+20 zECXUEDpv;c^pE&fACm;8KRCC%d(sM{$Jk7mqB0<>9oYr;yzwdMW0Qs%2<#!w>?56P z!(4Z)#|D&v9&jsU0BT-_WGWcdNPbNRTax7-|3w2qZ zE`ba0;{j?GhwaIH$)DzkycEn-CgQdC#V_Ky{?P#(s7~BaaLo^bdk?D>fj~Zp6X@pm zj=|fba*i~5-NnjZSuy9!zFZzKCbT$<@08sc__Q)3=T zB^{6#j-V%EqA8aUurQ>+d12VXi__}BFv&ues#treWGL`NKoUQUhhc(JVoGtLEIy4r zB#+zKgaiduO|u(3J~K4{eo(kP5DfxlV(0JIgsCgV)XmoFUBEK^*hX7}2PLjmt|DQ4 zv+Zj?l#H?%x_DXWK%856QK3fPf>O@P4K|Q4=;>*)A5LE)>Us-IV!2$t+}AHbr+eG! zhN7uqcS>4E$A%6i)a*QQJn``={dbggMN%@Ub-1H~3LWZ3UCXuCdKCZRecu6~OY*-! z`~&J_>xtxw&f^tOJlG;Vn#pu(KY*m@@!~9DMXwrPhG18lU_v~&fpq>fp?ZUf6w%#@ zHK8C_JaT#>=Wk(Ir`z@qN0ZV{^-J(Z#iw)e>ybsT(gAB)aS#)$Udc(!B^)hEJE?+q z@LmzQbbeNQ7C141O?=a=4RT^|+y_5U-;KeLEl3TmQkUY9N6eK_O`krdNaPUIL3%=; zWo%Hb77;u$IKqpPfOnFbhhwptO)L+3dQ}NDirEelzUw>m;c%&DXE*;MPubZlcSYg2 zCPt!*lWSLw6#Fe3iu?U34zhF}5r#6PqPR}Q={(Jig(&~gCtp9U|AR^@9+u_}2xhQ} zkOosHF^8Po}=QN^h+Yy|ff=j<;c zyjW7hKjug*K-$_BT8UwTS9`JWn#^2mDs<5hVjyk6xrq zl$5#(jJ|QM3LW@QLHHa{5f}%Vhl6s7hhmk5be(OV@+Xmo#GKpyREj${3YZtB2#f-j zUC5&8WH-(IyXAr@0Kf*qD|hXqk3ytIGhPLuleq-ebP#pLkfnSe%5No82)6D^Bh0U; zu`i~C;EPZ(R3SfLpjZ{ zD^3dZal^0^y|H4MOj#HJ?E&!c46gXTiV4|0M%k9yR**@y~mR3JELK+$?sDGAiVP(?~; zwQl~eBZUi6k&ObD7YzF>g;4Z zkx(U_I(NvW8wuQ}q*Tc%Tu!x}T*{OR$y!no_KD2_n*)`Z11S)z%tZdew1HXH&)}p% z*#A;P1+YdW7uE^N4IJ+SK67ayZ^dFN;V1K~Ke6l`=tLDw>Q|OH(oSiOm6<167&>)lTA0>KGDF4Mby8#{U>VK+qFi zFkp?KOByV!6M(xvc1JCXZWh_FU%O222o?}q@UC+!R(C1}Y(KCZG>L(AS>y?IWiTjf z69tvvW_wHi$z;&#HH6NlK+$7?vQf|*)K)Gfvs$VYCMIz(?J#)_RTXbpmF*v6U)db6 zIZz2Xu-`{r3F$kiBnb_ZL4vAKFnRrh|A^b@A_U(RGg*l&{dg*;by7iRZBhZR7Q*f; z*d$cnAalU|hQVvu*;AV4oEgUqWD~l98VeFmp1|k$gN<|a}%2?3CE$}T8iA;F? zW#`VV8IiJApZ@o^vu1mvu{Zzy_|h3uWYaC(h1Tofw+YpAoO0cmSmQ3%x%T_{bHAAC zLsF(>$y)i1ll1`k}QZH4?Gos5M3+~yz zkxk@{mAQ*0qO8Ja%!NI=X*CVepn;|F(?9=c%Zf#AHk(ff{#tg#YX1mc%ORGvMdWp| zkq{mMw3?qq{i%+`I93s7uyCjOsN}iDkU@h0nh+0@OPq!kZ2M{4u5g5g@$bI%*|@g@ z@p5l8#DY(rh#pfs1Wi%$h+XQQ;G()P&c#fX36ZREf($VjerS#X8d+T=J9;7sUecG9 zg?8@_S!J=qC|MTww*wl)_|;g=)=U=Rum z3-7<XHD;b`Qe4?kSHdJSmKwgbzTFXtp`Rx_raImn1Fd9Z+fVBE(ajdsJ&hO`aH*M-9v!c_i0>V=`s1}b+nKb#b zK^L=r2cKCKrgiePIUXM8zb ze3x%sy_rhybkcol(<1mFy_F#lAZCR=1>qS+tM=Jn%@qno*nACEs8tZ-cKaZjj*)H> ztdGYwZd~8DTbKMDThu7cM)Roti<-=E+CJDEsI(lYcEF9}X{`kp%(18t1PcQVe97Pq zew;uz4V1iT){}de4VXsG@q?DZhQ(?%oLOKx`gX)QgRz6ns*>4)f0#7`};xF7F|lIOL%g&qq1P~1hg z2NT#6juwK~$nlJAs0f|~@C~LeST(UpGJOC~kjIJVuW+agM0dvMqBLlsrl!prpL4;$ zC;s)A`W5-A;aQ6?JJ=kkavb1`s9pi^K_tZK)3Sw2rcRo+Wy_B4J$m%(*YB&TllSi3 zo10tjlTSYO1$-A@@|#9Yj^@Mv_f!A)?WLF1%c+Be=>c&8dSFrVV+?oRs8(VT<|E^) zmoK04*~A^2H+MOucmLD+P5W{R(t7_h&KdjGyPGzy^WzpJ>O}JFJK&7w#~i(Q!S|nx z`>3S2?5wlSI{u_?5g=L8Bwj=!Pe1)666||p-d?<6!{Do~rek=2?3hK1=X*T9ey0!U zcw*N`DERC%&yeP$_ugH;ZvEijU%P7g@)$24!zp~Hz^P5-j6U{6+7*2GDZTD5I=_IVew$0mr1jsrD=R`Cd39idRs^CO<= z-}|&LCQU9Z*?Yl77p7;{c>C?QVb?{2E@{x@Xu2Kb-_JfBKYi-#jEsy6FTSW*vu0p- zX>rLrZ@#r^`Lg5MwP08KnA~Bpubg^B`3W$^^KaeqrC&^()T2kQF=O7&sa@;x%P(KE zX3e;9r^o5%Rj&`gZtY>gfodmlBLd?bb<@ga zzdQAm;GR8=>*da0@SWQcnm=>;BY(f^p4)G$pIvL_q)#pz&=0YW9em$<;<4R3wg7^{ zXw=Fcl)da>fag_oECk7wTURZ+qTi`S#k=wv*Z+RO_o#}$p8n-$<30?)-FO6lf020k zhabK@_RUut)~&Vh+u6TA>-6%%Jy~hq8!x(G`SO(>w?yDbD2BX=kN zWZ~CUiV&3b_)ZB&(%jA^i{^Fd)>S$hN93J1U%T$NmoAz6O{8$opDww0@P+3spFcNT zwEGX|4cNGL4U+iSQE#kR@I5`rH?wEmd)v)VKlVsYX2w5<4Snpv`_W9$Z!y}SAQUv{ zu<0Beq2Qa3J#y9MmxYT87tWk{?WKb_kCc_n|7yks15S6ugZy6n_ha|mbX|7sj4hj2 zUv_q1Z>TUmT5|L629Ny5Be}I}e*VcvpS|;zMnCElE^n%GT(UisT&6m`tvQ<*Un5E{rrfBhup<;LV@##-~Vpe{O=pr%e&+H>!wbc*f1}5 z@poT8I($gL5q|cm$EHu2+USU*vg_v6Z-3m%B@2Dd$h^687cX9XSbW8HBdQ$-s-48G zo+=hw$L5ML=jE4PbVQS8K}mR;UJK>tPntX>GrQ(hzrEz-_9x7n`(5)^?a*YSRft8U zQ30zsB5RwmIa&;0MkD$SYu7p=!OMShN#iC*m6J%@=twn8O&1Tk&=D(Rhnmrk{qx25 zKB!Z>=8e}s-2eRZ9)4gL*t>JAQ|htBAEhyD51XP@|I&$9;IeCsW*JvRLGv(M;y+5mhDPMQ39X24%mShi@v z!Y?LFIHT_=YLIj|C<#P`7e7pY7!7ym*7?zYJj~#u-Q>xBulLD+ zJT=ed>X4R^O4I4Z+{Qid?d-T+cFN;Sb z9b32j?(117krvLFHu}?vt=qP7#Y3Gsb(Gc_R!ifKj+*Kzm&b*=2Khez@WT%u+pbN$ znt=!J`unqEKkU@0Q?rKkhp_9!6RuD=GWGM%&;k_~7cQ7LcgnbOjR%=(ULg(}N<4;DU__2e{fvU%WYA12yf)z~Y*sV|VlX{=o7l1oL;Mr)00*svu?ds?Y{YTjvK`)8WQ}iN-r(#fVN<8`p~s# z6wkaM3RbVXL-%eix^+44cWn@?nKx}Fxn|8;M~xaS+a1UD3MkKe z^y;~K=_1*EprFN$LQ|~qV7UC{*IqpCxVGK8cAPzR@~-^tnoC5_j&?UBy#w=nh^@@p zZCt;$$q|j|bg-#$ySyiK>Ku0a!VYh%<4)|-qqoKdF>l-U9d~ctO8=qBT0thAmHXH} zr}#26qb}but=bl@UgMEBT1oQ32%DWbd{{05uQ12uZg)b*h|kyPn3ni2@6xp=s`Z8q znxOH{-?4MkI(o|Zoj2dw;fVSjn>1!GwXeVVe&veQj1wrF!~#OZC;vaQ4EZQr45#OZHFT_vUEr9o(T?S*H%ozT8}rw-pt zpSE}BE;`xV295r3*FBHiHRP(BZ)<)`D?Fex$y`6T!Pd=N9qhm<<(0qceX=&iHU~6g zRLz3yCt^-Cu%kx3Idb&q2~!)#v?-HE zKJ?&gFTb#C@#2B!pD$|@8OS@n_~MH|ZtfS8r?M%sNQ_4EcWi4={|KpI(019u=0Mft zK(zyI01Ciavt;S~Z{~FC+WDf3F3MlLED9XPoEw%b`S`uD<$HHM^Y|lqA>^DY>;YvB@hnIW3_yMQzP@z}Q} zjCtq2A@@WJc9-wk$tLP&^gnCDgb5qgZ}R(U%=u>i<{i7EPNZGzteswC?2+57`Qk;( zks`Zw?t+&0-;e*JaNDLu-+lewdmo&0!5~cAuzId@RMRC37etFoini_aXV*TeL4#FG z7QFk;n*ci7*|6C|EFO{)8f^n))4RA`N4IRXZp~UOJS4FLtiU%-kolupB4xE4U1`RV zD1e(*gPl_MsE+_RX}U0#eqrG^vq!)9tZ)#76-uioArsfaeR?1o#n<^85>rZPxtxSKhd5#4|!VG!IS>W-7yOy@kyR&OLoV*JF?Ce(dp+ zr%fMl-uXu~YktYVi#oMx^W2NClxsUOiQ+;*rbFYgORl)`>3=-dy-(lly7itM{nm%$ zKRNZp&V$cBx7~?duD|sTRqFL!Hu$QM|9G-jpT7P3pV^?{k)4n4bjjscbnnwA=JAx% z$$0%yuOG`j@Hnh}bkKNvTD5C8W7;%{+nxaX$D7$pG!aa?A}(VZiE_pKKxnrw=x|~6 z$F}!GhGTB#Wa+pfpfG8o@lQVZVAs}dU^Fr{N6agQ-|)xnfjMlyN&g{}Kxe!RSs#59 zc9(A7b$`5N{>nA|PwLgD&uLSq&4iPlw2YgF4}a;2f1P~nv9G@IMyGyflAXc8Q%^kZ zh=lyw>k|fRD<3w|L?we+AgqG1)S-Sw-6{Hf_A`q2KVv}N5lv1!?u1JQU)jA^ZwgN3-hGANt@!BOZKU_3BmN`_^TPo3?1q<9_(z2VZ&RC75Fen*&vj1Hy}g zO*HdA#SX{xl?97-hjSX#@nPvD8A`&Q6|^4CSbin0|9}2jt=Ly4Wq{E^mt)(u!kRv3 z%O*7fT0IC9Xx~LyF|)pA@9x^QYa^nvPVcD)hp$|@^68ge-@0pWt*rF4w2UC)92#P0 znTzFz8d(*sDE~fT*i!Idd=Ey zw-=>Q!LA+sk8d|?#VUtCll~6{2by#iATdNRI~xnaHHWE52iS#Hhb?|Em1j;kSX{KH zW@Zi85(#UFgoU}<1qb0O>LW*V=-=<8TfOVPM;UU2LP3oO($Y;!?ZbcJK;pyFPzQO0 zBkve|Gnc%tv2wR-UC6Ou@%H1I<@wA|Q}+&XU^EwEm=$#tZQO8M&&IWXSe+S|X%u+a z2EKyo6~I0=f#gHxyRVf~8=&Gn1X#5|+wBiKJ+AnO>Mu}*&;(CfCJR~EKawpYGjei6M*WTr zjAmaQ+Ey6)mD%VSXr9Lxxc8nRci%l^eD8j=#}n{ssUhZ#;2_hC9@g1NLMJ2rNZbcR z^A*Z;9n6>dUD^`ho5n^@?sjsUHf@3|q?DdpT`;6dAXWz!f;3XQz2GtyEhzTsStuB4 z=mi{gbL()IOpxjE`oj1KaC-By+zmGJIv!xi-;|xLs=rzK=$gC@`U|A z*ORpP+m+$sIX=K#KeL_$C$+tPydOBRF+3P4a_dZyDKU|eBZ zSG>H`7YPR<5ftc!TekREUXNxx9uLXO2N3B@1DJ}7)-9bED%#2FLBvvoRREWIJMjNWDyrgj3mVzCdgT;INtlURWj?fJ*Bh53*imjiF zQRrHr4FNWK#T8c+?A_x;Wb7E*)VsoEYZonv6cjl^<_D9bX1ELmtB2B5Pc4-A6*#6AYIe1BqR)g zo(_PgK}vOmEIc(m2Y?R2Ff>M=>T;Kc^t_0GKVed!=#R#mwQN0Q!P2bU2BO-e*GA2D zusKi#IZ*9@8%Y*vjg{gvzL{_1fi2d9p%^SOmchHOSDr|9v5`3lep)DfJOc?Iiw6RyW zE`tURJni@s?)lqaS>kEIErK(yPG--X*7KMagU-L8Z^sTFj2(ly2}mofcRIt7(C6dF z^*jF9(>r$T*}B!`mz=w5$$V)xoStwHQJKMjT#jIl!nEx6EgO1v?|$w%XZP*c@$SF< z*&QxLVg)&+wQxkfp7G_$&6-|z-g!ORz|yhkM=$}gy2*Gn^zr*+&p!UxQ#&1ha;xT7 zUVhoiWs9P*@V!HZJn`5+WCk9K2QV#^I^Id|6w&0afqa%YvCfqa1bqoQ6VB<=m}_8& z0QsE3MsPX8U`0%kppA*97j>4Tb$TEk2tlq6ST*JK5e@iLv^Z!7n*&vo1J&-`1&Xi} zj+Ykw<+^Kce&CUBm#+Tv@CS$8d`nrdoH09C>37CgEu8z>BM-hX{?nO@SNH7Eb?7~J zfpH**UyIyv0fGWLcFLeZUoKrda?Cr^KYoAAm^Xkwt;azaL@oytN=qNQd&s?a-^03^ z`|lglrftiw7cYKo!e^ho_VR*nXUD_x@dR2$%gb)Q?&{lz-uvCM)z=Su@cuvE6fP}B z-C;cy*2Q=+D6Cg^`ihmWPMBC)RQklDk6|VVrb}RyNuGpC&?^=RK6LNh9gl4@bMAMq zPn`Jq>#xk4Gn4XV0TXuw%Svy(>dIU1ANtLTmDk@t{DIqVEic+D!5S-)xYOrBbdF@! zuQz?gvKK%9ELyVr@kbs&d`7I5nuErGz2GKojyi`Axu?T%ZD%i8^1`Q|em?ru1#`Xx zEG^TNP^kQt-(U6TVfTN#Z23*Y?t9>t+k(3bQJx?&(+!}N@KQ^?x|5c!cEetc9*qsEA!vfh;gI}Hk(lO3 zaPeqtA0_+a9EJxWS6mvh$e<9ViZOfv&EbZYo-+PGX(ifB*H<+^U7PCa%zSD z@rai2A;cl5q9+Zcr=8fPbN#k$ckS8%e2^FMlR`kN@$bFYFt_e0r<{Ue$?VTQx#ss* zF;~~Jb*t80J5Qc4!KvmLIU`=M_u`8#LdLx4;>$qtZ@!vgL}3Zi#;g#xN^Vae z?U*)gyL9fhW9v4_wus4L$^MoRGc}eJ7k)kA4f}kn*nL9!J$>+i6&Q( zQVR=9=6p8csw)Pw2&ql`cE?c5iw&-%(yTsY|Bix~k7qLx_v+u3sfN|s(q zzaI=~ol!KDmQ|~HtCpQRb=J zrm^x9QPL(^IwHC;L}#O}p*i$2>T=*DIUtN@LH9#uX@mfMvucT=7}QuLSX{D?HV3K} z2ddrVB$Bho>)NnkU2f}^?zCDA$pe{@?4}KN?bymk4vvRh?sc1Yv~1N5X%z5kc=S;q zL*cgVIaxJXKpv8-ISjhPJ2$NTcFL!-r_I{FV&(adKhB&Z00_9dKIo9O&f3BwjRe&P8XvKFb%_tNNLI5#ZxA+X#Tyo zKK$#@VU#Q-2l{Nr2F(942jFmQ+Oi59ZQ7y@8Nt3|TD01-bvq6NNShXSZC<~jUc2KQ z>1oUzGV__!tiiVY_40krtYgfX<{>^_T(W2RlnL|aFMR*)F@L@1KI}vQOnYT`sDILF3zXr3u4S^L$qSttQJQXpH0o)I zAReKfEKF=t3NpN=01_pVg^;wGFm;%67-tZaOhNHJtxI);ElFMd1T-XdGNwANAen<^ z4-}C4O(`_ZMBqe*V3oj-LMb%Bp>~}^twsdY&V#WE5MdUnkRodzc`*>>R_&&J%8E6o z{%KKw%2Z~&&y$XLy_=@|L@i3F?v_YeZM6a#=)=t&F-Q6IB-|!m(ONYOB~}uP9rTP! zA*~awOc|V*LhQpqP9mFOz9l^3g+pQ@D%TuyzdvhNBUmH+|>sJ+*4qL?jG?jgDAaMrL7gk@d70 z7Vu?q3$4pv{&v^y_3Jb0=lZ?R=p!IWYp|vB+|Tj^Ia=fVo&MvqZu+Fh5ZPSAxzLE!9o^`Gn`ui zqySS$q!SYNF5Q3Bdg$d{fG;8QFMmm%$ek!7XDb?fAzn+47A^--%JC>RJp)#RO3P4u^WFr@%Ijp;!LZH;I>e$ivuZ@j%CKg^j2-QJQAt^y zoLt5XaE)3ZggMfhb?bgI``hWO*I#|z4S&7%4-6DgQrD}O3*O}K*cHXOikX=uJ#Y{F z(|7T(Qtzi;R)cyJsZu2&O3JJFQ=3vUVIn6YEI%bC7f%6bw(w5ajnc*Z_SGUGR9B=| zNy&3!h>yU1gKQQJrZfsv|Ig2&L{oL# z{)hjQrTsA8zCM^7*!P?cW-XPdYzcrKU%mPbH!fKcD=Wbm1OVEzW?jR^jS!IaEk%P( z8nbeGA)D6Vr+?MTHOREN4eINSN6mtCif&nQJ2GgzR;^mSd-vYCeEH6uJ82}~?gQ1F z&c;VI*}82T0(--T4T)d5aycdYaL2K5;gL;`qN~7F6LuYS^Kv(0C{kYL!Ief?>E3ng z>fwM1I<&?I*-qBtM%Ao^nrEJV2H3Fp`|sHS2R#a67k8ls46mD4kI!!Ps#Qq*@mOf# zlEp_hYYHWf2xi@}h7B9-Sh6%yT88O(SyAEMwdvF8cGg6-UbaCY+AT5SXxH6 z7Az^-y=Fs$#z$#MsxSo_aUUY*RzQi*7wF%=KM*u;-aKS;{D330*UM!*fh~;m*VqmM z^ag+|R1m6I!C+155`-7%{3w{W+TfQGnA@p5TvX|Xd?pUO#rU6F-pUb$aDdTgsM(;=`T^WN%!9m zl#^<>&Qlas$U@`~<0Yyq{2)F-Muof;ir+8RzBt$%;2o%D0V{=Qylcgw+b2z2P*FDs``#N#a&;v#Uy+qZ7RI!wHe)To(#Y>z%KJUs&E)YGRXj_=$V*CGhcP9NVa%1$|G-Mo6` z;%{c(H*^T!N&D8VzW?UyS6>|Ih?IkIxRJtIu6FJ0cD+u1{@JIa#d~H>owWM9ug^Z` zoRYoypM5X}f2ZBMo#aSMd-LVzU1deDz4&Zq<0IO3JRUs8mOSQ+c+p-*OMKDt?W>lr z_pd+7z-BO>w91>o|Db)m=hIC;dgBjdsHnNz-4{`J@A3_RZz zE&ueRaoe`nlx`jYh1XR6yw1xpX3 zOH(bkOajtaIZ`tJZC1&z=Jgpz#>?z=O+I zu0H9QR<~St!`&mEtJmnLc?%W|zvUJr&9t<@6C<8}a>UceHET9{^yrZoP zkJQRQrZmfT?mfF_?<)ph5vWn?jgLQRSiivs@4vficRqR&(PPx~VB*l>4^T_DmMw3) z@x~!fkH~9u#KJ|3hyU@GjT<*+)W~{b#B=}p*S|ZoX#U3N*PngqMO5L4;#9+`_h2{_ z^LmT(cb?g&_mx*%Q6szd%O8G}SGV>@<38BFeLE#0*tk<|1OJIiHbmP=7XX3B>pU*!3^~Yw!rj|wxdE3GwHi3yx;~J_IqP35F|cz zl1N@LTZIxhaOD=eAw!Hso|cR$=s_kWQtVBNgd(16F%vzq?vPX$ib(*Rg{q*x6=P)q zs}gdtxnVsyE2Z)!#CFa!-)Y%qSn*cX=(ypysgX3AL57pcWyoQ)1R2Qw#2E7vQ+0&{ z8I3TWh_Xnt;20XnWl>_q^FR|#Gwm`7h)F0?$YmuaS2CEp^nt3yDnosfl}u?!Y52oU zZdkn9m)qP-G%Z;pydqd_(W{mURyO0SWMjEw{?hI3o8`$D(6168()LyIt8lu~RKgHg z044ZD;PmA+Xow9pFEkNgOJ=QFpdrLE!|2P%ZlQqX$Xkkfn0V>7 zaWcxME9Hgkn(=cqWcctQLx$j~4mU(?7aniRY%wFfBeJ)A-KC+ zaCdiiD;2CoN?mAiN-5=O3lu1Y;O_1rxI@qY2|+{LbNv5Za}rA1=idMOx%X-N-WPUH z!#?}W9$7QZjGie}vW1ZauL89Mh&v8~o1O`3vRa;7ww-SY)B$ zc5fFeY;;lrgp~DP7(ZE%tx_uV$&#aCDDuqB0TkRzg=muSGg;p7MnUHycP=H~(vJPhj-<|g2-=ATSsOttaY51*t#s9pbjsLuwCx3t~!4K$+xHN5N<%Tq~|3(k$x+Wlw{S8?Q+!527L~krJ!| zSNHGVe>d{Mmy@SLD4L0_F-D`r%|cJAtS5Ox#8GCl`Utd(4=05o`|F%U2$(Z%GRkq0 zN0l`*?V%tVgPT7^3-gf1g5A0=KL0p(-dwP!$U9eOPx%rXcr0cJnB_9KuyZHX-p!kK zZQU$DB5%QI9HIoLB2*VocPx4EWye)XbtqMkD#;(*2 zlP7%^`}C0v0NDi=gmnW^dP*uE$OwKV_Uo4~ z5T6}iQ8M9G!BfQHWG=fdacV1K@jmNwpO=9`TRyOyE z@$tB#B0*pvBtub(kW0E*=8c=vk%i>*`ue9RkkzsORPld*xeQ?@zftRY`Z)65of~+V z;OMb)*S5ur7Rn@Wk5Z+SQ52Xs8bL%c8TT5jO7M8JxwR!gjncH(!Ai+YO*)aelrYF-mGZ$FW$7&X zWkS)SW>Lu~3b>J!-~jfEZo@K_m`N@=2?f?llq8Ml%jE^^L&y#8A@iDyB!-D#Fri4C znwW==?%upEF8@yF_uqfNcmF~1p~iD^n#~}u#Q*(A=ZPrYDJMZ1P;j&plL@I1DnIIk8r!)va0rjB!vqul_Tn`fi7>TJX zH38h@2p`e;KwJCXUzaAw$134fdR<->s<`^~?R8-P9{P>?GfqH%KvUXn$BR`B9wXL1 zuCTDM*K2mM)Wb7KW)mCdEd`bDwA#VUOC34!@P{$QfQ8Ii`B0Wynh z3~t068IH_9q-ml{%W#!=)OeIJ6(lAq3w)*|NfBT4)8`^qeuz^g|V!_SrT}10jJzDNj~H$8+`)62#R5_m9d?Ye@ywA3+VC6d3!SJwhza+Qdjf`F3Ml6v za9E9i7SU%#*N&JcbDnj801!1$xWfq*8cn0-El=ORYp_}6p%~Gd(%CCTLZ@kR)|f=D z)hQ28kzoavU7lL!^e;XwR-)8LpMFXX*iAw$QKrLLomLhJG|6SN%$Pi>M2X^+E0m`u z{6;n8^fH5%V;#UO8TSwioP*!Oh^3FInQq;>wMLV+bC>o8Fc02KqoBN)a!S;4{D_Pa z01{4XXxhBl@oQHxbtet967CHQiUSs-L#PAw(WhmJzjEU;OHf9u8H0ao2I9C;Li@r; zg@p7ArC&WXD}HQ`-kDkDpL*B2@}TkY~8wV?_PO}6?L!s zpb*1UNWlO@t)>iKe+S*5f7V5^-@ar&eEsi#o&v8NaHBwF-frKx;m)-y&mKJZ$z<&N z#+zZ+Zzjekb?n&j9nTAaoH)+vQqcZ~#1licI zZdF87WUV@Ny7%m@W{OyVAcPX(ddUxS2M!$Q>+io{!GaF$S{*uch&6SuzWvISudJ8= z@)c4GmeID$m(HIKJ#Npib?Df+Y_%#Vd~q|pbpG6doqLU1efth=kAxiW_trZcJ77sk zT)%qN&0BY>R0}GQI{;CxX}h*3LQkc}KKJ(WIu;T-dGchtJu`UOs+)Ii<}X}m;E*@` zef@#Lrw^a3Tm5TFQgV|fjqgT8)M?tPWSO#wvCmH)J9_!T#h_}~F?PUfbp6^@k#}yz zJh?Y-j;8nUG5hu&C{wOHM{W{fB;wv(j!dHoWy+N5)4!jmub(5+vT)wqE*(4V+`k`> z&3=7)lqy$|2z|5;aowcH%3L7$VZ`0dTX!bK#I$YSp=RSoVEUe|TXJ}LRd3M(^OU>S zZ=61NcHr>gcO&la-@awy=n*1Qi((v8#bA1<75;{b16_jP(WZ54 z@87xo{K?~avrU7BznLQ-@b>N7TeoeFjeb_TRLNe0hUW0~*|2VH_^n%yq9P0?Q@>%u zjV902p&{qboyn6cpmXQ0f%ytz0t9jk1R3ZxE74dr7E-Z%gCW!OWnAl5^>C6k8{b9!XA!_w+D_gs43G^xn05--*+wnl^3PtXVT$ zNzshmyLn^7hK&}hvt6goHR{%f2jNb-d*_a0$B!1upFf|ECjtQgjTBApU^d7O^8s^N zz-?H$ylm-Gw{Bj)dGkj5HtlNFskdz9%BV+Ax^(GMvsSGPt2I78{>1Uamo8qcQMGd0 zcJ0mnfm^rjaAcCv4<$l)VL9z1?FWXO=-gZj_^>4&Ma=bSuy_QLtgqsEL9Jsu1L zuU#Xj?}>E(Z=i^00`IS{zk>oY%U-QZX)Op7ih+h0xQR$8NAB+4vBgX4yng)9v=PIC z7cUI(^_nyB{kQw|Ui!;ofZ&VaLw9amhgDm0YHX*r%`TljQMPE|xie?1S+)!*5@}SV zOxBk6%+=q{NRNJ^v!t({^UbhDx2f!<0s>W4Yg*Z zn=;dewQZU3^pTfAbN|xC_queBxOdB>x4+q^%c?okY$=HdX&;UmwP5zF0)c^hckcN3 zjR7YP9{_PqpE@yp(&v*u{?M73feL2C;DH;K|5B<@{v*40eEq?@-W=SMncS^egWbEf z=FaImcj_05CVv@m0f!C;2jbuYX)~r-kB+)CwqK8Xw^3_Yj_lim-UB%k7``2LY3g3QTM`wXU#A<9Wjq1mQ9~Be)K4Bz5d9ytz!lcGC6G+$8!t>u_VQViAhH68J+XO z@ne%my}jbsB|av@w;zmqt52^LOBYcqM%SJ#>ovBFOXp7wY}3Y`neO9Z7~7|3_^s>6 z!asiZ?WB?Kc$m#tc78Ky62h_c6oe2_ol!>|L}oAy&%R~#uOkNbxOM$vdScAj0o~r{ z+v)bT%b95jBfE8c5Psdqq?i8z>{&<`c|Y>_@#C_lQ35tPLaT&vXAK}i$m{DLq`)gDaWnaur3l5R zLqZA_Dl~fRI8pr)*Mk+MH3KED&ck!rvSo-r@lp4WY~Ob5-UDwBkEEEVr;Z)z(Xa1{ zUw-lTGzb5(h!uH2j+|R}?ioCID95#AiUL%$C`?hE-9qSBmg_@@4t;OTyVlh7pfVNC zo<7sMb5~R=XtdFi7b=kd^yN#)WR}NKtJegF9y{8jSI+~xcGPR#>Zhf@AiGv9QTU7Z zKS24InUr*H@80!$_LQhp1&w9XF~iZ2Frj@ai1E{AysSQmI(Q2WdjwOrcPoMH--@d&= zLXQomZcWvLGLCBG#jT$y!^*(Iqpdn)> zq}l9^if=l9`b6UnJvf5hg4sDr2~?{t#}{9G(Y$$cP@q$nZU+w>C|9a97{p2&k0G~S zUbbAt6i1Xrq(>74|3I?}WZXP?a^0TorK?oVaM(K4s^NwSuvn}kU7EfF28JFBDPFAP zyJOyC7h&W(GlN>RSh{p6#|(At)w5guI%kd_>)ogO(IbaS6)!RF-S-?u&(0k?4(i>_ zo|$^BX7x9E_Kv)Nk0a~g1F~r>JFkR};bdq1!-&WnMT(j|yify+9@*P_`LY$n){TAm z;K;6B$FJY?^7BiNevBhkmp*;bLQr!YLs=8B*qAl4`SBqqM^Vm2i&sELn}>ORCagF zUJjiJFoh7fL~1!nOQQ?I&(Q3FS7Pb%6%u1(4Bp-VN6C^U5AHu;u)88|+)j#&s$QzJ zC^2Q}ZUU@P{<3!Fq!sj*O1Y_Jt<8@$>orQVyS%)8iw6ZoK6(s3p%7QfL>rEmlc#@u zVCODR51S?R@t~T1fq63Qx?4Bz0ut>SHx^B+?35X1&i0<}M%d*~KKp_*zB25Ze1(d7 z#$ug1+=Iyl8a z!IQG_){04_meVYWw{1YWza4h%{m(zf5QV5+d5RYI%NH0G5w6iR``ip)v>qq2bB8fWF8YAD>xSFAw2o<&Fda?TD%Xu<{4aQ9oVxI0)&U%3O^CDd+lmv>Y;sFKej{L z*1&PY0GmK$zxws4BVb*pe!ZiiCj>b;P=XL(h}r1tIqr_?VDzam)U4OQrZuyKC{w9g z?fMO^Fb-ffSZzRtu{bmgo7Kb$zBah+N>PlO)5dYzUa z!xSu-pPR>zALlDv9MZ6{%z6rD&P0HooJc3U`StaWQs9;2oGhfuY5h}ehp zjd}o$quT5da{_`*p)MefHm94zBFvs1q9Q>RskDIT6Mz2chkN07_a4}H?&A5t5+#`b zSQfCi`SeLN#bFl6A>~XK=A<{L*`}tZSuE5F+{KsyeJrX2q|boB+*s)`V@sDS5BeNE zb^6G~i$^YBK6L3ywQ4oJy}je&;%G&t-IsO#K?8 zFwk7=4n;1M50FGw;#a9P`1pE1h>8%`BsQ`vw#;YI2&#f|#Gmr=71BW=(aef!Qx+S5 znb@JA(9or%gZ6rC%)_qVs8OQ^Dk+%R^Vn#FHxfU3@{lb-D||`4Bh*F2WeKj;unf|q zq@-qM(t6m*m8)T_)af9EN`=B~n6V2f(Y4e^WAO2gdianMp>3+wn*8K(E`NWi57
L#Sy%$FEk%gvO{bw6ak2scACZVfBSfZ(yYnwE$U=tALUqh0qk^1K)RD97 zF3Zj;jM!DCeEETIy&ZbxO2{Sp@#3OI3-fVAr&dF+k(~VR+_^)Eq-Qr2wnrP?7z%7N zdPzfuGUTN9%tn(J-HNi^Xy#A~4M8G5ee12!BS&1m9X@ONGW(x(Hkc51}YmusG>z8v}l1*CLAN#f;V7;wk>a8xpe0IIZq$2n|JSAxEaP#V+?Nv zTBHN+-E)Yi7?7|oMSPQUfn81u)*CX<#Z*@t7yUe6F2B4veD`kJ9vczKoOBpmZCbV7 zv1#r7bD>FhuPmQ8QoQ=|rac#q?sdgISoXslgwh6e>xoQFi0QV}K^|s>8AHUo&j+Q6XE`-VHt4qJ+6>--c?#aUb;}7WcqqY9I0pzF7q)Tau6GqJT{QJ^q&+>|Nc?d`X?EizPNB|eAci(; zD5KtCuwmDZPE4>Je@(WYEE zD>Z7dXz}v6hY{CLg>GN7!Q1Zgwx-ykBj!#1T$h%pb!DPrf|-aDlw?o;7{yQqBO46i z=^HqP!_F>3dO|Xm+$D>b!nHMcX)w$L8}Zb{xUu6#f66g*AAbDNN1r4l#<1BLdG}u4 zGNp)N{NUl8BS#KNY+&+Lyo~`YqJj(o^`Wm{zJa)}ufK%?uO4vYgdpen>f3efaOvFT zx)mx%J$)j=G&asG=CRzBGq#wUs5y-m#yuB8I!Bf3%S%+oQ*g7*nTg#g9Jz zJRvbfKorPfb4zTyfgyv_N|MorLf!8%U4>uW@F)s zRS}dBB)j(No>4O+2w zf>2@W`etMsjvNm67Av@wQX~8dVGW=-sj1hreC9N|4p5 zt=q6ktwzn;wQSwFd(Wx_F~`tOZ@|l+MUI`LnZ;0zg}c$y%V+H5=?mul*r-~K6>HYi zZqqJP$lxKz1Os^zflJrEOSki9FVw79`SG*3k?)SJ->}Ju0fXyTtF_m_h)tXV ztRZojVs~4aXD~Jp-Vw_IRW%-j8IY`f$Iit{m+RN1d+mx97cXCKw+W9PJ$C%LbLZRF zYdG@VajiOb&OnIPXwO}|bUgHw$?PSA6ZWp6!H}o$7zoAQ3BwO_L-NN~RR^85g^Cq# z*{}cLp1qsYty8c>Y4xI(5I1=X7U|J+gP$qR^DHWzz+8R=K-if<|jjqX-U56Sij8K996g^6%$gaFjdpt~GMkBv9} zFVc{`d-twgy*fBJ*s7H5QfWq!Xwm3gw+*?w>CQ5t#AEh9WR+GJ0rCCZ!8SW@4uc z9LDzn+!8s616eGb>Z_1~Fj2C&C4mvR(dazAaxkwzK;lU;kua^5xx=(|S+p**2^STS zL4g_uyV52IasiQe12VIKXZ|8#dFIF|@jYY}N-bPRLqZZ_Vrti_zWLa;#miQ<(L|N}GCS=O*6L!g~8UnUp3TQHbj~aGaaZy4nHKND> zn!Wvu9^Mk0g-I)12}#aF4%-!lDq;UrfuiQ6v7*rszjqM>T^uh7dRj##Ey5}KZpiAL zD-eN{tOPm{pJ~4Mn+P8lvojz}i&9==G<%zko&;}UD@+_e&?G=^S+Pq>!8n7C{84)F zJ1sJJdgt))M$1DoszGHSGCa@I18kBSa&j9FEdY>c6ohZ{9uhv{Dli87qJu!iEr z=Z8G*1{!(-F&80Ec5QeOIVg6GoZx=4;!&)3^^lX3ld^*`uh)acfpuW1khS zTJ`%4n~Rh#Yq6m|muZimoSaSIutz_6V9T%-EmaZ_6W31og2{CF`0>TFXCkE4s9pQ# zmEW;pr5YtdsuanGOQE5x6=buEIf@6ls6Oyak^J?9tHm!-oM19i;vYnYmn>OYsZhZv zb{EBalFYz7FbzBvpLnoJxB&}lPCmmalDq^=MKpK9y|8O0FnB6JY4Y`hA5bF5Ol8>h z60uz&ue@T=Wk5;k4XGr}gh$XYf)oh%Ath-G-Sn^wTjD1xq7j2v|$^AJs$&rYOppkpu}cT!M*A= zvq^0g!dl==!ZXm7Xj(LH#oCHAiiYaMWEMm&13DMhSO= zfj|muD`^UpWU#4Lo6P1+C}0@^AuoA$ejC2um4u65~KzX1TCAbKMoFO&xl~D*A=Z_vb?ZbYGCx0>ZS@%&7p4`qJX?Vc=Zkw#h5a)b>+ zo59F8YC!Q!#DwQh-W)bKChP{HT|ns)v*ygL98?Q~IdUaOA4?jQNAJw+w18iNW|+>Y zy)3{^w6|hA0~r7!N)x?}6=!M*bRj6|uxU+%=+LZKwBTn>a>$ia6C2xZ;Lxw8ePcnL zqCgeJ1gmixV8gYJI>8sv4oQqARyIg`D1v03yATyo!U8=`&DOQ6*Q{ByZp${TBBXl2 zgDe{G5l6C?sK2C#V8a!*b{oK^%_v z#83w)r>VfJM4J)J%A~IiV)sJx(A;2MhoDM@agsu&rV$JVx?iMe8l@0P%B6CuXvTvR zryBSTSjAH~HVQGP6Ym~5Oap^LBy{9-RbjsJE~X%;h^Zbd4Y?H-qP_^nlWHr(lg(lW zN4UY8I3`XW%F;o#S5_NFYRHxA-0<#uU*M zf(r{|nPri1C<5oA#uM&KkBdMeJX;M>wN@Zk8|o#fqZWrWljg{5kP169^*T|N+<020 zRDL!D{vP0_s;_uhS2gn#9=#%eh^!<=Xh5MsKjQN*L+?f$yK`^(x(y6W(HL`vx-6kZ zu3(!Uk3^&?<{C4B^|gse7veL*auE%PI85R=r?cTxqqfzU0IX8MKZ|wlobP+|8(6$Z zaqPG9c2GAJz3hk0$BLZ!L^-4&T<}2tCX_VIq z9|*;`s0k&JE!ugW3+%cJLS+Ly^P9vPRITPGqJm)zDgrcYQsxH~MM$O{lqQwJY)C|G zsTCC=cfN8jCy%ORa3jE(k1PP-yyU}Rfi1Iw5nvVsCtF#vkg!_9U!&CindyMmeOr-3EpKaP(K)?)K5P06tfoU&QG*g$;{MEn5c9xE!G zI6^MzeclrCiuDseQ>Ls3WIZMLEM!sbR}`R2;Xn$trJ&c>KSF_5PE{fT3joS^bvgb0 zeGNE718R(3rlH!jWWXZGoH%wYBO%VqY&v@MX!#12dJi1DbLaL`CqtSyY1*t+dqj5j zCl4GxcslfSVBUPa2M_ZL3}Czi4NSDi2N6d$Y&fV6RP5EO58%G+mqoX3hsz`o4{X5; z%0fTD5d;hWHM?EAcKi12BeGzR-n|0+b8r+6zxM3izd+H#m=IqJyK?-(`elUXp+wVK=XG z*ki4_4OXsL@%TYx_Z~fnNCHMWGp&aXNYISH+_`%Vek)I4E+#y65G!a9ne~TOKYGkK zA(Gxux@>umJb7M)wUV#A)r~Opu(c?!Bu^mU~jEWU1fAGmCpM3OTr!JkTryBxgm$#^H!Aa2DJ|%Pc#7l0TJmi`DR8__!#&aqYt7oKAwH zfOjNuU*rQ*=PTS=1Ys$e3Q&wRgkKeVlC->NG>RIBpM(gqeBmw7T%u43LuE@GRzN;V zOr|DOUg|~=E0G4|{Af~1@43}1vUea6zrcFNj40CiRqZE(>*`)Mejp#ZGF4jvd})Hy zg;G)0LnIbfCu~*mLREW}^!1;AgaX2+U#SbBQ;BC{1igLhhF=%``1pP};wWo)vgRquept_r>z2&V<>&SJ+rzhPUe5@h{OLzae)^HnOE{RZ z<1e~hW+PL@X(P%{k?K`BN0>)4`Y{C7n>84;$-G8Qku{Y?+_73#&H3hy{(bG48JEwU z89AWuoBjIOE!Hz9Pxa~C&exF{b^X#?ty`fH&X+siwTo9x1df*3C7=Q5Al3m)T+BUb zmCJtU&;h(9`B}tCL_ZfHiZmNnE+08$VEEMwwuHEe{re8@+c*5m6-!#mo1Hs6jJW5a zaZLSW!jgG&E0-!A{qS*20=|@@G*vhv79t{(!f6C>Q1Unk@Ra#Hrgr@XvO~vOfw^q6 zKYJLl^qXlPy+1y{$9Lt@MQ;rqIAP4YIemPWE&OTb-G(6CJY~1G;c2C(}>7O zY&oiOy5_T|kPeU$gaRtDBNV`u_NWFbSyuru(k$S_DkPhcDBJ^xL0u7GVZ<7?xmnHwO%PQTcP+nwfhzV#aSOnNapG$>AlI17anG-8a zex)~6nF76Jp7mF;5Ve)cHl`#)=^(>^(q#)l_@HV9O_owr4W%LSRp>6TEbLIxMDP=? zrjkkilt|6pW${hYfWt%&pqXSM_M>WhRw^%L(I^3WSh{Fnq#;U1k(`7wDodIM52HDJ zWYMAWXK6?x+N;PTm8Mp_B41UV3;Z~N|h=hp|DSgeU>LiCdAQ{Vr4QgwV1j*m%^A3F6ZbG?+h9~ zqEmqdf=>`xJoJ=W`%)$4be%zHW@oLswPwwlfsUx^;llti5x1RAYmH+^&z(6_wnXvk zSFhBoU1xCrzC>M#H>3DfhK33|kOUF;1lLdp=`|^d@uNq)+2M_0xeFE&OB>nrV{2Bl zscdr1nmN0C`SQ7Q`po|Fs~^{GtY5!=v4VM~Oqw)f_BY`-Z{{mlpl`2UI{yHw#SsY} zUY@$Vxg+l1t(qrawxEZIuOv``3qc%;Rl0;f&JO`21!o|&x)2*7Drh9%{$)9iBZ}I$3iK%|pCHykG2|S`o z46pyq^}qks{h~WVVIEne`*VK6rMPIq%ccK0-Rmd+AO(IKCI4V0{?4K?!eFx$qrJXE zhK?Hh{?g!86Q@jBwq%h=RX_?>J2ag`+FHi`pXVFCJb=V^#%782)7|Jd` zJDFl>CsGbbuGaNP4;uVJ?)^*;Y_!m}rj zSuMaGu&)q1K(YNsiJ-+YkAIFH*z^5oAO5gzW9~vlMHQqJsSYvLqZC8+I`zdlQD-b4 zRI^s&Mpl}rH&v=w<>`}Wu$R#jCTv<6+@^K&om)1@p)GC`KT)v){z!{L7P3rDNfI=FPLGXW+|+R9n1o!RW^iVP7Z|+#k9CybuAAy-m3h6jo381?dsM5&aGD#dFcV zyML4C66{+N3xYe*Mv;kZcuH2keMq1(RKKeGY=Gu1`H7G4UWL5m{ijQ*VR)YX;a_=^ z7pW>pr+!h@$xf4%tE8d7q|34{cWLhD*$}BR;gP#pq+tCgD(UR!?w{OcDcL(KPY_mp z{+PczrMqR>_gTe5DR(K^-{oIXoV;}XI|>Mi<@5UbM=9{Te)>mi@;4PORwS~_5X%8p zf2=l%wrr1!!!blOk7}ifM%2!M03K#pn;VETia{_D>QNpdTynO6gn4V-y5-R;SGsrW z{Poz;`}XV+jRAXnC}N#XysIBRdcch4d;@^b8)tBPrp}R}wP(0xLQD%}(_c2wNF{MI zu)Vz^(BB(_@Q3Zf@5{Lt-kVj<84x7?8 zkP9c&6fa(U)0S;=EXffzMMwV(uDs-E3$LS`uC47{^94BivGhd zufP7sDe!Lx*nhl~|3(>_^|HCosaLzV??_8Xc>Ew@!B6uWwC|WBFt-Xm#}F5(1M>zk zUsKyWs0cA46O0TDnQPE)PmYre}k!@ESpr}H;IgW zrgp8m{re7ha^#5Zo-@WydHwJt8%sEJ4u$>02&-gcFKg zx#Pr%6AvCd?AonIqlQhw@7!_NoN0-v_PB&%#Y#{$HsIXKSW!nK!V^wQRIY9*GV5

zxi%B`#Yul^+ym7B^u>)s7>41(WFpYActZh{mgJx5dJ1~Fer z^e-xfN=!(S1@7+LvR+mI>VDV#+}dXoR2FLIEI$;Z7Mao@lAnzcp|(yGe*IfsA$M3trLmR0+IpX~M1{~QHG_6Ukg$|2&RYZ;$V|!a2YZFMaLnVQS9;|KjPGekunMp zEcWIv`Xy@AyBcqwn)Ms^A3TH|HUx4Epuyj=Wy?pNzmS-ifMevQO`8^e{;`OZE?4s! zH44;fuyNZiiU1HX{TAt%auJ02DptN8*luHc2j~e5={@d6MSebVBpYg81qw|0X#tCX zpZ@jz=U;p!fTQf^cz^STjku9a|Kw9aNvGbcScyZ&LJ72j${yz+Y-Y$Dp&Q{CBLVcw zBvX4|$f*>HipMbj=S81fxTNv&sN107kp2UY9X)#Z;Gqvc`Yc2C;;4r!GBPsk)JagG zWAm08gD3O)v!52W=+s44Axd9wC$v9y+ib_tHe7dj5t&`$b5TJI(lsF`PHvpP0BRSl zUS-CNnI?no__;HtJb82F&zCMM9jGiwEpU@P_qpZCiwneYgPL@cs2#F@$yPqNUw7kG z){npY;(ks2RX@9PaHo8^pzM5q`}ak;e?UXv?+-7kO7hUXXDm7VApuYRkcm8zgzlA! zy379a(z}Y=XFnHmWiL{G_ilE(sD}G;=in~!MQ+@vC+=6XGgbe-NTaHwxRl~RS$hV* zOZ$f}|0cITqg^Q;CTMve@m3@iBM_5~>sjbq9XWXsJ<) zh+&;lVt^qCICw#bFwM9WryQOTM+jLMGw|f31P@|Lm_2P+DIhoUHXHXiMQb!)eDJ=X zpWla*zQk-xtQ7c&A+LC5a0|pC8KG>z?97a;3#uA%#1S_Zeh~*1J>t6-qCyI#E_VlU>gGrgRO-4b5BSFT?AM#8i^Rh!3)K0Lq$m z{7(db9^$Sd!2}7Dq%5OdHrh*Mb4rAnK~Ex{qfX&mUDEsedY~u5VX#e`Ht&u9pl#>Q z^41GMQAxR3xTB=YYzz|phJux>tur^6y2sm#^nA@4%}TTsB1?a}Qwde&XZcEZA$#(p zdXV+d{e^l9YjE4^xO0}W)Z*wxdU;b_WKQGxD%ObdRu)I$MK@g1FK%RZXXSqAewV-5 zb(7nyhi*ClrMA0UDz9g=F1c5YQmuHAncTP&bCZqiSv7SpWq7V^XGHFn%l(&|EE2t_ zyqnrWF8BK~E~zXFu~s}&o^f-HX>2z;^PaR$c)BNmti3RkJ!UxC?ommi6}zk5i!itXfjD<;NV@Yhi^`k zj~}?sJB(wt^hAILz_q&f-~V9C>b2o_?-Egn6JOb;Bakd6%rX&}18N3>$*=<}A@(s( zCSP!zvXra_e*k>53esr(0s}dwm*nxuDPunW)YqFhqGHFwB*g0)^IbRxo|gDzVrokz z!2qfRn5!gm!h`HE(k`(I)*%V2wY5a~pjfocoDYEaIRRLRhy*6*$(^rt+YXXO)GDGe zAZmwAXTr}e(?wiUR6?|qm=)sxrxNI0Xt&XCNWtW2gn%>wQc$i1(y(OFXuN!VJ-mIa zVnwXoxPE=UJbBx6>Hywnqc(2}@R1w}x$#1cATkvk;o`5{WTTAw^?&u}1rGi$f#jSG ze~M1>>kF{^T~hgh|FiS?{cqW@_1n`wXY$gUa;p;15icHd^YVRGR_?aHoLg3+Yz%cH z<4di0xehPqsYs@ZaHBSL=0naTlKd4N+=*06B_Am(>(NU+pnmy#5Lt?Pef<|G@Q3dC zFTC;pcoK#s_{=C2hcO@wtmNa;nsIUiwiWR4!02>%I9y)X+ez#)DebG{-dyg2OsyA7YXo*E3(@zVJL8_OJ zIK)v=y-l$I7zmmRfC6ckPQwSCoJ2rWM!&qw7sM9+O5`NcvfQ9;I>XQrBlDCfg2O2M z%ZP&=)kFC~8gw@<+lhT!H?Le`cBLasQ;}7>W|#KBp8eJ8)?U17CE6Y`Vetd$2y$w|>IL13SV~KMmA=}J;h?CzB`1=SwUTU= zV2i>>)FcvO5M+nyQltlq7`gzK22(^4EV@7kC!#NzH9hj;S=wl2y+H+G7h+bg!6X|+ z5)~6pr-;F+F!=sM7RCtixeDUF*LXrqZf$=1?-`D5U)A23A0qpTvkdi=tZ0G+ z$r8{f4X>kkVN3S(iDNj-kQ1p9?x`oQN?1`*xPyM|!ICBfZ*_7K`$po)u^_OJCki4V z#pTOa@FrF21g)|gq%nB0JLb%q`OU0ZXzj73fmo1|3S;!b=%V&Q8lfW~>6-V$5Bv5W zBCafnsVCYVVh%!QX1#!&)KH&pB?xwc%sQB|kH~N$4UF+9HJ&05`JhCQ2J5BSYNOBhDA&KZZ{!%X0fNk+R|4|cQfqIQ%;OUf4da; z@m|)e*Htpa5451JvSFXVzXl2C%FNmAg0LeXsNN)!!gEb!`=g@L=*BL zbQNLKhEpZ)A&~;u$oj}WW;+JgazZY~h6s1q;o+W$tq8V?fD5kGbPr60s#3b}00~N^ z^@2qp2BM31a|n^fh;4+98E6v?JC#scb=rl%sx}w_A)EEyt+4m|^|YraVwmiZoV_l1XNbIhX=2~*wzLEZ4M%HPm)I;GsLd&h1p(qDRH0+}v6V!*f~z!mjj&JfRx>-IfH9IRd`oZGvSSygALx*an^H6TH2mn+pt8UujvaS!>D}?L0)Op?=Q0CLz|`kFI^%& z2tSd}i=W^|LT<&DB&}Oz^!pcI%H#Fz|9lGQzxwK{zq*yoK@`DJ@s{TqPA^|C=BgUU zSrUbmu5MrY{(oGf-{s+6ho+{2JKDe;W?+tTL1dtWV+rQ}@UQ%N1GcPRyK&v>s=Y#wAd3u1&k*1qX&`uH?Mzp z;&`xY{n`y@PMr>_9t4yjjsbuVBkylsv&z$GSh{%8<%<_9lqnN_Gi=eqAD%pVRJ3?0 zKW`5w>vfxT^|Ii#!7J|Hzf-Avd4t|~Dm3)!<#UfC?=4&M%k!sCDp#qvch}C9%YLmfJH@_w3$v`b6m26UU1cE8^|r z2QJ;Y9=2t}`f4@n2~l*K$cX#vgO}a8bSXolNl#BNU8=0pX4$fS)wEt}v3`Sax`VmFbd;$q^ub?=E`I1)HS z20g#|^0Ow*nl@?Ko>&<_&6}Oe*LPL$ibDqu6e^T2JvH&y#f#6JK3$+-!5senp#1uE z>k(k{=gG5S%}SRwBjjkvx;3kbpj5PI5vmPuQ1KJ#CRv`qSsm8Q1@pcO@b_K4V&$O& z`wQmF?XYGnUA*vA=rJLJpFbUHvsl)w2&OIf!f%%^Q_9QJ8z%Ss;e%fm{=9R?7A$Fw z9o*NbNfTUzUYLAHKo9D#(?mtkUn{+h`d=0 zi|5X-U%lc~=<%F>zIk&8Ub%FBC#_)_Su`mhuW-} z-_7|pFd%pCz}#|3ge(Ju{qu))oO{5-=DAS&1>TVeFB7&aAlCkF78H4-vxPSYaw^sY@gs~e|EHk+**H4EI zZq+O!In~G0YkZ&H_rtDw=!~;x%-p(eZMouw7k~flkKfD`d`4r<*s$AHF8X=y#P{n4 z1+iFm+ge}=_9!x9$UwS7U%Rb%>Cs3!M(dJ|M|xNZ_`&J-q^NbjR{-o z*r)B9G+48GS&1SA&K}#}vtg}gk0Z@G`?eKJ$GtiDz>cjP39)O#hVARtfhGu*D}P;b z_xe>NYajy8M;b0#Qe<~zRt!a~>F>Qc^xMf_2l)8JJd7IGr}w_?n~=T{ZuzhPFmV8z z%{uDsx2|5jnm2du4@SLneE%+Fb3|h_5LUuS(`NJmxQ1G77cN|=9aIxBmY^$Zzn$~e zfWCxbxpLv`8@;=}JG8$eGxhwbsE*M>?n{s&(tYXQl7=pcJhR0&;)vq z)eZTh{(gG@_ODaF`tpPEIlatlm;Lh2fWGgKd?$~;->-`oO_}sLEcV&MhvSD0qeYyj z^Je=tu}`APlqk7s<;ry{R_JY(PbZAErKWg$`3F_4WqsF*}B}|K`EzjKly!_XV*=|H~-~SJvudkc;?AWXn%={)n-kZ;%heA z(o@Z@j2{>N{K@ngsK}9FckkTw{iLrKZ`fR=N)?HqCc&oB)QW36)AHNdvr1I1=ASdi z_mjR@x@u*OdUf&@E;9YADSF53%jeEtKYD0q$ce&*i?(Z3ziW#&8R@C$VKn$jF8dXG z!>GqmCwK2YdEA|%*hiWeS7sFuD2~M@#uw%X02x8xDj>g)cJMk&tH7@IqI1r z1qy7~yqRUUm&Ubb-TG1E#;2vF9$dS2!O9gJD??m-qE446QDV@5LA!#Nj~O$@)5rVr zxzm>p?F-(uyK2>{Xre#;^wUK@&EK_a_kaQY)~{Oq&Zu`ieRHr?xqr_NZ!dFtQqrwU z=RywcZPl)==p0aNu+fPXAW9qB?9e1cMwKd4TB)gl8T-f)BL=Al-$LP8q0YRlmhoGS!!>rF&4 zMOVU+^Q_riV@8Y^^7fc^?ZO7MZ#%G0zflw4zkB~)k0wo`?%jz^NW69I=+Err9jjKH3k~hmr|%@zsuSNg zz(A4`#*Q72^v+>1#$35@2d$E~-mcwyHF}RsIio=47mzHM8*ttCF9A8XFMe_3xjYf% zR+sxXzGfxL$}IcmS5EfIZul$x@2dyg%3|AM!@$qgtW{fV_3$sOT(j2dpMF9O9Cq#E zg_Fm&t=ph>Fm18VYYiVeKJ@tEnO{#yd>Co=Gh36B9Vj}ORZjcIqeg!)bw*I_+BW!r zc;DDLfQop>*j1yFDP5@wXW*79TM2`YDzzKhjK*SRDri#DkRNY`T|;r&v1vmULW%7= z{k3aX5m&3%tfMny$)YV?uFCVc6qA=P=Mt7ESMK1!!!VjVx39Ns*}`u00OAdLUE}60 zA|t{Pf$_Cz(zK;jYewYn+N0+WGp2@zUAh+)Wz3(iM*aF$6J9n*e!vcOrY!evnm#-{ zT;uIqzDiI!3v#2TPNPQq))MuQl`jU19AYbe?~csOl}i^d`DKwKJq>INs!lHI=QHELU3#*!7P zLay2k8(X!eA|=bB>3{O%;e!W{kk7j|ZX)}wXmE78D_5@EzH_Hui)Jn(CX%K?B}#eZ z%TG4M?`F>wrXW#RVHct;5#ndBRoF+BSo}QGJZk;`Q z&faaCj9wm^)MP^JveD7LQ-|$q7SEgV>HM`@4cX{kz7G8KZV4nWp+~1N+UJ_d~;Gt@4*F#dahn9D*{kqo&7l*~XFNY~qQ;go<2E`91byhdS-(9Yywg(ykf8cXS2;<#Qd5{ z`}yU#dg&tB5e3_3ON)H)kTcg2eetyP&Ebz2%qxC*^VV(KZtspQH*en_GHj?oq-<2G zr8mH>#6RR~mxsgG*H@F4mY5jtACL40fG0n&KA88dpk=AN8d2;2UIgg({4JcGZ4sM}N7<$W*6bQ~_X@VNO2+N=# zxq%{Vz2Ye<$7`uQS2!W8MV+`#2C^fFrH3?{lynq;Mz|Pnj*<{H7th3OTn$}t*FcPc z(xl%T8o~mF%Civg$QgLx*s-hU&%qPeOHAaLx|nG4mEA=weX#9B-gob7is?GVu|;a& zD0+$e#eS~dOxvUm>=0^QOBOA>f9K}Dg9i#1D$uM^BlaiRCXJ1Ke(}P2uY7rr91E${ zuo*1^QzIfC6B* zU-|9(>$_Kq06E3t+(It|w^S-2o3DkA*C+0AQ$+qQ3q9d(;d-FEER|L{Six2MPH z)0ZASjZcb8LbXz+bjkSmxK+W+1$|_IATH^@o-!k-a@C>z2gE#yBJ`B9lNF>wD=W-F z9jOC`vl zq%$^a-a7Ke-Q8<~?eQ^(_w2fP;$+*loyAHUIMi4{6O11$dXMe{2d!BC%h~<=x^(U4 z&|_>uRmq3{^ge86YK;#F6_T1(t7aV<@xz=MNzWeKICo~p`i(t$_O@$H0Rj0bI^y~j zZAMD;(S%ntHg8x@kd~OFn7xPhqYG=*vdz8Ir$YAcPLFxK@VglrMzQ6!QHwRD^ap*?_QYBwe8-kP{}g=x^}Nww!+fjl{SaISm{zN`}QB$ zwMUcMbqkhYy)V!tax^*?y?O3W^L=yX95`@La#9j=5cv+BvdEi21mFdBGqn)6MWu?M z%=V$qDmI3uuV&9#zHE7eDph;*?sMz@J!YxiY{d2kzZwfTE6y%BXEI5V3r&dXI2<-< z-G0PpUrw9&e*KE&KN>muozEsUY0<{w#DU&`?gm3eCN*Y^^%^!IfNGO=?FtqwEUq@F z&YAP-%3ZK<-Hx4y_UQHHPtibne0>L~U#- zQHfY^TtWj5L}~RFNTx3SM+u_5D1I(1M5NkasL3OP%~O}=`#IkdgOOZlkJOtZ ziNQNOdMJ^11;4OAHSF7Hw|A2cBA42vj)v7k?(rv<9 zBO6x>%Cy+L3lwHC6B`@1dhXmnIYwRb7UYLnVy!5hg1eG@;W29gm(D|iu4qkoy$%@i z#^tb^wJTI!vT}8aszLlpq{B-G4t+XlTItF`9fu7cH({d9l6m&*X=8!>{<#7&Gc%{W zJNkG?2u0AD5`+dfF1wTCX`PxuB}N!g0BOeZ6WFkTvmN-DdD8Riil`o)i{u%T*xIBn~MQs)P+ywun9o_5Lqv z;%>!nd6E^FV$o@yJxlgCY0DM$^^jm7BJ8l6hhbgAGG&8z?J8NmVwxq>%g>j20=(ed z!7NHmNnr)<<>Lo!#9kS@hKx*nKv2w5RVUhHm(DB)0E$>B-W<4!FwGPOiBKqLBepBV z{bP#p9`7J30%VwYi;e~{+G=6y#m2_wDNvA~q!$2R&hSM`j;RI`D&Qru8S=UWj1rNJ zd@(b}MK3PURH-#jPM+APE3f!?hS&_h~(7t9DaTv4s{_LS4d1<4#AjuhZx|EbONZ{e&0qD^UhYs#twtU6Xm1~55t7=F94w?aaifO;d zur3VS4RX|j`pRmF{sTjuiOC75q>$R_7wo%vNqU_?D|R_e&1p+*-=W=@(PKMx>Bb;Q zNJ{a|k%Rh5GPP&`b}=iYN<5*4j4G)lzPeJ7q+so$z?z=N0AG4mdV}!dKl}!YgEH#~7qy>LIj3x5Qz=kyg#s>(SLp<|S&dEGz%|i=to0 zD*5Zl{NJerk*GAT%(OHnR7y%pa$*8X=fs4#jEoE`BAH%JrD8_hyBG27`EyZIGU0LZ z^Dy6!e1zDm(c@4pfRl;Yny9ly)Fq#r*KfG&R)7!~rvTZglw$+2=*>(@PG&SS+YvS6 zO6$@3=PS~IP%C5 z$sFDb4(N0kKOpMG#75t~bsO}|6`0pRloB8qH%-(oPCM+x%g0{@`QspPOS;8jb?NsU zIvgGmiMlyGJuN9AHvZ{jduAFQg-{FVEL^N)uQ%VBHRn4--BiA#bD%t@3##6{dDE7W z4t!HOssT${D)5>Z6PJ>h6dRv_$E3I*0kV2sW=7hD^Jn|NJIdSJn=M{Y2~x_ofWwLP zAr!V|B&#h&Oq<0^6{WbvP9PRXW_r2=ebf?sCVz5LT4H>X&1&WJ%b0{DpB(4qFvSB?*Vx$0o$a#mC3f z`)P>o5>WsI6E+_c{q)MEOBO3(*EOrxt<9M$uzlw)+|%!w>B$jyZsA1=#*6=?gB=rb zM`9GD#K%%&G6(%p-NZ#lTQe-Q?!@sBNMY7E5}!q<#73va$Kkf5vEo3aNl#9-XQs<` ztpaNm?TlODrLV77lL7(_uhfMwgSXrA<-xRibibbMf-1JHRH|d;GB{`$fZfdWS)YH_ zs$#jfhxG4Ovl?>-@qFjTb@hX)4D8ararv^FRxTIm3z1vIFo(%$UGl?sttyor*1KDS zie)3NU1aqRC~#aXlCoK=o%G2^vuAu|LI>^0T)ucgr)t%r!tdR>adY;huaZ;JBBG+& zRjb&gMzuDT%6D(wa{so?;%$d#Ii?>>QJe+=@yzt0y?Y(oyE8HRVb=~VyELrbt8wj) zE$ZL8d|EaLZI%v|%XO+=o~VHxgDMSe-R$w*yR&A^x^d%1y`akd+qWFvw!_*rYeel1 zGz)?_be6OU?+oi6RAFfIdi^@KcBUi^R78^OB|IO=RXzEdGzxVz}lV;4|yiSB@ax-c5%YR5 zne4W(v*+5EFLz?!zVw8I?JK@RjqjQ%H?KFn~Dko;FWlg zL>rf-3Y3B4zBlTfwpA(&?bW?Om5STeF2_9FXie`}qiUO~<=R)P*g2@u(w~21kmwwi z^w{Tvx_9ncr~1hLy(YXn66v_lz+t}xuaQW(mWeWk=zMk~$*I(PM znS$CPj=64KyYlPonag+WI(zwYgU0n|eEq4&KsLx&E8gwzhII%(qgjFd!l8lWhOQ3QH?e0FTzgb^Ml z=NNPk8Z_w4z5Azb->z1%+|q?V0mI5%&c12KmUFjmoVj!B+}*put5)U;%-O$BueXK| zxe|8y#NE3+yY-|}RE%mP`EFRb;^x&0Ay+PhUb&c_5VvaSFR&Wc5Tflxwc;66IN%6D4 znF*^w6M`7);Y}Pwk!n#00ECFTTSAXR$+f$8?q9R=R@gNFhv;3++P6D@^VZor;m2>@ z^fZai>1^onsbj}1SiSP>*|SH(!pf8>C#wskdeUU1jvqbpy|2H%bmPXvNuN!dI6f&M zUMPdutC$U{Cyus2P<{R=>g_@O`;QsZt5>f)xpSSm8(ynMO_UgTu}_;l^XP?hpM3gJ z@VvQKFJGZzl*r&o>4bZ=GV$PBU z^I0aN&t}sSdu_H2o3&`ANl$UwE$D?Ah-{1@QjyclnKR%267@M4$^5deP^=30#k4@{= zT)%QjDumEyG#kDB%apHBqgI``m}umFl-Z(dKz#vpAq*1M(B*h%{Mf)^#q6G@vK1@o za^#A<8$p;obi)RWAU!?w-adNhuP2&|^XG;0>ve2byHNvd>b?DPat4rSij*SgROkUs zN=(-Q{jl!t)w?fz?#$U!m}fvE(NjwrkQXzV#G%6mwrtyc)W;ulgnULu`uv}MNQ_Hh zds$*VnY?r6E7Z7UTQ+->;^R?oQ%%8r=z^iQY)N7xsQ>WGug!ir9Y$k=X3aDyDbdl< zM4l1Fv)N3{TZ7rpVDJI-FJHcV{p87U<0cq9J#{9Ne_(*@3`0Fk3|*PqwruP>aERGI zx5+K2L@)C876oB zey4gs-F@HZd-vJJRi-aPhuinotvXfpKc`NrH7o~Nuh1}tL|l>LO`0^-WM=2s?D$s> z`e4wP!@f{ze{?K#f+$YB8uNSzTfQdeKj22Gii$z6%|Uk5>8>Rt?%%w%W2eqc-fLH{ zYtl39xwd}&`VAjB;>NAp=(znX=GWfu*RSu}6UUE#_np2U4jP1WH**<`WBy#fezjb= z3dD4v03~b>TuZ#0jHrmRm|sC&=$o8)=WjiF^myy-m)f_(Vx#o@>b^G>h1J(kK?(F$XtJbc6@BI(T*R5Np zc5R3zpiLpN-~pM%0EoU=i&;Xevk|bqXRjePws2e_A+Jt1ZqnpMOP3|5rQu&rw;nli zL=zN5xKN*W`p%v+FCFuPTC#902HBhEPM0oQj-&x@y(PY6nWURH#ZoPzGE|fr(+fMd z?-Sb1sc=Zhr_)&Nxj)RCTdGdOA|)!K=)99~S)H+D=8m*;4gC@-B2_r}P`CF=1slY5pibGhY971^5&ldy%N=)Lap><~77g(H1^Oqtp8`?;L z&HR-_e*n;pt5-t`7YgvVf|so7Ng^2JOj|a4%I=@m5+#I&0ui-!pAUi~3oo8Jp+&i( zCw6YvyB&$QE*{yjIWsM@eAz04`*d48b(9$;y9k-2T(xPkbFBV>fSp0__7AXX((eD2KA%kV(i4Ui@P!Pgx`4qPD|v@#~*$$@vG1M3>tr=V~lg? z=JmiA`U|naiXH>=fL3dEIc+H^eR}<^Rgaz#g$vn?+LBcoJm03%ovWAL=-B4f_N~nB z-1K{?+`MxB{P^gIAfv0-^Nmg)*kg4&$dHbSGc_eU&=O=qLE$0gw%HmIb05!3B{HLU z5noBSF6Y**16ww=ZQns_u?UNji&-SNNW{usJ523mixz%4u>bLmYuT1F?S4AGf2^5U zAwX2r=m_n=tTI9S<2RoLx!xe z3@-cXCG#SS7pqaHE^;$|`@9M(fpF6EQlSxFeD=+-Aqi(rhlB+?oH;O~z#xA^c=+v0 zSI9L+RHuW+qvGaT-;pmVfU{HnI*={VNb<*u)#b1)m@)11(c=&*G0%ygwO0St*WZ?? zT)X|tFV`zkZ2R`@HJ)zLy<^9}z1ip0o;}8n8s5HH^VLUAgha-IXL#jW&Bom9ESA4| zgW-OTjYV#7a3~wBg9|{J)5f-8nrB~nG3jpND_y%RTfNplG}3|p1L;t4C1a&89(A6@ zOO`|wDvku#vvrF?MM^x?qK&9vIb6u*HwzVsXEMVGBr?3rWN&er9`g0@Ap-`Mtyb%$u05zIC@;VQf0B%KT1#~F)~(x7 z;?JKy|D#^7RXcPfBr*o)8(;)46EPbK#M45LJu~C)Z}cuwqU49{;s!41H4UF`RKG!8 zcBnja`edh;&!0PUwoq&V?t5>*N8P%0qwveuZmnIrwo>&c1cEhMG9YH zCl*FLK)^T!bW|yzKy0@zork{Dw^-22N zI0p^;Qq()*ji+oC6h*yBsH@Xoxp1*d_a3i*IPkgWo5RLP7yJJE?^*DV@BiN0Z@f-= z!b%OEmcoMqCrp~8H~E#VP_Ad27HlaO6cZ2W{MhK(<<8B?X30R6bF#7xQPHSkWCuHj z3;BE|t%ORlY(Wu+6;v`AT{`*X-J$v>62QuTe?qczUFPuJw zwThHU=gwU~GaVfjmj?$PCuyvar`TP6{KRp3js30HUv=6tfeSEAw4J1~AoN2;#}T1Z z?TOmKh2sw#IKUYsFC|fF3KlGQShd=fiH03$5Si`TCukH%P$kl=M{MaSb&f9Q|c;xg%m?8#HLJYx@=yUC)m%{A5oF$`AL|pM`Nm3vGUo|=YZ7$g$fZoiiapteP|fL zsG7TX6G?qenJj1`OP08F?kshH&0al!p-AacxB*gOR7PU5)_QDCPfWoga~3YywR-KS z5hF$SBur89DIQ6~#oDXanXHwoRS$}nAp!jvlUGP`P>|6j3n1JdX_V?y=~v&8-zbpp zfSWu^zyU%Nb9Va78C`mIH(1QFZKK;&u2SW&Qe{SsnUHorb?LGt_IqiSYt*h#xeA$~ zHg4JIaM`zP-OeN~Sh&#T%a^xq`(92F+><9xW9jPB zy&H{tzIE#jYuD}FyZ25)!mLG$Yd3F+?0Rtjp|ht>0|6wWnlon};q2G1UYj|0Uh5ZM zf=>X};0Fdq0V?#o8bgP!-P`tfrEm9cXHTC7liB})7)7K-gh$l$7cN~qa`eQfL%-mL z3Y9CRBqa|2VpvX=4S5~#Ay6OS2EM?5BX#;n%V0&5|Hi*iQKYzGGH+S_9%eQac$~6X@P9Pci;xAwB+@Zs>Pd6o#;FarFZrp(E_8V8P?AW>u zKk?&7kD|`*(z6H1+oajEn)Iycvu3BHrp}%{TN@Qyv{(r`0=8;0nb=Wb_RJY~Zr!+Z z`}Uk!Gum|Qg#2$)!A(#Ge?j7AF))-YU-6TP6BkXGv}DN=C@zKpDk{6A$=JA^c<0#h z6QjNxn||X4sn6-YjI2z}z2qWAiohHdC#5Ha<@k>LNr8N)D#0}yNez9)`SX{rA3xQ8 z+Dt(KBwXYdmuuwM$$xw8mCcnaF(=+1F}irA8Z5M58}#v)gFpJ@WTiEBtY)>E zfb6mbvzi{*U!`{aR|kCf=1c9lzEr)sAK@&h7tN#7?lfT*!=u?~$o1%QwT9SI6-LgO z{a&X|V~Z58QMo*V6{avjNsYc|+jbhGsdD3{L#IzKS)nr92craF^=>tw)eP4~L>}F< zw{HD^&08Hjdg`5y z9RbmpI#tGe{RRHsqrd$!CN6Hw=uur>c@^7N%bL|7M9ZG9v~1VGCZ2T?xF_x<0Efhv zYf+>Q`0SH2*Dv+#+WE(|>zyW3uF=HKzli_Ne4$|j$ntFWS6}SZiMX0!y?YJ#c0}9uFMm66@&~5eOrs-lJ#dGW5!5i^tBI)4D~|-o1NIo;+ESap7Q&W?XL^{l({_zxXDuYK>m+ z_ix*=lf`77Flx-8&pwBBO8h3vCy(=&om(7t@2*?4Xx-9VLg8#s2{*KfTuddO$y0ti;TcfNo_Ikl&15Tcqsnc5>{?N-60x@%zJ5} zp^-Y1*=YiKy*co-j*M()u1y~k&Wf5EXWq{=`1`Tcpc<$o(o&P~J2sdCM7+mv$tzj| zG(qkx>L7WdfJbtqia07MR^SY*_hE9V=S}R|g@PR5)R{7jvg6mNp`V0=hxL4`Kgt2q=?H3x>5=in2$SZ~-+EHz(wLS_o__1Cx0<(oi4;`) zf)smW^K#Eg6Cg zeHoUbDF#r;4;K^_%dD3*KR}&URX6zBtor&15psYI2=Ui_vP(nhn;V zTvQtZTj)+j@4(_$#5r~xqg;V%oV&$9$SY7Io6ZV^YI98fV58Dkle17(1K^ZMXfPP9 z7W3n?FT5~+-t1+|mc96LS2kXx464e+N69L3Abw^R|Dd=a9tE5bDXujc(IXRi$!@(LYjz}B%%N_bS;(?3k zPaLA)7B-`^C}X8KHGp+?n=7n9p=X*j11`ns4YRC59h{fN?j|~7sH!D42R_IikiikL zV5@_0O@d<}vphlPG7)$b2$SP}$Ol_62L$H;PH1Q_19*&K1>$ny0f02+KnE8MCFEU| zqGNYSQN&X{o!&w(;OYsd5hq1xY4Ed2usA%~C~&ONCf&c^d+1QBpFg5Hd$vj55m=_t zjHQ7h@S+xpR(KgG>d~0}!s6m#6gKji7%Vat(6G}~tu8D&4hk@qgohZcra(XGL2)S5 zT0_DaK9c-}2l8l~nP?-P*9L=friT&A=n}SJwr~d276?WTklr`YM(EGN#UakCY>&4j zG&1J-*6j$rl(s`wcpxDKj~caQaQKctBL(suaD!9=)(FA?5VKB?G=^Zq++r$ycYRN5MEuN*CZq)RH;(s?Adb;Dj~9D-T?;j zNqjAc^kK>i)FP^pR2oSP=L89Lk;;p-k0Fyhg(5ryOrSRK0fO@sam*kcxL}Lu1?onj28j}RMb!hI=Gg646I&94%O;#RF^$rf&^*3asjkIJVUne`4pEFOk{*+ug9u30;+?Jf z8nKjHy-nZK(|6^kuBP`YxiQ?wm`V3!Fjqu zb81PQsfq8%hZGR;EngjM74BVz5)_GKtb}7h1tu8tj{OcLfCFKfnaq49(KRV1AQDi3 zLj|){+`O2jK&C)Fvl?wh^6l%(CytRYHj!0P4!DrnKw~vU#Uop#1XL4Trnc#cS>Qx5LgA$$#d z0L}!11T$d?5C&38^>q69|02OudRrxaxp!~{?=Yb2;*{5zK(klksNk3;&K^yPaZ+wQRG7OFi5OCiiCh7uA9lo?X-If1ZIj}D*R5Fg?Ps5=P$MavU#X`s z1^@>gk;Y|m+ReZ$ZZgyd=RB8_s2PG@AdAZCC)#$P3)EE_WC(@R306SAM+ zE~5<5zC&i0M#qaq`MBBEIy&zvwJA{cdI6gJBp>sH^rb|tP@@x42@pFX(v_O(m1 zW=u~^xKX7-*>$T|%$qd>>8e11ILwzfE?-=?>W9|t+I&BG{ML1={LR(^DggxuzGdy| z1>a9U{LAk6LUAEMflix!%D6FMLH>*8&dtrqEn2J?bwYLmCmr^jRg31YSUC6K?j8On zL*YV&5)*GspD_CDu_GRXnUz7|5+#+isUhhgCbNS(cg&wYb;qU+c)yh_Re};u9y_*u z)26Bw%7cdKDfebhnNYQSxmC-SZ(P4NH7OzKc0$E+6^`!Te{BCjTUy$j$rBIl`6WKC zaAZhGTGH+Br%Yn!0h0yIf^+utDFJ?#%coAPoImr@=`&8dComv5JUkKsSA=1wbH&2h zq*2_q=BHB!4pgjCBPZ?tyzi$hojd#D*;6ES2@MI_xPJY~6GtkPDPz!g=X(XYCNYO~X+OV+jsOVT! zD^|U0-t5`Ww0$WmHV(TSV@be`eumY0aaLeVfK(yB77ekShhYdfionakt3aLH>AlJe zn}U$4NnAn~a4Fx^1$o#GI`f@-;gJt;SgEw+EcYPwi*`*e`rTjN2v;Z752Al6uA*5Ti_Uy(1bn!i2&_AS^UZ@bvyd>>NR}WXMz6K8&@vB(f*}% zKdmBk-mB9<%Pf)?@}{7VsA6VKo8G6>%eY~ntseHy+g}bDijaH$^r}Yu9YqS>Fu(T!eaO&gY+e^yQEtC?p8e{BB@>9PmE=aNv|Nqbrv#NhX%$ zq`UZ}@7S>;BjY~Or?e7AhOWl#Ieh%&kE_=W7&M?_gZh;!RYKTD;>NN{A{asK2VD(O z)HD@yU!E4iiz|RBBJ;1G6@05pAKWVV=!Hn{E9b#hG(`$wA3_iuLPYQ19RKju`w%$^ zzImAhpM_PZdxSqcIDY$Cu6$4dewWX`xy5^>oKbbAX}_=ggA#r3`H}*nkjz&H(~$r+ zqR~-Ubbj@(=01wTk6W9{8nXh4Q4D=+LPv3I8y8*?~5m=J6ccx97yBjkDIREmyugxtrhb_ujbg z#%|iUp?B}Uub4BxXYaQ|qhk>;f_A%P-#^GPKhxJ;D7-)vFG9>+QGSeur&! zP$HuWx3T#o`d0wXt?k&U!?qn;i3RV|=RKg8nmE($E#9&@Fe0`?&)1)+T>iwd13{q@ z{)a9C>73tfQ#t0COs-9^h|||6q>(s70$>rMi0}t za(>!$fMEJJ-hH=Lty=TuFMOqY56_)DmroxDcJQ>bW!bJ>x{`K3NnGu5Dm?1|Fzew`m75EiP{4DU}Ywn5xxIojCsB>eU)g=4WFjVCrN0OoH^W3W&(d zpAt*G=%%#Ef|wlMSNR05lUMP_ej_`i=>_L)vMjOeub2qY7!ya$&7HZ zg_T$LtWe78oXm3!&t*Q6qlQvfJdP9vTYVO;& z?{;DWs4=i_-^xWwR4Z9Vla+n&@F4&Rsj^nRdKk?B76A?M=WyM)cu^BzDJzKvO-`5l z$tN4$C8HkuxQOEfaXKdKX#gB}fxj;)2|ie-;S8jWPg3yTQHco`BF5mU0*!GrtdVyugxBjPR&h2l2CJ_Cjv93jQG z0ktv<6-lFkJPcO@KA&vZkbTw(yS{w!RQ-na@UFqT1&-LbQKR$c&l9O%vU=5nd-v?x zxxG#6=NmM7YWL2adw1_@_G~je>iWF(?y3a~8$I1*-B0Vh;g38HaT+faa8n2(VCSAN zVZz=E7p5;-H29r&cJJI}&vxM6hp#1-#L^)Bmxq9Ywhs@b@8%m|%_VY`0$DG7i+9lP z57BgZ(ck;iLns6&A70E!?`IzU;P93#-}1zK^^u}fX;KWgH|=Bk>y1$_9yos%}kKXe7=Yi1IA*0VDLERzI^QgAkdFpx{tA0s0ng zSTL~6=7*pUb)YEN3Af-S78Dc$fG6EaC{nfp+Z`k)-Ue<1!$K^6es>dZVK&4EfmX13 zkfatsHz{G|USvoey7Kh&OuizU6DXf}?p&oRA6v;0=w`#ld!fN$Y)t03m*~ODNwwfB zc&CBp@C^J>B(t>_Vl;svq2iaJ(*y_lA2@nScmyd1HSSwiFGUrM$2+I-Gfg&b-NIf9 zecpQO?%liSRU9^F=Z-Jq#rW(CE$Y-eK6lov&;H&!)Zf2Wqo=h-@&w?}fmX(2lFdBD zcbmuQEq=A?)G1uK^7gHpC{TRi=_Y*maJW_~CUBUJgEaI5iw!36uSyhPk?$a(D8}g~ znc+y&e3aAjo4_FN+$QH0)bd&V^eY^D@t5b)6I?|QiomWaO};CwE>0+3hT)a+A=QI| z#XE(|^B%>YQ?TQ_>R-wg{d; zhF2sPxtvHswqQr5)#}EN9haVVf5W=Aj9QvZ7 zq6$?gH*x&9OBc@Fx|uL}(u5L~D#sNrRP%`^9QW>R+q{X5#|X9*p9}FW(?&){vhgFq zoet1gtG{vb#E&ah-M^nY_Pfz+{qsbvT2V34#@N`UOBb_`@XYV0(<ScEXvnixw{+vUKzMbtiW2YVq82T->Np!^7LzPv&Ip z+O=!du5obd&&PJ{AjcC;SoPCd_K57&`)`KOkSpiT+q1KlEn7-@LeV6XNY4Q?)h=JU zZ2!SSSvK3|EnBajIMt?gOZM-?fd@Sj4@C*&g)9J#KhbcAVmyzj@S7J{y_hRsIY2-q z;d;OQ!>bn=ISDcXonG|JgY18}R6XV4_1+&Jew7g+jgZsIbouMezeVH+XQ{H61E}jC z+^DXkn7rd*;qqN^6*(jKd7qYdneQ!MQ6S#|H|}`s_i7XU1KYkhTx>~hH09{6*;@Z> zt+i#3*Y2jLwy0KagFsE>_$8~L<#WQH^Flig^|Ks$@% zuF>8}y56`_$$@>|ZvD5vm#S3Vq4$5~?SZE*UT;&k>htxg?f&^^t;V(l8Q_WLl$9OBSnFeg%vm~RxY04C0M+-CWD z%A91ot5cH(ZR%FPlb$XHl?{9sYRjn2m69vNL7_By)`Pr83P8>Q= zuUzrO+t-NrDPFGZv@xTesaR^sgt3FZ9h+;8u<1kJ8#s8y;-!ttl{s_eswOZ(t6_(G zzv^|L35tn(rd)|_tCtdu1J6*~IG7z;f49zGR0KLr_2!*k8T9$6{{5aVU9|rjZ@m8L z(7Mf@&C=-#R;;XvDOCH}7aZo`z(VB$%GWAVznMOwFka4crcY{Bt!jhn)yma=qGh)( zhmW5c*|*=#>o@Q?G_gGq@a(c*yKw5AwoMxqiyA&|(%ZwnDp$Rh3#V*gokvPMI+83R zIq3+L%Q;`^BlIW+ys$A|u*wh>HPSFNgH!Xn=uQyM*}MYKhg+k|R#s4%?W8RbKT1(* zbDxo7JQ#RMZa~N7#tcllNP%0#{*?e zr4F235S;mmU+6*p@a`9k=WYr1VSU32#(OY0NDt1vJVI7Gyi$%lN>&E4)ewbnQn(d= zFk|5~5+X#1!UMn;^%up9@(B-mDI^c*;c^6yK`-HxVz?$K2q#RX$>%3(=5?jXV_@RZ zNzv3Bp#hbgs)mwwxK3Eqg9qkS-uL#$P=Jol8(NRxN024?f@iGJ-vPi`RTtLiAHxuHs(;=CtCCP^@ox6mlkeX35A-wn z1vqsUfKnVYv|0(0Ht2064iyaoE*>I<>)rOO)Fi(6hlDw_D8WS@*Eno!sgA^osZb*Q z2*e@d3JMmtBP;!Wa9HSwFTc8S=G4sZXWULr@e7O8!kJ`x3kq^g_Wg{2&>%EIHnt*w z8nW4fCiPxYXn3R`COi)g#v@TQu zlq{$0>?IPTKtWDs1~t@M2zDXn+?bM*78o2PWL80*dG&GHvs2S7q*db4>^6s@#-Pom zpIEe@t)eDyAyoq&CTN73p0A>=mr!zOB`8EQu+hpGmmZW05!=aIz?IZP5S)dU5LryK z#B*X&V1*;37aft77;`QIF6m10z|aKmDvgUF9E_K~kT@JjAh^s>K^X<^QXLa6=PXp{ zXIY`hB^+Esr(o*9e;@i#2%YAgfmBV^0oEd$D@bX|tKHjhltf~@#r583cCj_){rZSdWFK?6@a%vtSWMJy6bk$@l$uI6ptoGNc2dL{-ADzLL|F188@!?Ygjn! zAD^gtMEKidd*nMEwDM6j0`?LNZbv3#)AMx<>A8Q7tleQz391DYcktKd7@ASg!bsI;buE2l0Xxb zL_CNz)*e@2WQ;^|$|NS&A$tNSA&}}Yw}A;nQ9&PbL~K0D4x9!N?xX~2Mfj3r&O&69 zoK0x#*}@T}IS(`hC_4d9*N_9mMG7Y+03trar39Dh33zHiM1Wmv0l=Ks^y2O+osnchH(CaDkAM>M9wWJdwM1#L)^BG4~Cau=OVMPRSQE8#sv&M0VR zf!Z(wEV2eNt5CQEA}Qb__8nCd)CERq0f|<0CI|(LiCeu8mOkYRFwDF?z$-LnY#`MO0P1vhhuds|%W#8q6RkF3a&X9&m5fO^ z1Hq`cYAOMr1_{o}I6+{b09mOCFY*3C5B@MD@DQmX#f#)1FrJGemFhK|r#LIOLLtR$ zs3_g6YABKjMG0$qa7abDmOsw>jz5+Hk26#HA1TSW1VpgcF**g&h{KkFE<=xLOg@lo zhg-|$?96yPB?M(;bP5VHjj?3P^b_HRxxw4btJl69_8G&U5+wv%M=Dox*CDYm%{-KY zlnrVF9S#LysK}Wmz!!qhjH#0k?%O4>N=?+NGsmT2%N$wfGm!*4nf4@RA~&f>OlmI4 z!zqJ_D~b8x5C#*XzWRylQ4%26y0>rNxnRK}DnQmGnkKWHP5yx{iZGC) zRazrusp-0T^TtK<<^sJyCl%&$2kUZ^fdvy-$*P12N=ib-9+X~!d9Fwdi7=T?pFVy6 zzJo$uZWhsxMj@m_Bqa!|Wr93Q*%46?IfRf#Qge9azG;)jAKbeeR5^G0)Xwd@xEyI3 zsfy55>O(OyQc1uB-U?&^r~*l>mvB#vgQu5G`KFmo^;Xxt-OCL&hGk{uEN}fki6vinvVDnndV`wqsPio8a8EwQ3 zOU>aK3<{~aEF(PbjI>nR3N0i&SbX{UC6BxhtL^*n-=csTw*Re%|2Lk?5(9CLMKpVB zF$@hBquoW;Kko#Q0c~Q3b5f&{lL`D{(lTU4zv-Q6f&~JjG8sTs=iP*a6_X~pxmsjE znFbzLF3JR%O~5gz!8tHYivyzM4*^U$!w+1xVfE6BCk`^N)HO;bGzwxO>xifhbn=-y zCH<_%tXw!)dMnJ4d4=!~j(`HLT(?5M+48EJ&o@q zZaMpR!wV%VA5W*^)DLzGMFm#{t>9x|tAHonN*XCAm6zSz+Do-4&mQhLh+k=-OqM3{DK&zw3r^3%cW2FGtm$LuHuFJj9HSSVN0OzNPj zD^CHEF~cHk3o&GDpnQ`y@`oXsSMEhl$9IJ;A)6qW%y8v#j5u`OysDjac#3_)_uqQ} zZq{Vb1AZ#YMRIP@Ww2Tqw6G(ou3C-I!y-E~a$)Q&DVPFwj|IAW$dbu!KeAR}?1{G( z;evEPx9bt`T9NIsY)1i2xMM$ z1LGE71(xv45!sDbl)}U|y4X*Q%pnY7#y#mr80erIk`5qevpaMaGgA_AiQTzWFu#j^ zs+gl9!;|P$sTxEdBi1p|Q=pzkDTyDUq_`H@8Z`4*MH+ON(O65`C0R-%E}`!Q$Bjr- zth~9Dk?DpdxHzMPGXZ5B3h^rASC;oVqQ`Xmu}HT&CG$1w18QtX>Fh$WZ4~5S-pWQX z7L(P+M)W{8Kqj*qwjvuL8d;CiOhhf6f$X$;lCGlcHt2I@J1|OQZW{p@kXp1CE-+pE z)xaWgibPDaW94OOfGCJ=l;sGn)*_XYG8_*Un&ps$Fv{Z*?KpMQdVX5EV8OBcA<0nY!~4jkceeEMD0tk0h@Y zoWg`NCtHb9lzY|UB>{SU`?jrFw3r~3%V9EDu=0Ub)KlYNqcZLilz--_28~Y~&y^*Y z2ER+OXQHIZrCVj8fV2;HkzS`e6~U#KfmG0+XeN}(4jUPyoOZXrNhEr7O|oDEx5YH7 z7>%H-Xiiu?Fj}nafXs+=h_oRZ4F-cizqCw7jLTy&aUZZy zDy904KaK+VE?0@jJP3BEO_UAs1G z#*8N$Kk0Vnu3os{hxxM({<59?)UgGNqGZ{(ZS(!)!~zAPr%oDwXwUvaafSVfjnwHZ z9{2XOs~63jzHj%=g7F1|gM*-itPMnTbspHU_5QtN_NbgbapJ+dp)EAQS(xRa7vtW;^g0Ds^H z??AULd-C|v<%<_TeCY7Pftod{?byC;>6{t6w(STF$G1Kb5-G+cdXipUI)Cx`u!2S7FJHXy!-9FC7W4NL zC!aWY5VmptoTo+CGR>USdw1_v&iJOAg68)IT4 zLc@dcNfD_TeF?ZHJexECwtYLdojQ8x=B4xVzn`9R`$m1y`LbnEr!J$iTpmLWM>C$V}V4b@Spmv#*>#Umz+v z#Lw@irHjvTDjbI&wc^V6!Rh=?Jd4jw;r@VV0`5ah(Jqz;*s zTFu{k|IN>8UjNgNuXO2ve9AI?)VJTPUbeJkeB8wEMvwaTYn|P;a`A$9UVEirWVl7A zyLA2nrZ)u1F{4Ir-MYokVt&7Sm!H?K6VZvuKGMvpN<{Z{j`F>8bt7;#}4)H|6X`h__j^! zUwXcUlQ?yad&ZFZ?^An?edlI$cSCLevy?p$mF!ACnxpj z+Ud%L^W{pF$heo%>e;3z59}!(7d>kDSMz7h7Lm~7u#oQyBdrMiA_>wLyrNRz6}fcJ z?qAleS&c;h>A(*rj~+!ii8pShrKPdV#|SB|8n`#e4vmxr5^BiXH4p#y!~_bW1gn`e zg{T5t#}4lA_FNOT(Xr|c?BaIz)G@Qxy=>mBzxV37a^*@y-5(Y&e7{%E*ytFR7FVuZ zp-j3IBLTuC5kvN*r0UhHvJ|_0>*n`geKqKV0sa>2f_d}))~(xnZ@nED6fkS{%+H2= z!b0cjl?&5{e=9<+E!@3xcO~x=?ww@fOsvVowryKGHYc3ePIl~xZQIra6Wg|J^V#3u zSLD8)mzNiy*_l=-&&kz1%cOe zqT!%NgXuRYWEF%Wso2ZN+`c|ov0j$k2&KkqbRT6w{9XEjhq}!w7jyR5O*tII!nR;r zW?=zUpPD`9+6U>=u_{!&Hrf}|Xp^=&ykJB$63l;I@YiCK_oY_}Hq^*8aM-M}mr|D_ z-xKcc14m$((--Ow?->79DHRMhDDp2HykLcgyCDooAika78AfvRg76I`3R$lg+r%k~ zIZ-i@XX|-g>{NU@9Q5cmE=i{g67p%`@NcV0v|er(-sM`fJ+SgcoT8j6pTW)eFy7aw zzagzG<_90FJe|F8j-0$?R#*4lAVfrB2-bZ(=cZT}t*H#^SW3CK13%FRqkpe;G9#Q+ zCMF)Nxk3d=cC8kTC8(9N+KnYq^S_Mc{gd#WYxbec*wAWHPAY&wR;ykEGi|frQ^K>* zHObI!e=~Hq2Y}4O9_7kK1(L|2A;Cndd2`!>Bu{_5#gn8r;; zY8U68-_G0J&#zR#aa)Gl{k_AgcWPnK({?_-f56@X+PGa7UoLl+DVOag8sr-01q)W2 zUkF}~^#;RB)O8oF2m%UP>Jz?2pp=99TW)|PP;$P{_8?2+PT|{Y#t!+Y8hivblP%z@ zEHXv4dW)!1>OD12RM((j-SGRc|DWLA=_aoyTf4_#za?tr?deH9#I;VkRcj?Ni|BLz zaA;{J-4Zz1Pv1k`gaP-9$BVAB#R~zCK|!W%SLM-7y$qfv%Vz7#dz6ODw-fXmn(*%| zm0I2PIxz(3Lx0L{&k1EgdXx z5R;T7*$lzjuA1lsSM$A3+K)udz1s=)0qatU#SJOBDSx0;NM=+wTRUy)@Lh?n1Gjy-pOc@iop!q5aW3o$QRHKGSaW6M zr!&}%#5}Mh*_eCd5iP4fFx(lCO%blF(rTcfMv+OTt|BfVe#Q2v##5jiX=Q5yA%9<( zWFpimvQzC7Er+ssMHF7d(qSM8nGYa@_53ks}8P zC0Bag%2lc(w>RRT&3k|NlAjEj%k5ceuXw6Y{A5wP=}aaj%)DHaE|a4q94h#HFtD9q zNYbr}=Ru6HW&xX4BTqjt#E{{W%9KV`PNs9>$6wDE<&;UrB|5#+3%9qvjLRajUulNn zuBXd!^s?AUpZnoQ+T*X+*}SHRnonl< zZI7otUaU2M(E02>)xi|x9&e!Pm7ng`XVGMCcz3to>4lge1aT%qX90`tCYfBwfB?Eu zrHBM`FE|#Ys^J-@)_9av!lH{HwFBTv-)HmOXZLlXR}|*giHXzw@otfZ^*YOQLtaC% z0uK8-0^K`N@xWjr2HOIIP9LM z5HwD-^BzuG*|C_zLo2ATNWmA7#O1cGs36*YH)qkn%8EXfS*}}UY@HXtXSAP)lYx}A z?tVXBj85tGcuHSAVPb%q#UG!+ySK@IjSUsV&$~ByqEd_Q*A1pQ-c`ukk7#CA%sV%0 zQLsV=7i+gA-;YA;0{t5iI%95T_u!9KTbc@`0*Y15qDsDZTAy0Eji9gRSUBrh44$6T zducCnG;J-^^j;9+oe0~wN(4NuiUQoS&dR^2t~FeNCF8vlC&m`f&pXbo^)43#oBh1e zZd6gKV;X&1&DC5?y$Cnfct$j;5+i7ilxDG}^b60YO~>8-Vp2&?0zr1$)$b6DaUcIg z0*U!a>~u$11g+OOjuR9Q<g{;&r-V!MVPgSjdCxTf2+ z5R;_%F58+bsnY3uLZU`vC*b*e?uHc?IcLprH@_DTLnHrwLkI3W&}hb!A(NlJSOK17 zAKecfG}#NTZkMI{JH`02S&=BQMCg2{`i*@r%$!9WV!2+nu+J8?Ug7ZMUPo7Y~bd%2m!tyB7OPZLxdur zLg{rV-}%%v%Fl5~ML9V(32{tN&0+q;Nj8}QTX+~qvC^&7@%RmtB5;qUj7VV^T5Xn| zJDR-sFx^;X$0;k@ipCTFNb5oi`o@-@(Q_aw#s-Hed@E{9V#97V+}MawO{?Dd9IXS_ za5w>zyDZ{xl-+g2Xkfe1Y||y7n8oXt9S%efBr{OU=d4g%Co1%wMgR;h#hz3wOlAsv z94^of#Slt5xWG-k3Uu?+H6x zoWCT=rkEBL|H0=~K*vyS@C4;20B5tLLJ{a>9Y zR3oNWY+60v({qo8!RRbD7x?U3L|o3lzURyS|3a4sGT-+o6|;oJQGQ}Me9W5axc(Un zS%gY)?Uk5O0wQ?#ttp?4h|4XUaaylNP|DTlcgdj|9ZhCK@AQ$9DrO5rNB)VN7CvyL zKLf871;*~PzSf2fpQSLEnwcc1fhcE(f>nBM7lSSh3-#t(fBOqfsJ^$}(ut{R4qa|| zfsmGqH${J%aM_3o^4t%1$^%QUTrugfa*|?W6QZZ0vWeu3P|?Z?4B-?7+JT=~aH=pf zn$~brggbS3?__im0F(}{Ut!4mjhIc}VHKQ) zeI8BriZv87H~g-1bNsf0$24NH_z?Xw_@3~nbrtZqoiA<=M$m`$UPfxAn# z2}!N=ANNkf1ftAE9Ydk1hGAhC3hyREQUpgPSywgE*IKP+@=kGaNTz2I+(Tk%dg4nV|F0Kmy79P)8~Etl+79g1And6Iykyw z$sYnD4|Y39pl)+1L3+K28J9UJ-y?^^au-3ZLd|P8T!#I(^T{Hn{zAUm+ON<*9CTVC z^V+~B*t6l0{86U(j-G1PMcZj6I+Awb0U4{MYMZ%wwGm@+FM+My5l7R*@l3OZ1ET$W zOOk!mzhQHmpIK|X?ADrPQZqVkm)=Ziu| z8Nd*U1dY}yJl=neh{&r2wi_g$R0$|>sp&@cd}#6@+dUzd z?|M*35^e1RO#EM2s~2BHg{yw6AIku$J`u!;Y7|%Z&f4$E z_lz?-J6c~_gyB+SB1(#4n2bi#-EZ_%(MdpwTU2PKc0;j-*jyr2LZP}fLt#1A253n; zS}Hmb^gci>EjuZ1D7ELq=)>eh_J{AnX$1RkROT3$xLU#F<8#|=Qb09%2DC^4qMKRE zT&cspiAgGbt5xI>D>`N#Jtbx2b~9LGnhER<)=H{o!oecT$#mD~Ug{i`qu9uqt63o% zosQX}qLpkCjZMP=bZD??oS{rb3k`FQ{SPgJ$Y1Oys*cL>m(al>7H}(O{&f2Tc_^y( z5rb1_Mo*9ZKJ4Wo`OlsbkUNty<`c4MUGW@CelTbblQ zgUu6@<+_+C8T=p3kJ)%-lNdBK$w zc+L^zlHq4qdz@6a3-K{(lL@O?1i1vs<8-+$3l|w@0!C4l;T_u3xWd>|9H31{6&9|@ zjIsb*1FR;=*yEKdh!s=n0xPw~G2*aLXVhb{IlWIh;{Yzr7LdqCu1Eigq>BVVm1%j; zZnNc62lPSoTS&#Oi%SxtX6O5?cDg=f1|t+tWrWsF$&5yCU^+qQRXs!Q7lRxo6T*qM z!xOFVr0}iwvMy%khleK6p&{3NAiQhti9Vvs%5c{#ZE*x`Cf_&1UN6sX_)Z5|5~ZfJxI|FgM3q3dbj_o{Ujwc3smZqp(%2Ot%P16e|5X`FDGlr2!`b?vz<) z!=g=3EaB`W#vX8!dN@L2A}MG=wEJ!PTRmF}qXA#Lhh-MegYAY)HnWp0;3mFkG#Mrg z)h@M_g#Bcksx!^D-c0gff+9TsG8X@DG&qp#pN?cU2~nhZ`nP476n9=YS>blK-*BtW zXp+A)z_IJ~i*3o2;-ve%i+}8LO=w3FUK()IZ0oa8+Y?eyS~>YkV_@gTlCwdRq8a8& zj`YCA6~tXQuL6&MWRj`4IVK?P)wGKK9QK1vpQQg(k97PW5WhFDnSj zk&Xy*Nk(&J!v>DKbcp8Xm&RKJ0>A-EwvsjUC+%VWl=9VM7IkLmY8ZH86^_JB$I0v~ zhIPTa@3~-Xp!amJ5Q(F}XtjmPLX6N%`lKg^x&}NnHGU zTPb0_AZJ8Jb7=#4s;qnI$ZEqthz`xIi5N1Jk?5>Y;*i*p=Al>&`>3SpB# zVO6S6u*RI|(SI=+=7x4T z)X;HU6DGywd8($zKlMI6N-Bdqc`d1cSXm*(!HTi0WM4r#7G1z#ve)N&6GZSX1O!AX z*zkIQ4fFa`&gO{qi*4kI;S?IJ~78Wk@tnxp+Xo<=^oC~szri)BFLarJ2U`w-N86X%pfCx zNZ7;`E^88<#x&oavz0zHY&-y>UhEJ?%!mRL*q2$vw%LP%1v4*=!9s_Zy1W}g#3Sv| z3l+@g$CnMiEjJjm8Kqz*UD_89Ayi`h1A{D(7^uV+yVk{gM#LSAZh7LoZ6m_NkL<P zyX~55o*q5-b|OaQ>wNMtEwNY#sG@Jbv7BWB)b-QDUS-qh56muisf}wXK1i2_m7}+% zrD|pbHVUby(1s+TG5O)_Br22iV-Lf8lrM&%$&Y1diuSdfZ!r2^MN?}vu(l!fr8gZ> zcsiT)U}w%AID`D|sM-Q#VAU3K(E=N2LV08jSb6HAu4x)WzLMzHtxEmqPiF= ziHWy5_J^TbfAngHin@P?!BY^B!tDwzciJ@HS|gP9AA)QZ|1_Fs#T8pMVi|ExWWCj! zmKRRQbjZas`#5%evtr*9{GwvL_^gfl)qJtO(sDNTvq3r`;qAq3FWd5o0)BxFhUKq^|x5(`B&HV2RIK zCYJO&0sAvH)49fJ%$(D9jQ}N@X^c#%6^`|GQ%K z%qH#$yG?;#d4o6|&rZWLE+Ap}s9PPr|7b)CsNY_O3vQl&4XT{jaRH>l9RZ5V8Q)Rt zr3L=o2-zUD=@CGtj=JPmA4b;Vw`MB++RG4jMM8rvMrzaKk4&aUD`UinDg%Og`5eoP zGk<236Bq=;ZmzmB-fcqSn^cYpK1byL;S&|`deb;1gJJ1VU3*jODxX@#8!tP}jU)$W z77WLZe9H7xTQF0W;`BK1QS(_3-3PnuqCaG90+5gWYnx>hTa?a>!#()j%9t6`*OhXU zl+o+Mjd7-&sYheZgCN0r+RC?Sh_e zDSvO6^|0doSdju|)7`?!n(pvl(CQxYhh6IUJS+Eu9QZt7U7*T^DBCwpE%v<+$}{;Y z3Q~TnuMl~D&%f0TkeC&KGUto-j72-4j6NF7;E;Vww7PJOqllz2{qyw|Mm}vqyqS8R z67~uWEB?p7K5tb9wa2RqW9|WCw^;++r@v53d_rR&{#(y=&Hud1`*9p=7cj`@71{+@ z{*f(ql>LeH6V2@Pi2e(Vl_qeVx?Rq~%rvvi>hvwV#NgO$is5MF)+9a|qsjC&B%^?u ze7ZQfl1eJAigvx`4D87yDvb~!gp^**PSvsw&YC&`3M}SwkrxE>DI#am+Xh?lJi_h!><*rIDTPC_jzI1*-v>Rs+akRZQMdIVTE7>U5vF#h-~ z0);Sk^Dpm@X;e$yciy=PCa}zwcd0V1{%;8W+bTF0St~}|j~3}oPL@k8?9*~XGVyS2 z(RbmB=MMJGf`fvH%_14J^SFW4oG0??EFxZx4Nmr|V2&fUsOX@Ay+>z_m_zh)RFM)6 zt_yVIe*XJoYr`z-_c-R|IjUMAx$`QWmK}&)S)$W;gA2W{-z_iIPf9(K{_k@B-F)_5 z!gpVr!9QNYDZ|6Vcal3mFm4?bbiM7tk-6q^>pnwAmf2YbT{=~=>}z!8^*wc>uYUK- z@Q6RqpuXX9m&4kMt1@)TNDOKb6dbsUnZf$dJi2{<(`<#Vj8FTFsfC3puSmZ;!(}q>c>@v55{`q7r}p{9(}f8lhGx z=5lg$vs?2$+1^>SSw&UiG#YmUQIhf`TcaZJfBx#g=Jgfyc?FR?!k5Sgd^?cvVN=XQlWU5KEGyFi(VbRN-YTg;Q zD#=Q~!SIV^d-m7Ji%g1diQ82#I;9wPM28orrE-zX9gj;+g-VHOg*v@X z%Rdh?>I41V5p?p=!~x8-TYK9)P1J+`ROBZNUn(8P*&ya}j4u4Stl%vNVgH$CVaRTA zfv;ffGOi*z1|SWJ+9$UAJ9l+(lykD&CM#u<=ZT1W3CdmhYRlM*k?nH%uXur4FIMV{ zo*1CXXW4rF9fBjOzdWo~yY_3t?6KQc*xG*$DXM$Fo&a-&c1Pv`6s%O4J$AGEW#>@n zKlW=_wTAhHA0&J@+qIr9gLn7p{EROPO-ja$*y3M4bIvY#_X>J)vLNJ02W#IGG()$> zB>W!R1(6=SzLVS9wS-}F914@$B7j^^gt`snT96}ifRuhr5ZK`8%pe*M@y~9tJknlI zm?kqDns?>Q;NT#K-DiMUMACq(^1brF#DeWL8SL?jsnTf$gY~UiI-@FR z6UQKzIUunVeGZW2=;a97M{zEgVY7Y3<3Yw<$pEtmZ^o(-(TB=>fS$E1_kK(VmHli5 zJ>?Fvgxs;=f(L-_e!1f>d%T-ZVuVniKJdfqC}&8d+{pc3A1prP?*>%Yt(Rt9{!e{rJ8cEpa8bHLOM zRSw|dVratR@B91vETkzZMc8IpvORgE$IYDbhmv);?9^6nQ4*O|j3e^ca9TBg98@eO z(~{eh*X(b{1r3(gir>K0kMZ-tQVJEpVQ>ifpK8$@&D5wp`rImQGfIHhNcD3_jMsN- zE`04S_;>@~2zf-HT46iW<%%;i@dWI_DrGL{;Tp;RK!;K6rOQ;pL^Vi;$Jdebmb^!i zF}%bjCFFdgkJtx<=m%X(8FT20PNz$q0!misw3ftDAc|%Ctfh0z4lzcWx&mNnu>Bo^ zd`z4@3XKWlH~ZZ{yVX(=FpQhvk$*uGgu=9?{fuDuv-Zc0Oa4~ds2SzTs}&5@|FYK^ zHc}W(lGON}7pi%`@el(0C(?H2y5cfpn~tt{qKN2q5^_LW{GQ6Tj&D-5(m9DRFL_Pf z3>8D*JLK=kg}5M!LJsu>z*9Ip)Rs-$1g5?hEh+g65=avP7}p9gG}&2Mf$sO%*1ZO; zqkbOSgE4{KFrH`*XT!XS&f{j)z3R8tZ}7(mEx=mkEa3AkgQ7^G#6J<5-`}(2wRFsr<*NA**0G(65(jH?1=+zrJPs)B ziejgFovwct!D-xHEv*WaKK%NsHs}3W10j79yB)OecBt5_100Wc{mBsKFJ$^1K(RiQ zP2qP4eahb2fVM}=+qAmkAzwv(f;HGYQ7F%9F|}D|T0}k&EDP`a_HX6M+;;YtQp?a{ zMbIW{n5vgQHWiBNmHuMhKRA%fs_~WfhnnFi09{yTTT36ubl!!T1&$E@+>iO#pB=#Uy7=vd5|`r#Gw1&L@h?ph z(debY+cY*w-g>8;#C(i6O^44~c-%2e;a~jusKjpbiM+MLi0AqeO4x%0 ze!W|m#GW~GL!gQjunC}>X_6nhQSZ2mc6!Q7zz;>Bvkl#pM{rFC&UnsjIt>Y#3h@mg z_)CHtGFOlfa`>x+)UWTXz^ZAUB2()PJm2s-wU$<=x8h z%3t8)%zmHrt^*&Yjsj5kyNA}D&-jU0J!Y1t{@6R%v9J~x4=LrMsXz3(NkIJHg_Fsf zMQm!G2yoj*OK&YiFoeFa>bvGY4D5AGM6UavUUTxnsEA0|km8W3E;l6OZn>Y?m*TME zlkn0L@D3eUu2*knzPe?=-;Mnc)}7It@jS>cp-GWxwI3Z^TJO!&Wm^9kGu}>8Zk-hr z_54Cuk42OvV%FPkx+MeIWCIOxAtCE#tQe16fxPfI!jSxPD%DJht@Kaz6(LpZI$AA1AfbhS7wTz;i_#TXUeCF6ej-qBEClNWv=*Tl3U z-)h{zw?ah`3ISPMNB%rP*0;kNy72NV;Oc)boL$)F`x0WBGEjY;Ys3Nq+xocE=;RQ= z9TozOy@+NtDHgtg<4w`F^}D^J{wJcf3L4Zrp*A}&2Q3BT3e}(_|7S4X&Oqf!ryId8 z7W=jyP*n4P0RF3MSIEyBZ}Te!*Ilvde(R_TUi=}y6J97hf!D7VDlmpQsB-MWoM&X$ z$R+wCEyTFC-VO?7LH=XStvAw=|Dvfs9OC;$1&so{V^mgCrP3FVxO&(s?ytMR?`YfS}<@_GI31_c-hc_S6^=alFyCj~>b?Uv@_b&M0Y{30R z1J(H<@-+Rc`>fcn&q9mvCH9ST?>Wxb+nW6HEBd!S*YlR|$I%^fQJ>%Eh}+>-AyDIY zECKkSnwGa2rUC=R=X1~27176*clOnzUls&UIH5X>&mC@Ua`gx=M*dx&Z$&-Jb-EJa z`>~fF8;1KdZA|gKpm~{$>$MEZSILr~IW0kqHuFLeZ>Qev*<0Y?M+6e`>aLiulun1+kWda%j$eWe5hBI_b+I*lCIygcY#!1)H zCL9E~2<%DQ!&}yh7b9mtjSrbn{g`P3ZwK$e`1n^Qx6@(Nwu{9xdR!xM8>y*^%jgSI zWC>WpZWtx1Ds|_WF{Q)YyoK)Z$>%?(9W`+W8R^6CVyQ^*R?faN=_CWxA8@(A=-izS z=BSf_a6W7$tw5x>K<@Zgc*5gpEc>z_!IzPeGFa}T&T~fseBsLDV4*QPNkky zf5QA z7^6CKmk^Y#B$!|oExC|#I_jtRz9R?6qHs}6#LyiH=o_d?^_4j4)p%G~@oTD1;~7j+ zzY>n&KrEOiv#fIc8RL%Y55b`QQVPvWl#8(b{I z7k|E>4b%l={d5c_YC~lrL8vHHF#BF)SfygAkP|LIG33Sdn@~(W4J-^sQ6vP6tpx23 z+jm@W;@VzJOxGir>wGbt2098TId}{VD3Mm;j5}Dd1O*EP-bAwW#9SE$3}xrf5aFin z$im8wKBu{tIRqq6k|BW?1EwoE_*Bp}i5|t@6CZiF8mfraV1WX}LJ0u&CLJ+LEE@A( zOvSW0YMKZHVhNs?VqTME<**4-^d^LAwDDAVcnp5{AMWurx-x#o<4S0@%M1rzd8O+Z zT=w6Fj>Pt1i<*u91RUo#vLGiw{I|S!9H>NjxG)lktrZLZ;Bd|KAVI7H=@~xwPD5yX zOloObvD4al8vtex);NblhXn^qgDM3S{2zz_dSYFrR(xd}Y`yv7eki1M=qJ`gScrDD zTCF0F=6W5?(%S)Ddl&0LYf!v<$mRWkQ)2?S7`@9|@WZeX%R%90L2WWx_-GjIfqo1p zXgR!hpx+=ngS&|9ZB`wp719uG=D1&-UIb*sVpcfHJ6vX_=lw67UhvD-t`12(61e&+ zN?1v7WN^kU@tp_yMp0oqr^QmUl|BBrv)rmA;1j>(VE@r@_l=^N*#DzJ-%Q&=YLG%( zr%RLuPk{L1h4LoQ(w<7!zH!`JGWn(4Yt8{+Rb7K%ct6r`F0#qXhEQswDh-6}u~lD` zc)3L5ELvE?If zlu<(xK*H!&JcK2(6^SBcykmnDcY+X}DyAmh#hK*$Klk~X665-HKL2ON<-^OjHg4v2 zMQ}GzPoE(yNvr3b;%?h~AOap0B2svN){m;p*x<_5u2gU+Sl@is&(u|BsT7=M}s zm8#PNMyu5-0H0E`MpEZ3qXcXgB|Y}Jd|cHt%ht8m)P}_t|GRgh;LcU*9mxe&Bh)4M zZ8*`i+Nkg@`1gOf#9T;t>~*0fg0RZQK>xHf7)FARUSt(|A^&u~y7j7Or_ou{?3xKL zmB>0r#ShJM?A_{r4<9&Ura)F)M*QKVi-i2lm)y!-wg6pI2h#Gm~F$~z7lr!`A<@#ZV>_<_z(Wa`)-7ftTB z)z$iCrvGl)kSdVhx95Or$;zujvD?pCI~{z6)&^`bePFhxjL0_3v&mKEG?ash^46IJ zmL77d@ID8?!Pp=585@lj9F2goQip?S`d7~{ z2WYm?>R$TjxTyRFz~Xka5vB|(S+iCl$}f)7RWREV``_s3Cn4#{J7a(DvgvkY>9LJZ zDOTj*gQ$f_!5gMy!!k>X$&%vLq6og1#&Iq*;;aF1iWt^jfHGsV1Ee z!Pfx_!ZO4yct-sUee)bLzq6;5Hk_r9%qo4dxXaP@=47I!aodsrNZ^E>o}rVvz!3Gy&R zZ~3}g?ZDV#>--tC++@70e6T}dShsrjjujJ}S37WZXYWOwJUi|Lt|Wxq>Z0xi$vgnh z=QU$Pn~F6rFjoyv^$|(F8USOPmWmO;azT}lSiX%~icf42di`OX7o8RZk>8WHDSkmK zQT3yxRKj`VY-pQyl=O@SV3_UE1iUqt@CsJ~J4XRZ95(-)`vpN^jucGU8=Ft}4GY4a zR=M1zl+27t9GWX&r?gZ|7PGKs#^Y|GFYT47CITL6xffqaTRR~4HjQ7OI_H?q5=Xsz z$qrq3t#4lH(^gA0%rJS+uLz9RZXSs%fUJ=WtLNX&z`6fmq3Y%ib0%Bq7kg6VSBHq9 zjBimW+UV%BL@(>1A2E$A^zlFF-v;mZEKbi~g}{4GHPA>m+Un|j)h>nQ)&aAHN-U6Q zy*y98q=$`P+o+lYsYpsEdZ`@^&46&`9_r3UMLp!Urp5GPiC^XpLgLF(_1LD`va&45 z>`R1$R)Kb*WdW1QPcQT#O5LShGd014>DUxd0nO&dHC0G2R*fhF&q=prK^)~TnbF+6 zjEpye%EJ%>x8$fKt#K5wS%gh+9Ggs2;z=O-o^grkyT}DN88#EAZIauTVb}_R9Q=L( zFyNP|k})K$ixoiHxRKh@2-MdRS`MY8E#{dBc&1RQ{uZ@aI=KsE?x$Dl8^kxK>F2{% zg9n?4>yB-8EEgrfe1-oT%V0jsnvCyJ_I$A*Skj3*lm6|ikZ+(xupLE@;j zT^?^Z#ipvzy9ui=$@HD5{CKt{j4k0#=ev=NpR48%sJ%1!=wM4SRQ|iEzfAukv7u1v z0>g|N3s|@FqM}5jWxFU6!-44}Vx<9$Nx}z2RnmVvq?mJJ-i58iZH@Aj5%b_+@7j-* zh=v>PIDv*0fC(XBh6D-u5g?uI|13?Y$^rZPPKkOWLSbpnjNx};KrGocOdQ{HXV^Qn z{WcP_P@V>CE@G;HXz#E-p0}B|S#~4=)TF9cEq{=_qdem7LRRkHW>AURb$5fC%}D;+zEd%{{4kjO-~Jywz#rmiNN>96x_pCn zMO>g0++i51`<%AMbwO#I`dp2_Pu7mBGMCkI@GA#E)Wgtw?n{~hRgvR9l7h0QN}wT+a4mjeWb zhp`qqLRZ3v*`{E3b+?P;KX@IwJ|3z{X+&y#(25r42K7fGYaEOG90 zp~ByO?H^1N%pN`j=K~GdG8BHdFI}LXKfXd1tdy-CHE37v=!Q?@gKer|W*h~S02QNs zgVw=O8;c?m3(45?k^HUq0&?vwsHWd_gfLq7{dxUVP#x%YnPi?qd%}(ei)QY}?*@dy zD;7@g++FlIF-UWp{pXc6XhMY#Mh4>6?5m?XF?$x)FJGhMH!ypcAXDC7$QZ$9@hkZ z&qbgds!oU4y*3NSKK6?K?vNz6r&OsxA(IQ7?WpwEU=+?;iyfzJj^`_ViXuA!D|K8D0GcM~qgY5@UNabheWw}%{g-FzG=Aiw3fo}1)n=FI%~qbkb9QRhIL z$K7Gg-&Yz|R+UqQr4)J{3Lf;Fs}Nk<+O-P_mfU9Wts{Ch)_gxja1`!&x4!(Q8@1Y) zVB)u}A=AGOf2mN)qp6?mbl7MZ07y^s+R_0iF zO-(}OYTdTQDxGhO5?tm4;@&3uSV&NlP!mTI7bHQH!J@064NEqA+INCcPm7)OpKTCI z5pFMqaP~D}0@`>NccfMtm24a9W}FuiI?{I@-JZK74SlTwiRjeSAGcsaPWCgP=zM;> z;kM9 z+}uCn$?H;*tk(akShqRil>ZJWrC-1--xnE4N((bcNHgMzbfyvax{tmm27b#PWsMxT zT72i|1KBjkR0Q%T6nnW9U^fQl6Oe0#TSn6Yb~@A@?hsKtku~FSW=L8x<;^I z&=b-nb2uF=E$kcCa8N7=lbMq0$yE2&$o`HuPf(<~8QF2oY zl&vX+ZHEd4*+gE7*Dq!v@vmCaqyDP!6JJ<0&FbMKBET5K*{5-O+eNB?GWe7wY&}_$ z!Gx$vC*=b^^o=U!#k^ERsK!jMzNQVh}WmnA%3^wZ&AZS;N7o5JAhF6w;1N> zj1^*w$)xmWMbNiQyl171@ya6muB=shq_+RY>AR5Q9hE&i2HbY>#B^aHBdHj!uf2(V zXpIIx@FRYqYR$D6+E=rsLq9`1eC20kQBpUzu+4S`)(fLqXTN!p2L>P1b{0p*Jc5fb zRIt_l`6_V`>AOg;d6i}fSXL2yz{i*!oMviRp~?Z%hTY8L1(zy0N4v_#tOLvEj-X;} z_e4PtHAn%Rnc1X*?X$q+6&dsVw+6N`CHXg_nY>~b8c8FdTW#^-ZBDSSBfUW<+1G_f!kPFJBn!5U#2j>J_Ot)YaB(*04G zkT)_B#_PrhqoH8G5+nzWjL3f18LNs1xz3X4Lc`Pj#Q3{ghk~58XvzRl@k69h)>oTE z)A4SqNC_W}g*W#ej+;SO-Uv}>C{I$y^OC}{{+F1N-!b({0pa{?^%M~M@ zJSrd>Tm^QUs5AD^nxroF1Jh{OS*RI*%8BK6PD_rc;?8=R!+~7Z85Vo~@kM;29Epz* z&iV{3dMT1VkxiLdF;2>vb{GMoB1IaEJ4@E1H(z;Q#XV4F%{Jx5?U>7BLmHb}FdBFHMc!3X`}_W;H=!xw5>ID^|B z#Le(ruSKJ#70m(vPN9U+;l;_^SAs zcI1*ztiuT#@Quh$`dh-{C2;~{^8zF=-O^Qydqr=o#}2mbOx zne+68WV^g4a0s>9J3rFpm#k&Pg&csE<1-%8V0!8WDE@8Y8J-nI4q@t>+$|$7W`5Co2z&LNr81RnyK!+h5Towi`(G(trKi@Xf2`g$3O(m?^ zD?gEwk?qREPR_}QOU$!GOjvtTc(ag#^!-u3k4UuVxlUMqR|oD{CsVn z7Ih?PTj5>5WP(DZnTY@g{yKo!;HbBJCe|hjBHcUz0!j`%uKq-_mw;U4yA!+7%o+-P z;^fq@WE>_WLhAUJ%e^>CYaV#CyvaTaMnGylsX7meiFRx(N=J3~=WOI1MKpG533$X~ zzre4(!R2_E}xr0%w@4Obf{@+lFT`uY8?OecZ<}DiM^Cf&BGd$6+j{ z!U7mNI?h381k?PyMC?0LkVfqDsD(5jIxH_p!iR~J`bp3m5E`^2zT1b2ox$#c`_Oe8 zRV)5XWtqMHRRzfgkq&Ow<5IDu^4GJ;Q$8Kh_thhpnS)^y2RHgNzy=xrILk7+G)>7o zH~P^dQ$0D<{lm6NnYLUGjdGk`yG+jIn|9LWsvi*T*z#^C%|Pt+ z7Ko)%)n#SWNq^}G6$&waebv?D)+Bx0ROnrs-*44?^%p`-q?rSOo_Z{pXuc#ezd<=! zDk}KWLo=eLACfak->o*YK5;lw2oRE{<}1Q@JSU>AMs* z+9#J&N7>7_)>>**8pb~lQo{EJDJUs9EmwDYePdLseS@Vo2TLDfpxCeE;dOt&gUvv@ z1};e;$;~q~rV9nAcsER=(Uk^GO09=vWfL{VLtHC2?x!}}eBBf`p^20GN99`Cb|2op zY~np1uMXVNE|e~wUA{S+t2X^?Qc*17_qO81z%_`%`#kpXiDTIO`Q%hoUvzoljTrdv z_*TtY$pMo_4HwkukkP)t=B+oFNTu_4J^aH9mCwv>IXyYc2}DcPuI`YiqJ6Yo>7#U7 zs>I`e&9D>vf{>RFuQ#1jQ6AQa2_^9Te8C=y`f<}sTXSo)a=M^!zA4|FiAhtX+2lA6 zLTU5uf7@i#s9LT2@M6;#zoqB(<_I0IEeOoLWS~Mzze&g7iiD6_F%~63KTemYx4&ko zQlK>Eia=9b`cq_bm?_g%q}Jpv9|^C>n1vaGWA951h#F)MQ3WfwHO3=hX*uWiI=8r8 zEt1XgJdf-3^Vzri2E$>s_`djd6~mZy{hlkFM5A%K+ND!sRsDI~)lz`I`?JLYseIC) z(_}s93g9*tl_&0Z9`4_8G%0+)=zXa+1_BS*uZ*hEa$GIHUxd|IGlU*G(xr0_{po|k zkDlr>+37RacVEQBW%YZMlkDNOxpgyTl*PLr%(y$8y4o!0nDB>QJDVE&e4gNJ%#3OY z?c}sNtWgiuyW6kUYknairuWV^|Fr>qy<5A!1KB-D4rAqtdc&6Z_WKQ4rG7Y-Ps>NnzJR_^@V85`q zIB%9grA+(IiubpI#>DD_qhglAO2^gm%)bs!8$O47`?=-|M`zhFq^0Fw#;JqrEp|29 zi)DgOEkn3Vt$is;6>SeLR-BgLZ0GjyD(eisKKB91TqcSEX&CL+T9qM#mvr+?M|$ub z@ezf8GF(l=mR?Fdikn_c;0Ma<7@#LO?t+%htvEWO*)PYrZC6^AI&p|MBnB#VyG|LG z^a*hZb1<**xnlcOw4yGj?#{*?Bh)yir!vV@IAL%5UmmV&)ENzwTNEaY{)If8%pI9X zgv^RDXIZv%82^6&6G800beo-IWHhPk4u?ybviMJ>rr2s$-@0{kZmv5zI#9K0HA!6y zy#3*a4H`AEJEHtnm#btsW`u~vt#Y2PROXFIfs23wAorR6brIyJ$Y-IbP#bX|pCaMh z+xtO-h$3xpdj$-w*B2 zNj(XglN@$|ZgzQMbvvjH%LwQtOP6VS)zz_yiC+(SZ^6_bzZ*Fs;N}&JD=MzTEjO1) zEIVb&q^zv0sWYa{nm3!Z`D7^<7g~;Um#fG(Q?q~cr(wV;udbl62mAbOwRg))S^sNOjG!$E?TvlJ&qjbBg&wYKDEckWF zf+b59Ez_gpZBa2EtIdOuZn66;jvTC6T+t{!#5I-G{5EbFjb%vq2NP z|5g2(&rAvw2MQDws}e&fIG3u;uqxTAtV-+%%IZxqMb-Uw_Mmk8FxNn{BL3oK$&QTq zFx?i%PgYsNj*nQx2CaHlYNfh8+b7>OS3p!xNib<~douj!CvP;K@i& z$F4geA%Q4(YR78|b}G76mfHkmVxC1nEStx!n>bTbl36je6`2_lDLXj2r77oGXsGGAsi(TO zu1c@KqwBf4;6+lBB9)EeB5{iiv^o&1#p#D#&-LKv$%c(xE{hoju5B!mR4dkH0Emhs z&nIgju?Rsn$F-n6$`;Rmd`7b@k_C@GaNp*0`|QL(d$zGk25x`>p*1Sr2`j*3tQ&xK zI6SgACxFp7>AFNk&Y~XLcVNV*(R+99s8hGbkpue{&6&$m;%>Kf=X1A=LM;i26%6gU zVM~I^IZNkR&_mwL!=%6kpg^z({c*te$NLMAv*MU!Q338hfioIg@CT$SUDt<9``y#D0z#6m4ZFa;`(E`*|S!H^@|sh$3H(_*Pj2 zE5BNy7gW!O7fK|O3iMPlKq?idi3}l1U68*<-KiQ8Sz8Wi=SGclk5w#=8l-(n*P+VH z`ckwg;8Rfn?gAszp8=MUU>N3j4O|uUIRiaFbUq_jtX5Eh;7!o&_StOKe*GSO?63XM zkrS1qVzJ>Q7A*>D^iU58mr8XI9;nsE;m`|sVdR7}MK;!GoR7?y-Tr(~(Tt-QP@wn$ zx8S6btp9!Oo{{*Bm1D*iV$jN=06HjLAde`K1Yy|lm>tpvitreQejH2Tx(fGIc!pI@ zWpUq)DqUxHT(K+%uW@4{7qYg~SVgQBtf^#`up=OkBK1L~2|D zb7u#h^;Y2o7&!^&Cm(A6FF80DGZLbijq`t@ruj{Apg^&cxRsymV9lPTkvK<<^F9a# zw1jcWc*#KR{MaqKSsJJvj>z~{h03Ou%wRId=wYh{1XTgzfW>Cd!#AK!bldnWWLpIh zMS(07=%M4!0PGK03cViEHHYud$Hm!cg{0#%xeJ9D=^1c*=4m9pN5+<)hU(i8t243t zd{mtoM-imJnSNCS3wNH16h;>C#K7fn2w-IVjHJTh>NfbU@d#O)3+0|$g@f9-hec1wik=tkiH=WiQK85( zOWALS5K(vYP%LNJ5>w8i8YUf;MFS;`w8BY~lW^+c()~a1REicpB4aib`aEKI5Y)n} zDa!E6=e$rO42X4x#*~Lcl}M<>5t-Y_k{VCVvq=Gy0tN+w9mXWbpHTq7!r+In7>S6i z&6R8!EW6-vD9Dxzk|R|v*c6IsOp$DHhJ`huwsiOHFYCSSAsvWJ&|lEmY}ks zlrUJV$eVH^8=;JGL452D6nrY`n&3f*@geudQ%OsekqZhd+emP$l8|A9*r&^x43kgw z2`?$Wr2=6`*vY6kRYc@L6;raMC?nbOiH$luK-uJknS#7x9wr4$3Pe&s^}NV{VsH(D z02&4!1+T#l5K`08L1SkP4HRw)&O#u5pc`T`SRxRLb~=c0I17|XMmb>EP*NB#_Eah7EIt!+bYbtzjBZ#wT-au0uOXrnz%`IWP z$-5MVGf^357H9G^Vk0dWgXmtwua>+9cOaEux@rnkz04;9wd%}mdd$Y^Q~kQ zA{gAo%9|WgYRMS67EqQ;BH?}VtX41?nMJMdQxO_y9?ToDk$ZzUVN53w{wA6t1>cG! zlCF7}6fi092MQ?g_(NEcz6Ej6S_qcRuKs}h~64IY-Djm#ew83iVp5E&IX~xFogI8m_bhwB-u+19OORWWM6lQoQI+% zRzH!yG4w_PM)wUI3j7H^vK1x!Sc2AwgIZC9!iwpRy^1d>^IElaPtWBIyh zDL4cMm4wc7;0EbZ4nQg3$337W;C0$Sb|1)!4qNwm*`fgxOJLpDnjHiNfZzwDMGT0a z>_JBIBA{|lOuTJjBVd)1++%4X+Y=kODFGtgYqDdoXjBL=z)vm|FM)R;o6Gy01b;>3 z54h}Xyd(J*XqJuXC0byZxw5icisWi=74k z9xp5hRE2baxbP$q{B?Gy;43c3jHrf`3~t0j@!1(U%xs8XUZN}k&1qv)Aqn~1eqk40 zUlg7axd`~hr;B-*6fh}p76po(9XE2T2;55Litq~(iL@wVbKh@semi;cWUpSmSb7f5 zVP36f6u}s=!sTeWJ{yLayuE0>S^ZepvJZ4yhCIGJ|( z)}FnQZdn(J#UC00j7C6xfRd93)DtuWzGYi?w}7#zno)}&jf#bd5^_b})MUGNUIUIr zi4HWwIRrt$G{sEd5H_(?LHHH4Btk@E0eZx9AE-KI$`q^J(zWZYUUo{>*uBA)=k|&l zL^6WXTrCKq>|i>iAOa+HBz-&nu-xL%3h*{$d?`P29&2!`U z1$-8+Ag{~zD1yjDLje^hfZ8ZlWQ3qbHt2Y0ga#iI^v94!lvKhLArB=v@_Ulg=`2;I z3~muA46b0z!=!*ofj>|{&H{A=f^gFcPAkPX z8zBHId?f-P%NwnTtDcOLC;Q!YYt3rat5>U>o^s-`+wZudTGi@RD{fx1dco}3MCUY9Rkq*5R--2Z$u@wBi0q)C?}G z14wiSJPw<_cG+@!eEbsw`xD!5(`VRACj9O3N0C<7%$#WeyZ*Q{t&8Y7kkSR;wq^gP&M(;1yM}%`NV{MSmcIzDVZ63Xo^j zq^QhP;_N#(DT>sLirE)%+QC>r99=b<2lN>fR@k!_#k{D7tVBq8nd($6fU^4y{1%ZJ z9g$Tww-hZ5Y4WpJ+-|qi?qJmL!ZT0H%YNUkojV5i>9t_%=9uU>c+2arynK1hD;n0V z&nN)iU~m+^B|juA?8onSKxc)~C}tSPppD`~VKtHhZ$f8(Y;4RYpA2o@q;XnW+Knw+ z{QArE+k4z$D!ElPnJ1G1`6yt0_0?DN1r)g(x27d!X(!UM%9oCjAvt&<$Rt>H;`p&2 zr~L5U#Bm4r?QPby$+D$OPN${C#>R~pIr7+vlL*sxhcoTOiD6%US-(+3m&;}K28Mq* zET&Y6y}NdNKl0nd2M=a@bCQ#fPXB3I(*AvUK3~bwCHHLJd@B8Pr!HMU*^QgGj2by& z_OCNzqGKvosY0qsHWnN`uy6mku@eBk;ln+2zs!*|FsZu2Y zYCv$>)St(W8MAcR($b|%mnmDu>+}9R`G;|1$1PpH+~IIks$ALQ@vK;}V*UE{zb7S) z8U1a$_E)=e^S&81V)DfA*^1%l;UhR4zwWwgGc&X1{x;{EZ%1$5yy^1GYeYv!F%W}Pb{WwTYTSixzxM>!nhe)_pl z(LTG*`L4vX1TbSp)Wr_?A_Pny;=U0gWv!2 z4QpPG-Q#gua=-mz=#g#PV!Ukjk!O`HgW2^)G_gpuC_(bPcA2u&{SG~_b@{T#uD{li zcA}KkJ?O43d$+B1Xl}dL?<500?fCxv*i(R?oKSo^6}f$9l4!y7#X4UwWZ@tn2jt%}?ImdEWFN6+c=->`D&LpQXyXCEu?a1Xxy z<~=(%T6|g4#trM=tIOI&vtlg1mwISC|tNBztFAvR^C*h;dl)2 zk#OSS9S)U{Up%d=6bWaZSsuu(ydxQqnB^BdRATDGP(U!B=YA*rA)gSrDnjr>LPAl5 zGhn2v@*rVne#V8$|(wL`W2p@JShhbUf=)m+j{nV@L?-< zcu1XEp6B~XKUy8p@d>4nTVHwU1?f<#qoSBV7c3?i{y>Y?tx^tl*}eb3fF}oww%YI6 zvp?x?0|x4jsJTC)G#53+haU{Rxo5A}Uw<8?#F6AD9;TVCNo6fzI0#@KvZ>xx^JUR;^fmMdiz;{W876si&T1%ZPOwH`wf!DpjiVzWbh^#*hEo<4-u_N<@2n3l}Y_ zR<-hg0gwB1>qkRA=yc~DZ@u{@WIA&6*zB1z>({NzOl*TjjXwJHi-2xBnzU!>(xvS? zcYp8wfBGzZfi6nTb1G>4W%itrUwq;{e)8cb{~j9?1B{A3NTfS~!k|N42_QTubErRO z58+*XIz=D<6vZg*ga z!j3t1HcJRtw{P4ueZqGQs#d96vBJ`svlcH|7GSG0)Ya^Ci87r|Y}M+wXbw{R{`>D} zd7th7Xyd9?n%BHS%g9*0di9R2TeO&%#x2_9>MqUV#MW-#-o5tNn7WM`xUH5POH|9N zI;QT~?e#LhX{p(?WmL&BdAeoKo;_NLxLOUGWNNlRbX=(*mcDZHA{NHwO*<7wd#RngGeA%4YTh_1D5=zx+&=@X4xm#Ry z6_cl;h8J?UEm*6taFZEFzvi?!#4H3c6TQABb}TzKuAe>ey9QONG^&32qUqCDEL)CP z%xa!?9j|v=tl0r;lZK5C95~=*LKMg2Vvzz9!5p4B1d_7!dFaul8@5bdxa7MjKYsD? zr%|$mC@5-@9AFp3J3qLd&Kwz{jwfkwoIN+CT}?a8O#YWI|Qn+lnLbJiJ>3!oU-;(tW)YW zXwbM}o&JwLBCGJRERTvM23bR-JA9fWU`I`*ENYk{Mrp+}zJh}&JvIXoF_;PXs#UAj zty|Zrzsz|0>1Qf#+u27E;g~h0A<7J{Y0MWP&$A$UHHyWs;Q0_%!m9{E?Y=CI!x>K=I?) zYB>y+3~8rRf15e0a^=d8^nVOk*mvL{0=3QRKpO&?L^&Z_OIi{uRdSXuKYh|fU)rhB zUw%m+#Dp8cleOhsVWw?o{Lh+3QK85_@gn?LWW(PZx^?+w`t(DG4%wq!Yu2y({m2pQ zNdP>|@i4v+qYlA$o5$ymE0J(*x7)^jHz^}4FV_>8IcHv;W^MMD4)pfnL*Gk3y!&|4 zuHD-}+k4!FSvD}rU_op%bbr%UZCRu^^0SXTsmW8n z8+UZ;W=g>uGp5ZTxkvjwWG@qsMq7pjz&T~nEX^A?(Ok|EAAZ>Jrkl#us9CXcg;5`V z*r9zpNFl&Yewk6VIvFWI6+#$VlrCMC>s_0-1ah(&Xp~MUvv>PWPgWLrx&D?eGiUvF zVE@4=N6f|zTaO$*F7iFgouDY+`UUqXy2a;Nw_@3rO`D0$Vz;xaGDDHY3l=PyJx$9x zy?gWesXtD-vsVxFGXNTG^pDMyPy^|L!fo&!UIShE5vw6G8xph#vmmEf!7I&j@GHO* z$PVJERJyu0I!QP@$Rt8sfm=S~&PYhkLFpJq7Ys8}G0u_z`Dd)4#Ok0#Z=|7y4Sdd_ z-r}!l?Tr*eul(s>Bi%oJY(6N`6!>G0DAI+M!T}Nd7@z0lx}SaSg=Tf?-+5=xUXMS~ zph+{37>x&-5W2mHgMJ4xgl?`|AJ*dT%5a~(69SFJfI0e{9Yz`!7 zJ_4555}rpl-P)~kjhel>-`=WTqtCwj+JjsthT5VKpcjjQvW~SM%NMa#(eCdzV9Va5 z>)N*K*zx)~3zvYaB`>S+&Nrif_<8CLty*{Q)M@wLJ!L9XdVkE=pQcT{wng*%Zoj>0 zvlfpJc#=0!*W?9o4}ztJ9u-&O@mF6P^XcbTHE1wq-1t`4cS6e7bo;R3Bd%-Frb*N0 zOeln>wre`ws9EgITK@%I3j6XatwR>Wf%lq8fyG8v*AAa^F zqB^Y2&3>|?Zi07UDe3efef#wxLq7cGn{Pz_BEuUn&!3r@{_Hc)x2{$9p+_FYEa&dN z_j_dW@Jy3bhz8+;2e;veXF?S|hF+1s>MOz7KR?6;f^kT4#_w!?VX%r&ERuS}FB~l2 z8_}sV?+CROG)o>r38_obGKCMZ=#O|B4mU4M3Y<*=Wlei_a8U`NBV=gWl4IJo{l^>E zO2iDCzJUcdn8o-STkh23$4iz@K#~ND)w*slO4h>25)!tERN^;=&v`j{IT;ycN|r*! z0oDTkh;%+i`xYAu2tDF4O09sgSe*fcad6y<-oRniPM^3J_ z9g+VrmlmNG13w}KC-4X2%gxS?i(!2X;{qpTfxBd6raPTh<~-5H$UgPLoMg!&?*!br zcBd@|jA00nmt(U#bETAs+v32)Bk?mnkW5Oh$S{=h`!mxsVq;?|DM_*nKPxLMCN6=7 zgAV4@sZ&mSRBTKv)*@J=`}}$EDtHXlVWaTeSgaYDS*~a&$qRCW=~kFe0G@d|^;BF! z3Cg7uGpK`+^yN@*N%cv-#YedjtiaGvWk@(rMHVpxTlKOMkR^+P#9{-)Q~4H3N^J5b zaS}itAV3H@^NYqoNF{RPonY$5%Mm#Vf38xKG(C)NV1L-k8`^CnCCDzaKeZ<1;t>M~ zDHl~Pb#l%_a&DCFT&bV)lk*pNR!PYVWb|0HV<$2*(@u41S-CXEML{y3zjV!*{)7U> z4!EUh2|$U9CGp-Q-B!8;EfbIu#Ve~81&c)|!a)a?5in1s+wfQXdSF-rJb`!$bfn)I zhjih97o{)iZ_p8FN2kj>bVpJo9$BGC`U=4X@DR3jaGtn$84a(>kHT@hpAfr)0I|51 zlvpvzNpi15S5{Waoq`+H*9F;R%_E+wcu8gs>ELR`qLhOd#o?*|rHY2a`rOJB7P@B} z8Vn$fmdx9t}@eXm2R5T#*-%Km~AtJW(L1L*P$EEnuqx#VjQB zR1i>N2m!YNqmeJ59SS~Gs4ayx;w3=^;(Wpd5k^42Btcyh&JSZw^@W^_Z&iha!$KFK zHsOUu1}sdQ81G9~r7X(sP_ZAQMkQSs3JYbG$x$el$iEVW3EYtsDbCl@%8S)ulbJOz zM|pPw%v5g_M6vj1&YJ$?gz+|}0+m-lglIaF(r!qPg;L-qSSHSrbYw%{`sGVscyhqc z`ULv&#;dCw@_?kQRv`f9ifX3*}WsV_Bv+hY}uxv4;fuS!b!%!z-MX zEP=(KmzSTIScAOiu!W}(mz0% zUCJnB;Vt+E#8rv$j$Bd`QYTUoybNcO%o|ZD5SNYtXMzGE9S71+rM~v!i`i+XHs)E!; zL5#PJPo!SWqu5iR*a5eoLC86u^a^^Qf*5+~#&v5K%$bFLK{}m?w<73?ml4)y8{L~A z!2%Jvv(i%g_qe@Ur3#l*vj$gT$?v;eE*eKa&mGc7hWk5hB4ry=5-5? z;9K#LB3RA0WYy>K!Xyxu!548dD-jD;gW9OnH z>iXr&z#U;l9;RVIKNjDM#}XNW`15A}I&H_cO+p=U0wZ}wU1PTdhP?guyKlcGZUDqq zP$4pIwwMEOGa`pFL?=1K(bY%KJ-2JuF4@3SHhE*vPCiAO!A~GM2=b7IAym+Co+N~% z;)YjFS0Aa+-0!70$!LPqe>)o@L;I;Zm z8mheH=`8prCG#lO6p+SNtPYH?0UM+x5txuZMYsc~MR_YCtvIsI1!?F%f(E>V>>~i> zcY|Se2K9(@V7Gte!nxM?=z&i@)vQ&^rHkj-DJ6uf*VTJi)SZ+GrP5Y`wlC&?=2A>=k$5*zy9jo z+cqGyiX&NE4`QbRbh^Dvi~{(=gNPWRIN(39Yul53dT4nWymtL{*Dc<<8Ph|Q$T6UH zjyu|Fwfj6#sIEbofE80e(Pt}wuzEEYEQS2b3QrL^1q}u6{WhO>`s4{i-h9oQ!>UmP zb!fqu!jC%N>dA3wKCI3Cx$fyxzJKq{mszptXRWE-%C{5^c;ZbL!ZSmHfY<7<<3CUc zz(@xcq6PNv*fDLw_@|zF3U}f5K){6?8F!A034>gpL#%Nu81fJeHJ4DT^wI+dnzU*u zg@$k*B7n}kBRmY2D+ns&7p%m%5E?=k7D>VRTr6qz9hC5I0me*DCV>NT#w zXz%o~V_yybvPtu1$aeHNe3jtLWdELBQ^tOiv~OR!+n;hW<<}o35<`wZP^N6TU0c>M zHFrz5+W_MoTQ+|+?6Y~l&5ClmDpsy6#N*s8&5LIyyZiW#sVTHEUfS z8y$1_;DPa@N4II!V&a5xp4_ZiGiR(=xhgTS+;?NgG;P>m*ry+?Tejr1H*oaup;paX z02jZ_o-y+4;Tu-3u3V{NOhN*^8}C<$^hpQzj2$_A?}oL^Ql8FZ4k<8c(xjHnng!hM z&p!INbjecRjvB$(qH?**+&S)%BfeU`cz)H&l@dyo0ZDDS*%QW$fhPO*?x|b5wi7dI zz*0>HEL${x>cnxowr^vdQG9%85dU}vApKZ{(kc08nx;)Y}`!jaFEj>OF(es8cbIX>IUw(P`P}1eqsz*5;aMq7M9a^(iEr>a6 z=#UEK%1)RtVb+Wp70Q-QEL94>CjxlF5rtoh+une7aQ~jkW4|pK7xTsEpKRTcc4ggquvMqSzG?m1ufH6=WWl1i zn5fxPru2R2AxnHI6{V8AMI;xU`idosH>_KI?9jncUk^WZ;&_94b*BIP)7a4?n18NR zr8>}@la;k(!Tbr|j@q?r$7PpQC=naGZr!R$lfG}pu#(XX(^5ZC!#Cqn(3>g@aP|r zQnBOVjH$(AszuMs&ggwxx77>hHmF^5&FbZdoQ(I^EM73=-M1>1EBoOqFAsh19cRFc z+I8YbALV3aBlqdvbz z^Q7-4{O#d~X8btW?)UEBv*Vk0-a)t)DISAr_86e&`rRXjefsji0hP*^N!q{bjxIO& z1Gp6wD?o4GRJ)IPTxK~ZC>A|m8<&py6fTl?{%l2 z+OlHlUANqX5}ac6zrW{%kzdrRUUB+QQ(k=T@1%-kh>$G)xe*#!9w~}*O6$ocixNY^ST9qn(`p&yQj2RPS zw=A7M|F1XS^82pc(Kg%jgP(#AFcc8|IeIGck8(zo;7q7`)+}H0R{#FvzWzGF<@o2{ z2ieQ??;hWLlj~UI- z=O52Lmz$YwcRHH1YT-{#K6T<)`sq`1X3s>sMJ>WNl@d#Oty2HXU%BBZOw=|~G|v|Y z1w>z3tPVV1dohT!TeDbTm~`;Qfdg*6{f_$|xYv)ylu4!RobSJ%Y;nanW8%h)9{u`D zFM?1geHnuzUZ{CF4@3Soty`uY?6z-T(x8EZQAy%EY5&240|pHWSY2}`;b>bFw?l{g z^ZMIvegB;|Q5l~&mOOX%ueIyevE$+uHLKr$OP6lT7cO|9=bcOD%}K0UdBN+#7 z+|lDUPi|hb#!XhQSl+U2JGwGnRjXI4aaYf~fBk0kqmT9XB_u9ew9o*ZT)!JUd-TzM z58nSkmm9CWt4o(V@44@Z0Rxiv@4T_YH3#?ZIg*;TZo!QzYS>Hy4C2BBVT#-rQ-*8 zr=B`JaPU*i$myP(6&u$&Y|i6Hj{Y=u-0H=PZn>*FkSB12P6K_qECSLz9(UI7hpJYs z$`D8VdIz+(-+sGJqb3Q7<%T{xc*La1En2n3N6H&-{KFb}5Bf7`lCx}_#zNkg_KEw)L!@9`}wDzw7&6%uRb4!7u_o^um1Yrfw=ZX zB|@^YJPVZ@N)C?)o=Tl+v9Xw?qXtPna(LmiX{&ecbhw-+Po7$}YK!=#~lav?JQrd#%piC`|=Zy+dR1`$;pET4Z@G3D>lYjrqu8I526TKxNsrU zvNCr;-HWQ^4~Nk34?*T_@u5Jmt9v!uCNgV6$ud12dGzHc2iFuBMw}kJ$d&PAsd@a0UXriBeQF${@I;tJ@A8Oak*>8~8-SiWQsHuBl~a zZCbaM9Eyx7N^<;-M%=v8i^roD_5 z-oE3t`}XaFydKs;YAR7(&p4fq7w@L^Yj(M@qn4SPa_kT^S3|@P{uQoaj1#RE&2>=-76dA`nB2(8nGA9mCah^X^yIO>TBs~zTB+s z+cup%w7*I9Dh;bveCMTSx2{->$J*UnHeA)Az1yng1$-^rwE?Hu`bZ#Dn1&1&1XZPK zWYvGeR;_HVXrIMiy;l92ZQEO-OL(l#s&%hCdhnpt=l^~Gu3mR`Z*fJ9XP+930ZHn~ zV=(v5ojW&c*5s3ay!yfWLtN1@xKChRShXsZj~{~?q##K#G=wS0+@@$qiVrTCCH*%P zD0UJz{SA!9N}Gwj=U#mA?z`^z;Dh(??b>b5<_(Bx%;0%M(Tl55=B}6)(_^FXGpwCT zIgyZ1B0DQXJ9P?_770#9%&Pm+znIO_t!}gfjDeM)2PD@H%V;t~A!Il_7Py`Yer+`+56VW)y zl65n}dSv}4u2;w)z+CI{>OjTZC5s)g31SO_Kl11}xDhFfjf1jY1bCH3$inKItsB;Y zSu9EdIrkk%s#38Mk=SewD8d)S`Q?{iptzqga|Vw(weKLh7WfC69Jr8dSd5V~+0+ut zlp=#^sVOn#Dk|XSQ)WrAL?L6C5DIO{3;|@5wfF+Y%$>4sUubXR5w&KEilV4QD={ne z3-hf)Gl<3{u7J2$?Fg@xYn$eP-@t*0%6nzqx`&gJMvoth2LejU{%Wj9Dp#XMa!RtS zc-1Z2cJAO!rVAh=Dk=QXLS?3}i3>AFG1I5A09(>xiqq~4Ko)?VHAn0$pXwvp-wv1y&_|UH5oz zU-4C9i;7*gX8ob$6ZC0t0^?rfai`0A^-b4(^4_bhyWWyv)w^`*^649Ice?2gx8}lh z+D|fQ)UneOt3Bzl*d^krs>Bubk8D`0rJs(DNoaNBE#FO^lAe)4r=B)z7QXkGmm~jR zlsn)po0y>a^A2p^WKBQmVwXvgK*1`T=E085=GC2kOi_FeN3PxFG0Z0djQJb1YVC-M z{eI$PHatF_b!r-RjZQTGe8?;uuHKPTJ|SyRn62-49d}?1mLU zXn0Xf2}k9syY}q|(J<$RCjp<1fbJ0CS}n|h&4_|UklL%|X!cycjY&I->tORsN{5Lb z`})8xoPPI=DU%k?n(2f?V{Qock#8AiiIt!&fQg1M4_3Bf|ACW25Yc0?UDfG^U0b&= zU9~PMHh$;sq^(a)gqa{lq`pk|#r zeLQ*MxUW9nw0?!dm+ekXcAq?15%Efi+q6#Mh>W`UnMX0E zK(VWSH5f+*wWVcbKJ(Jct?D)GerL}<{Rh-<(wwGj-b)CAy zM~*@y1ZG%6kD5UY0d;GK>+uTWyzaW|F$rtcu7hTGUe)eeM7=yjIGaP>6R#;sPSmkH z*2&ww_dodQKi}`rsWW2o<4-=hb=Tg`?K)i3?wW;*mP&sXv5fWQh<~=yWiR_nms@+^ z+3S@z-cp+b16eNd#6l*OB#vMxV}1Y;xCh(tl!eghaK8H9`(KV8+rClr9zA;PPdbQ% z&o^Km&XgqV)oOah<_t5h%yz-8yWpx|6btmp$NSZMPJ0sKY5Mbq`1rSJ=u2>G7 zE5C_Q0QXy9E`tI`E@K6B9e9HwC%WB>m>dxC7OaGfy4~)9G&uXh#jaS~1wQ%sW7sPG z){(}AjUuzdjB#mTwdKj?_6`L3T;?-$rg@R*33pVhRriiZAAjnvkG80L<&)1npOTSD zh2C=e9ntZn`gFVP=4-BN)S{J;NVg>)KQ?ygN2SUnGFG4(8S1FoR@KDR20TIPR{VEy zEh8TCAJ7Lr|HAtpf6}IYBUnC)CVb<5|Krp(^A^1D{44eAHM*tm1B3tmELxxq>o(ad zR4Q98anr`lZw`Fo$dP24tMX4^ird8)q|a4!ncP2Dco7MsY=7`oBBC3efq|&3G7n^> zr6N;5~jN|GU>^-XcRB(7Vk?xl@=ElZ*@A|4C6%;AZiUmE2{hQGSgF|;u2-k zN^&M+d?a3(StFxJoCJe1gj;Z`A{N7fG;D>#CX=aYX=TfnmBO)h5pd)=iIZpit5~#C zsp)YE@kA-S%9qEAm{O%n14%gKA~uIegL-;;2Cil8RvSAxWTj`sCzKE)3}6k|m1UwB zf5W7d&O@U0Cu19PW9Mm$7nUkp9^DQ2$~!n7;6?CS#8|AW#Yqgf2#$l#!k@T=)NZx< zhQSFXNp*Zd#e& zFB`K#1X9V($w9xQv>uRJiRCm`1R;07Z?l@qRB;}KnDh0 z5J^F8x)wm2CrVo+JKgDwqU!@2!H&lx@B;O)jo~tAz^4F3AeZN$6_5asp!q~60>H@5 z3NpAsOrwV@T)@T-Z{JRO0dX>6snCN|4Syg4Rw!FVN=LPV_{p%I^*gN8a70@icGMPp zf(+<%Md>#2oh9lA;s-OB_fyxf3P*G-7%5PP1cab$%Cu#^D zfDvTQVvmj!eBniy3QJMEc_OJpC_bln4ri2X5-D;ela`jK*bgM_e<4-wev zv@w@J26$ifb8y!tE)_*zE)k0UMYIfFxK){igsfcmw3#zI-*mGF4H42bcm=u8gDZJo zAQu(BtZ0=IN$SL8w|G3lK^gE+1w?ux`Dn>1RhqSI#kDFRD_mfdi087d2J(XTxgHP9 zR2HVgtC(_RFahg@Y7` z&h7C=#l#?7Fb$5iJ2WFIITUoNlW||T=U~D{UPQv63QRlh!mWc~Y9d)&kc0~-QOfoQ ziCb1?>tJqxwJP*%Jdb#C+4(E26uTK%oybLY9o1^y-rYN@Rjq-DdfE}iGK#1`AnSA* zwj9-}Rf{eWFRmv-K^-7g0WCmhb3dkH6 zPMzF7cAytgl*6Y)M>Z;;r-N_+s2aCH$<$r z`SMP(MF_hjoUGIYjqC zdLl{6+IME~PymxHaWw^hQaT*X%h&^*48Tnc!VSo{TXTE%?%d&U#gwmdg~f$-THd$e zaWLaF@@k0^@jxLN2Chkk8DkD;3v_Z1Qx#4ulM)Ph7|qC_2VIt;HYPq==IJ!B+f^b? zK$b7C#8&b(P%ZC-e@9XjQbV{ zVP!Xl3p0NHxkZD9J-c>q-LS!wabp1@Xf$&87uPjv(D&vWJJhc`f6fdB$7CImO@Np= zE2X0Vr|5UOh=s&20wXZWnS-uK-{dDEvFIpJg$h7$hnd5PqepIR`xh3TSOT6sJ2y9N z+~CeGH$QOmP2e8xBpS$j4A9^~h1J|!o3Q;(KDVB{=Xt#-VSs?l)YP7L+;MY@W)Jr6Ie6d` z#LTYkUAo@VrEQzrJG5u-%B`E$@*2YiK^zj0Rp5XTHtu3IffGWQ6G%ihh+K%{#WOi3 zmPoo3TZjOc5};OldkR_!45MY&r%m~xb&bpKy{B)FYp&a|dk+Z^4ns}axBI&Kbzb=U z-?5bj^;!`5D@DTrS-9Y%-+98rTXMDTfk`*r5My?gocm?V^~ zb_f^y8kHZ#Ov#66rGQRIN(h{tg933DH42a51YQ`2=roYcS$HL=4Go2zNTF2L_?BV; z=)y8#oHFE61t&sgV!0t6ilHcABmqsxjsPKpA+jxAP9QGdikEiAC&BJ%uuv1=84Rmr=CEw`uF9wpah9N+U7PlJ4TS7CbJ^=4)7 z*|KT)8?U`F^rIE~_uc;3qt8D**pq`Ck2EQRUipwe%1;w2X_*dFp7lh~@**Be4;D#~ zE)AeF*j6Y2vf%2IozWdW_lNJjecRo4v-~70E#=-G-TOX#|ElfVmh4IDa_gPQW5-fX ze(>=ptGDf(yKzTSQp)sc(?CXCfdk5-CKiwrxUu=YQ796`N*YrQRtfo>SgpAk8-Q;p zS^P?sz?a|?Fo*&|<{SlF7-aZ;f9uyTp+v&+9eWpV-u>D;LkK>ZdaU<7eV6Y4eev!C z^_w&u{^e(o4Zl>#oeb?|iV-^$UtTo&8u7AZ!G!>;bdZ)^s3t@!Yh{>$2X5qh$)mCt zkp%hFX3GKaM04(2w_@p=PYfD5cEXBv8|Q7`Rl9ag;+G*zAor<({TyXtP=mX0{2{s? z(iH0*&{S0aC>@}u$gbi@Ph?(@6m}>I82p%fQKfs_Zr;4Ldsmi5{x;(mP@RbZEKJA) z85ZgxTr2Puh*bI|It8=HR>lkBQigd%3!@d}cNFC^nBXzS>;mG-u?cl7KupDQD|#lt zR+*Vw+_W&S2=~ISNl#cQpTZf0dn>*oi0$Pgc!3%)P;01&W;=w^RWdRQjy0{r%}@odMt46-()n^kTY)3)dW(8JCqW1NM}NkLEq( z)GeDgTjNW1>vD6Bu0QbL{Zqd9X#3_(by{34=4S$Zffb7Y{mH?@)Wnncl%v;R<2eRS$OyM9Si5oOG-+*y5qIicb45I zB?7<*kRtn9V9SQ}yO%7PFngvG<3Fn%Z!M_Q8�QPXGWw07*naR5WN%ukMw=4tCS+ zuWt9=yZG&6oFGyqrRJ+iZo5yuJfwA!EKBcktlB?zi93uxS%U0w<0i zn>Taj*3BEM)U4U>k;l*y$YdU6WAGvC_Te0!FF*PC>MmW{wP_1n#>7NnLV?~J-t+bF z&tqK9KD~PH+;>D^8=z%u;`Tdr%aX+lu=tLOidnE=Ufp`NdiU-<mo8^qLixNF z{1ix?&Yd#{qra}*Z)m#+zxwnugP(cnc`W3rRITbuO+9_`6srbv)6EJBQ|f`Prwx`gCZ6S~ZRy zN=i#R4PJ=iK*+Ce-?F_%ix$`|0OcE3uWr@2>1Q8&FzC_8=FOUptvLg7EG67PjK!Om zeR9>JU)x;OhINOCr3m>5&-kL6@cw%b^}5TKle1yX%7=UP`fKmrd70U(mM^=jdlxvy zvF)4tcI=RI`b6c*iJLa8L(CH$xQxpI6~s;v2T50pb!}d^a=@bxO&mEqH|-P>r%18N zz{cVCf`T2dxfV>pDv@H7JW}=#ugxJ%7-&kF)2|qp2(l#O$ABZy=ngJ%gV&R`Ra==J^R$=b*pKP zVBCRi+lGJg$v@wHAJsH0mgI0vPHE9AxqaUC%ND-g|F2_5kBp0s8v5Lmeeb$;)QI6F zO5Lc8eY5IkN(ym*NWv5%rDqAJM7E!JEWS1CuyGepMDx+#o)asF<;c zCrFez=&&F>7Bd72bC8K`Vw36+mdJ-h2Qz@L3TmVK67l>@N-8;30(E7ciXR1voy1Ky zkZB)*AB0JC9rP%fATp{R{R4Ks=TuUY9utjEK}Sq{)`5+4XU$u7#9WA8oSt17lO?$b{iNr2D; z0YWd*d+#L(NG}SaG!al#1QC$lRZx2GU3!rsHT2#B1VSp1kV@+5`TlF45J6wP_x*m~ zeeeA)*&I&x*?acvnb~`uwdR?%)H*nz%w>s z5EK(SGC=os*c;ZZGh^mVhs|8)#37E&|7ijAa=l}RubvNV5zxsJ9v1M~sQ0>d=|8Z) z4fut^Ky?&+1FKYao5_@v#6JE$!-jwQ>_yDL_ugHx_t(l5D~$eP)R?cnBsd8u%XLgG z(qe+&Dz%$Y@35Fv?rv+=tO4%gqaL3+u=mvcC)h;3d>(b~^yyYTKMs8yQoy%xuby39 zJiKfaSfk11>8Z}`9q}Zja{gj0$7~etg&akz1`nN$pWXLStA%*x)T?jjk*wp{>Hepk z{~;xr$OBYTD^K}6MIg8STnw zRA!to0fWZG@*yQ{K=)n)hmN}VFrs_6UOig3iI0r|)(`%AtpC8F2`R~gh787WBY%M+ zFb(L;tXvNIp%_)KQ%~pSj%r-_ovIb<);D^3<3v-cbor>LXW+uf&qp5Iu)ckp=KFX5 z!c?F#xd-&2<7uCTNBR}<6{BoIRO^h51GBm!x-VY6w)=-4&ze2=_LOfrYup?Gh<;hQQYE7Gd3kwtA2w{#XCwRd z>2ve)H3Qx*5;_s4g~q4V;QE=mMDve?UAHT&Y@l|5d z%SXUH`y*M249Na=rlEs5KiJ?fui*Ru&CY|OJ-oteo*kF-fJB%mbH6-hUBWRSy}4%rBy46lmrdFuEv^zgxhJ~0~I$BZ6J z=e~UL5~`k8?mTeQQzuUr2MXdDl`W4Qu^8Ub!`K+n8r@Kh8T{gSDXRoXs12O{#36;d zwQb+-^sO83c4$B8i_cG;{0&xww4etW6e^tW`SYiAaRl5BHe*NBA)`2O`*U+q7A(#z9hNS7`z<6;@4 zh!G5`x>Oz-K<$+~AO4U!gA1ib_b*g1|I^4QMkCGI2kbv`^x%;rkDokw`aGJTH);@T6NkNI$zuEW@1HPv(wY^^f1LNdEXu6PWC&8Q zU||xF1*i>hN^Z(*At3nLMWBF&13L7mR=44(G2dt@tFl@co;GNq7z2vv6}4HMio~?qMpzH4!kgRA zp9_qN4DZ~zbK}O1gCE{^(P`6?U)tgmeM^?aq=n|k9&1#dU&q62Lwt15(HGM#@zS2)My4HOMOqo6%#S#Y`0WW&9MoRz?fDo0FFQp_h z1_0YF8pD&Yu<@S`!#cseSc!=X7b7271ZVu@@6dS>DIhK3QwkRG8~eljSzmts!-NSQ z`ExH^vQ(!x9@uw~2;|eoO%TOatudA;ar|^3_W+(I5!Q}5Ixw*MP1E8_vjG{D`?z7QjJ|8yZ-KI^LGERtoXxi;48#c9{ItO<3T9B{PpiY!QmfN<3IfTA5r2D zX@Gl)HRoVX3uLu`GUfB%+ZyzcKX^_uM5UJq>|*#u zXM)O5{Nk{k1^WOaCectWQxX&N|tdcKY&D4j|l@M*IW zHU$55ndelTJ}^ZVdtTfc8H>w;xa!RG^U3#m5GJMUfA99ZEC|#ob)!Tb?K^4Z{I! zAVep{7sg^C&~i9aP78<=xo8bA1{w#BqXvV{-948LPePrJ_}82v46aZ4&Y_+)=F^Go}QD-JB-XH-$CL)&5!BCD=IxnAm#Ek~3=xkyHp;hNE zSU_S*C`~3z3vSZ_ix{Xu6YFI34Z0CZRU8R{J26=Segv1`JM3y#kKEEp!8=^Q8K*Pa z$4doMu}tIFW4>+IxWW5<`W7l(+ChL1SEJt5o%nfjD?n0E3x^X`x)p;_Y zLL6){N|Zh+&NGM*bXAbrflrDUWRngio|crRLy};&!5>*D z$l+iK8(IcmLyN^+uYQ9n)vH;=gkB>D3E;=2WUAJ<<<3Ks5wVY`ER-M9aSmg{4zU3; zQCq1hFbz3`kqK-GRZ}>8SxAlg>I_6_M4}*y5IAJ3b#XJgyAv^nBJJ3^BVS&h4xQhV zr^!f%MV1u-zrn{UP0&IUFl5Y1l&KTrIBKRNrFq*cgpWv5tWErN@#X(*f&nwn*ba22@ zIb)WH9fB8N42qi*diZIxiI0W=4X7f3AQo(QZ{9xp+tE>@zGV21a>prQrBO@~6=;9# z0(MNe;gAi0iPeQNYbD1~tTEaXK7(YW5eH3X{RC~KpnSL+ot_X;WVG=6nXOv1EL**X z$Rtrn#TQ@PHDxwZ7EOleA=ap;F0Vj`lWzcu zQ)hI}-8uJ_lwzvP=F}FsC0wxJIQpid&f?JsTyu~k&bumrEtmY=Mn$9%VPYMcfYJoLlpxowt^_EXq@G*7}Vc z{`QBGIW6*fB4-=O9bYd`R!v_|_@A@yY`3ghWR*Wh(JJv{+_o(=^a)og#Ehf|wM7TlYF@;})~H=Opnba*HEZ|n z+ZX?DrPZB}Ud9|$mQOYiJI&4neE)Lel%R07ftoX+9DQ|(?7TufCwRR&^RNA@fVDKB z^HCuD*c58zFYo&%s=krTiQ3tfdgENqkDM0x_qD*=_)aolmXW>U*@3R=4(R~j0fB3? z!AdLhePZbhK!RpOg= z7BT-&OML1~2x$^)SlWz8ab|I`vsqjuEVOVQWI#5I-%;yy7+5o9D5{@>*r}OjD<*!R zH`KYxW|hEr4k?T>3J`IEt_!211Shd!&IkoV!C_&H@b@xW;KOP=u5%Xrbm8`$BbTq= zKY#J?frB8aa!P0m6m3QhR8@`$?>cdY-O#&s2@~zZYd4s9A^Yi-q zoPCfBWuuUM{+UflnU&o6C+o*QzLCmnXy1LqK!De~(?C-mD>*ikyZ%nvlnR4~a z?OnfN1A;E)&+Fq>sIWPM$vsikfhUnMfHK&v0G2e1^N8~OI%Z`j&VF5-(8eXQary61 z={)rMx03to2mZzTXxZ6!cn!m|kNxEi)yb}@vvHg)!v(Lm#$R5ab0Vh&{!d!qt$Q26 zP+(g$-Au>ct(!jkc))=@yB((Vw7BP;S~b3X?NZ62Mdr_(vTf}O7qw%{`c=cac8!UQ z^2wd&%$YM_{izGm1O7Nh7Eq6md#`n~BYSp~ z^er-F;)Jy4&t%C+^3TE05VE|+9|vi5~C=0kcw@b|N?j(~%|Lt*x{ zZ|=w~6adXm={)jR->={0e@yeb0{({tusweue6kZbv;FfJB0ItVeuq}i{``LRtMWT7 zyMI5PvzOBXf7JqSpT-SB!FLEv%}M|cKi`5|cmINFz3its?t0z2RV(!dmjd1%`}XYZ z)uZQ+-;QTBz)wbuaOjP&A{nVkE2mBUbi#zeg9d>G?^LTg`SZ_54Iipzg&4JC(&ULv zo42&3r>yyY_7~G<^zGXl;=ydS6SxkJH9RB))%%0qJ&O1h$8^N1U=pCfC|)QI2`AER zvD#gIwwixC zcGRYG>CmA==Z3Wb1vfYMUw=I+i{;3aCIpVcwOefm4jfpsXpt&DcF0#>d$_qUQRmc@ z<<76Y8m|=aENsovrWKwfJCz`~6Vnx3&OZL;j(|2tNDL||oACT>&RaMSarSSv*HivY zvOk>SxxB7!JJ0_4$RDnH{il@qwcnKg@5}jG+ML~-7WhAEfwvC0VWuVKv5*6J8`_(< zXaP?GsNBC7loSzJvs76j5s2%oQbmS_VaM0JU0XZWeu`0ZbW{XPbJG?rGT^|qx@vV< zK-Ts&GDM6x9FAJ`>J#$rY2-74nbagssossS*$o;ty?E{-Ml5w2H}bDruXE!@t$K7H zKIYr}h5WFU22fy|A#aFNSfg%ZtIm}v(|gx%+&g>bAh;|+urgo7B@G$*)$wD$&YC_c zIqb2!m#Zlu)u2*82)f^>bsL9~b&ZXMN|tmlTokW&6V|m@>`@;u5raN(k||cLcF@p~ z!-jqu`SAXL{vRZot)ssF26Gx1lB~B3qFnaGcOiATTlU?P%$|~o0P77qW!0){-`_O;)4tgl&%*@PA0_Vx9Rc^-`&IE&1_e3_UQ{T#@mB-D&4QXFGMDiDf z4~V4seS3woq9Fgi?>z5B6Tbhj^Xlq1(r1H|b0-U-BwP7p|9o}r8-L%(=#4Y~yC2m0 ze|N#F)D(|a;lqxzz^~5a{LN{B|F{--qX+-TmF{0qAjMh)V+QU%U7=x&i_$J==Ixo3i=i)RH@whWj{T+eH{k;^7*s*3;PzTSZUSrg<ENYts1jBxHe{NQ?D-}6MXZ65g_wsBI9q)s@cfxW zhcZ)>AKbk|)RbPm-^U?H*(kCCm`Yr3MimYbE#jz+6}Qg6;yWEb9{ly-p|#6b9R2OY_>^>;R`=uLCCRBN zJ=(TuQ@zHKBgcr@G=I^ORFk=D>sB2aH;s);`0=Mj8W%SrW)*M1%i8q|7WW6ZiQG}8R^29@-}`*XkVaLi zWtgpQg$v_Ho?$Uh9Wmm>i4y`npd|1DQU3Dfm~Y0_E?1%d$CT&YKK=R=Vgz_aF-O4% ztysMUjE8Nv?s&=Txca^PDWSutni*2;Qdq?Al#6Pk;C=O$L_S!k|S z)_TXF9t3wLmfFb$uQnwIsiRU_XFgf`dOZPtG38Vlsj84^c2a(F>axIrQ4|0SyA!T;qoeB8npmQT<=K{xBzjOZNw7}oC0PpVa zFZbW3{Qu=i>h?#^Nr?3R2C=1w%V3%%3vxH z7a4;V6KmyQQhWk9mM2dhAO?J~Ijnl43z47ct0>PzL;|4_5)$&}^&*)7RVJ)Cwk)>f zmx+1v<`q_*xKvUSiCbc`*kNnH8Dckq6bd%j(_f|l?gke(@~3D-9!X0}!)?c%xYjDQ zmGvE2GFWX-Pf12#5WGq2u%e?v1Er>=yLxy)jW7o*6uMQ`NFyfHix=^p-mKj&{@w^J zFp0H|S-BCg!|_M>a4H9WayOPC))9J}9f6+g7sLrzU=4{41`JpP8VRc^0HlC0$vMJr zfZmxC*8{XHqR6JuOA^hG^P*fUISUB`c!7qZ@XD(wIT*xshy}~|37EqBqwa&bBw%Sr zFqU>!PG*&jyVGujs+Eji6_A7V$w?+9K`j(fdOo;~g&&F(Zz3^3#Ft!e z%$+ZvnSO;HZqP|E4Xvxr6*CNaHn!NjzvygQcW)nhtjP+V$f~%6E=sqsoS;@&MpvcN zV+@_??c)O>0p2h@NT0;2nk>EYW>!=f>L$5I^P@^`w3u-%6IoSakGS$R6 z@ywT(Vi16Z^(2u;#H)g-cUZA8mwhQGw}uk|f>oFF5CUA(losmLm&K_n|Ss-PTk`n3LjN^+(c z;y=Y}KxL&d?X*9Ey10rLfOF2~<&Po;oJZtn&X&^xf87GoH{PlZKmlE(Q)fEXu2_0G z@GQVcZy}_d*mnr!DBYFyapV9Lmt`puX`46D@|lZ4`*&sn&6YfVwEz;9pO!so=*uy_XW9Rxm)U?upd%2pehr96^I2j zWa(QyAVy?CsW4Nf}iraEXSNX~z zK|V^}61$O8WMwnUfI@kW3<*Sj(0KcI@7}*}FPGEs6wb-D2qG&&h%zvcqFE$!O0Brg z@+@Vb!edf)oP~UKO43kT8de^bR+W~LVoFbE^rHx?teId%3QCj%2_>lrBpy@FmeT@% z-vVzPa65G+(aY4TLwoi-dGtU+KY@03T*gJqLUROOAwB8jMg;-ELGZ{)cS>tYa8;Q+ z15i;Y5AQ$lKobrakP}Ji` zj`rx@jg|xDDGMo)G@@GzVItquRFGVyKDclH>Q$>09EPwW74r*1orBL#HJ0Z904ke2 z6C@@%h}I7gM7(fzxPa;_DLBsfMxHEKuc+%B<_P=j@F)pafS|m1;T-<1R;!s~v1e(Hdwu;VOw^V$|wb293-BN`gaysd_EwX;RxP zgzGlwh#f1OCS;4=U;qTjofsziOxu7~O`{&(F*qz_rXk!FXijV?hg}baV&^s>K2Qxt z8H7O4pa?^igVew-ST7N>I6~AD5Smef-7c%ZpnL*DpbD5nhSW=j5JXJ131uv!0=4MW zuqI>=D1*@GyvP$jJj=Anlg3Y;01Hmm0_ap3+_F$>V)A8DeQqQN zHR`X%j9#;09fKS}H&kpEYJkI5z!42sbY9V|<*6Zr_(o=Mmca_Zj7m#MIK-uVX9*Q& ztVE=v!t#Ts#JNQtA;3swAcC0PDl?IGn~PE8wMb0m7Sga(2yr4vj7+C`GInA`2P6Y^ zr(hfrxy6`41UJEHwKgm38%r{Bq@fi_pekA1uS3u73x50o5hy1Rr4&X4fg})_%`J1b zoEG>eTHvkYxMjFY&`#DN1Rey`G7UzZ#cmBeb1v}YX^YiVt$NMYojbEQ00~o5(st}v z$6{lJ3K#Cx_x;>?3tql*_4D;a7tvk0*ojEsr_;BC`UggB`V=ofXYoDSp@7}WM!Q)UCBZh81^6Re` z&Yp5}^XSpLU(pi989E0)ethESaZI)w)o&0G5MU-iHkiUJs>!l^$;y_^TO2!bxN^0s z4VpF&diY@f-ks?wsa=S0QnMbi;`xggmroqMdHZgKYSr7cX_qTkF7Pep#nY{uHnJ{6 zK%3T-UhLW7c!41ogVkjD`Ntnn>G$m1`8fDdzhOhcmvez<&YV1BHk+$guimliyO#8{ zML+#a8ltHX`|$lwMny+I4?J`F=AE0Bt5$2>x^?bc-T)$vOal_31ys_p2&cuiXV-34 zm2A?qDGk4A!=|!j%RUGS!p^--n>ICT*4(sdQ$$2W*REZwRH=f>eD2)2_{2Dt9}5f& zjE#-WlQ-Y7W5+61s*H~)_Vr{c%CJ+Pbt>9g?dDCJu$Uyz`}gnDP?UD_=FO3jk)6A| zN1jgQ`^b?aXV0E>b#v>}r%#CzC8!-7`i6DuA3eTby<%lqvYGh42u2yDpf!uij%0Br zTe@T+wZph5B_*Y2&z_#1o*OrA1k5RO@#4ivPhBYXlP6DzX~U{%&z?Qov13QaE?xY5 zi)N&!tz5dSOXp4_MvQ1vtxDLCA$}!_Z`iasw~r5b!gwlc^E<*{St^m~`jxT`Z<%*k^nj388uOfyL+6bj?`AXWR2eF* zg;9w~+%Gfd&YiPw=k^K(3T~S_d+qdTMxkr~R;$8U8#6TqZh<_S-0bVkEfT6Z$99+L(Oy5=ygKpSfMt|Mw-Oqcz8+Pl0 zDlOsD4$b4A+;?|mtY0{P%4b80=Q5;)+qoPhc2=VgrVxcld>khO1!u1<^qwYn_vjZE~pE-5< zz3x3x$0S@7_93j0$nv}5s3?4P$FA><7&jU7kldd7!-s$E*>_;27ESi=-PfR2t?SpX zCxnG{?bgH1Gxx?h6T5cpUbAUylPUe@Y18NY{B!FL-GGehRVrP%b~P#aC0X_D(W{t$ zNsFjsik&f>DR>3xu2OaD(PQK{lc>tay?VUcv-`9eQ%zP&((|z2emgn1a`lYV&sS~M}4(={zBB@M$K9PZPO-wm)pbDo|bCJ zG%Z=SV)T@0v`N-mCDU_47sClWc<`WppT1BKcC%0fwQJX%J$ts@Vy%7RFloM@zd)-u zRy}p>;>C;al_-PQK(M0k3Q=S-uiLoA)7_(hr^nijn}$zD1(pd+vNq~vnKZ48d5EE_ zu2ZMptT}TuR!gm;zmj6^k3V6_bN1x%OXmXfr* z*s9FT^5x5hgg$PeGt8Vj7m>qUs*HPNZUYL9mno~|a{lDBz~8jMTlbm>Ck=}#sE0wa z9b=N`5l|J{GIQ+|Q zYr1Lw{)26L^>y>eB_=G&sGMPPWJDOx>FZ&`s}?U>y+kopO45aESL!usUanrFE=?Ma z|Kf|J*cTA7!i4hBSg4mlwOU=jVMD)IvEBm)^tWvZ( znl`Lk+JDXRmFcmuf(L@cKMz;g zO^q8hx2auOG^cjGdW;Vu!b63;rPd-yP#4XY&z)}3s9w!7#g{JoIX&UI9+C+R!n97> zf(3bmct}jQ zh0E3$j8z*o!Tu7lSgL%b*syR}BZwuM8F+UY%90!MrcjZBVNv0%t0_?wczq=FF5+H^ zswk7JIa^K({NomQ>wue{Lx&U(W*iOCxLHWS-Gg?D3=e0YE$SJ(v)a=;H*PH<;gL*X zVF8AB3rgMEl`Fy?KR$Bw$jxh4ef>+Jj!EBUO3evUgd2nrP;_&oLY#yvk=ePBM~_R@ zs)Hdyi&kyupMj^(?Ap2Wy&m0gi4^NUibEH7IEohaWq2>Zr)z8 za&_{{m*0FmX!zhE_ktfyoia&z7cxr}^NV;I$yz%od8sdxlA|KfYyl@SW4O*YcnRi; zO3y?kK82a8u&^+M9T*z*G!kLqo!18tBLRv!s%vUO!q|ZWh7bGX&XY$oW=^Fx%sSGH z+>UHTWfj|TKu}Y%q(6+a6A5W~T#)2O*gYNdG}WL?p=6e#6Hn#IezN2!KdJflc!;^WrZWV zBRV>|Xi-0B`@rTS(q7d9-Ci+a;atv^(*l3n0&g8~W1falvoIRCi7phvLpkL|6&Duo-JwCK~ttKV$?xk}H(Gkxg!hC#k`Q*vHclR$* zkAvyzC}V64foWth^^nCC4Vb7P8P+db0W99RaDMyd4ei^u6>!q&d-v}3&6qFq6fRh; zP8|y~V`^2A5~bh}GE62H5BGZQ+U(rFCpA6IY_;q^bP(e;W(cocydzMx*P+tJfl*N5>~506t~QmQGBHU%Pg-nB!&I zn>B3^cH{iXU-y{epD+G?mZDOk0n0F*7U6?=iB}2VNB3^3%ozoW6f0V_)|!=zBOct1 zi+r+j@xsDYsun0#4B9BSfAP@!cT_3KNeS^}Qp(>y?nV6CHR~vijQXXJ+|6Ocng!Pz znW$8kD_8DGXgK9ZHZa8suv#!m!L9{^a&h<;pqDa`1D6Bn8YOTFk4cCTX<^|<#0+11 z>&mrtE0@ds0CK`97?`<`8B2YJVuJ&SYc)plmrF!N*J@qd>%ZHPIA5u$$t+=U?6=d% zzq*YZqpUCcah^Rf;lQ>nVRvp&Xnar)?b?0v_;JOs8v7qDZlcmQ+zs`cxl1ddVx+Vf zqKFNnz&Jr1V+{H7_!TQ0^6h7~Kt^+uz+efOci ze`!Sg)$`{b-M@#Bv0KCrQl5a*_K=;NEvE(kwguig;6_oxgBmv)I-XUdHRHcxaQkWD zlC*S7_m(X?G^(GFl~gHTu7zT3sOMCrbry49&zWzq807^4;QFVVbr&))C# zXjrR(FX3vzPmQ*6wdzeeb^3D9;6_!eWLT^oMf@x{`-mML*~(_gRDUpF;IcXM8a8eY z)iCd;MX4F~F0ER(tKTpwDS5%-r3@?Z^6dNJhl^&;ZPB7t@sj>6d-m$rrE9ZBjSH11 zBPKRFBUF^o#A-|=fjEJ3N?V~y<I^pIjcBhdV-KN(K_H}o!RpXjiNrz$ zRV*hbKy2p&l#79i$T{c^MkpAxB)o`$X+UE|ZlQJ~04P|xsNkfu^Y}~xu0-xtr}K1h zE7j=-423G=E$kjc-HHUhS5`}tzEjfex*v7wbX6bj$;m+ZQ=YMX0dGqG2T)7g7Lz#J|2p(Zla<-fn_$OL` zVWtcM|64ce_J>#{7MsnWD9Dh-7=MV3v#5h&KR|F)#;Qz3;_@Q8Cv1R5lVM9wN=z

Os4s8|p4GB-ga9&|8?W|O3heE-|nSroJE*u8&cQ>GT z-oFyq|Gkb!ZrtIb72U!#?H7(Q4XfPUxMuOQhmLu@W zEcTF!)(%JH^ts0V83#``&h!cUTEN}IgSdlY^CA!g0VL+lh%`1#-BgzJcCA{B`(|>hRxMb^Q@9Zgd^Cy0 zM4DW1a4`ygV{R@AGj{RtPF$;)abx!a5@YmYg`a8Lw(a!UbE?*?rP%a9LExHX!+KiU z2XN;CkwG#g$`_AVVXNgPGzm;a-rPP0qYKd#fq0|J%=BhPx|K!dYz_<0DCK2J%c?bh zd;Cc6>dO7e)ndW*l;R?592v}IFtdTlI|C*Piw>?3;tUx0@iIO(cPi(xgfMrd$6X-4<+(NK?for4`KUE`wT}RhSV3=0E~fW=2|CNN}*bn=A30__(^c z(hYI4PKb|#uy7$Hw_fiC2`+>E%(xe^tZ0)fj}Oc#kcmEqK?B;P3&`~{@yV0NdA+=t z;sb*Ko=iN;!8N_Xl#!N_oE#S$|Ki#6iQ~t1?AfbHtF|x$PPAf{%>Z?(*Tuhx(dmpy ziHVfYWHG0vW|%Te=yWbnysoYmOM395V8Gp|!_8iUA12U@WC6`sOwyWRdi3a#&S3QN z^u~Of+u~YdV-gF6pUBVyMnC!x)`g;wQPqWIKt-?VGn#%wX;hy>t*wEr}+JgUlUL(sEwh$f{6)nFCam5 zdKbzfVYDcy-4+@W(y~P}&Wm)RAuuyL= zPmt8z(=$`6XGB74a3gGrh~CAduTF0uKni(2jdzIuy*X1RFJ7}!L;`h} zppx7{rpjebL4cm7`VZbHt!!B+&XZ*%#8A~@GL^A=sober z#g0`f{V;O|%H5^F6AerGec0vQ78S}K+_MuWN0dJB6sZ9}di~U~#{NYHbO~rvvDBWO z8x3gBiY^E|LBGr$H!`3~nT|EzY3S!WcGzHRa=d7Wd|;u4v|zTAxF_t|4({1Lq+hRA zWlHScu!hj8U_9DBsAqJTe%-OYUDa}XwyY(@9q~z)|1`f#ht>f#E8n|y5o$#uiz@>T zpba8w#e%t%d<)k3dbqHg3PvzOR7%FxNN_UE~+%a{DHbK6GmlzIH{4lK6llnP)e4XA*& zf@1>G0_eOGcr*dl5}*bt&Qrq72z{NBm#kPW@ZMbyue^)}P}pTonrxjj!@`mol{OJy7{9!cUI4uvw7dXyR0SFtl9L5<3S=F-54E>RRwGogWB{7lqkw3e&oZ3d^~2_q-#O<4T{|{yma_niLX?AhOzy#=P^ky^f*Loh?_a4b zbOaGds3WiiddEBqpEO~N*24u+#qFs0ojZ3v{GcE9)vj8Fp9OHyKSgD9+`Dyi_3Rm2 z4(z#m=T_rpO(uUg4tqg(6;Mg}(&6^mwnz8CySGCbHwIoz1)&OMWM&Q-^;LeqQW+XU z=_=J!d429bd;ozkf6>yeeclIFX)*#*Fp5`c^*=6I*11^W4WNvjI))Janrx(yie_~M0>q1m)4IGybxebU$$DSKspGYgDTqc;>7sJ^j7zz04Y8haMjg z==AEvK#)$a7w;4t^*P(IS8vV+bnStfclq2U114>Z@D+511ku37O)~)AX<(=pkF|;#Hgo_iN@vt*6P#A;gTs-D1TpZB54`7Gq5&A!K5z=N>^SH5N zKORbXDqv3u+k;i$f>owTWig^L+4 zlSJXB;-cXbJH{2_Nbluct$zKY6)VKVJV#k?+p1+Qqv6=W1IR6*;CY%NY$-kpHMJ3n zK(cW1AkxlCh!rQqOIM@9S<16uF+xP}pNz9a1}T7WF$Pkek$lTl@;Gv~oEG?JTHvh% zZXpN+WH1L{kWCnFm|FwUNK}alwd>SXV(-xlL2a?ECTNl9ScHrM(Mr; zR067=Mn;w?T>&q8htB9*660=DT=X-cBb3C>XAt!?;?#+gtkfPA64Ikb4?#_`m7WiX z+V0%EQKvxz_!kSNC!h_ArCPIU-I@Mx5CKMOBdzP!N|3u1%b-6LPg;w)w+6OR_; zhx%D$RQ}2pb#87{#LdGikH3F>%nJZrR6B0L?bT``qWpI1g}W4KM~@7hzlk zbf_%og9TqycvOYTl}&23Nuw=<>9k6f@FJF$6bT_HD0=6!r%z9uJ~L^`Dfo!B_=ZP^4&a@=ADtmp}8X%8h7hiUU1w#+P6unP*yPgn9S*`}d^UJM`+ofk74ajC9{m(!46s7c zOhr?K=TuNwf|xQXiQ*@EG$bZr{Irh5#CZRbWo01mu)DdkzOdC|!e#?g2D&0HCAc7F z-n@A&Wa5~_h^R^pYT3-#r-5@a6Ge(lTgKx@54zXvD~4#oz7pzBefH$Zq=@h~9onN) zCMCvsl49O}fA+dC6e-vSCZrr@QDU*~(tH#NY28G-K0&iZs z_9T3Ka(HCL+O;uYXWCAA8i)!FiZ?4c6QxUH63CRDhy@`S99Bc7Iw2}z&bQxg+qYk> z7i$bFQB$xRVV;4Qi2P-gforJ(tO#5Qq^r$LOo}N}%3qJ&WTuL+P^4v+mCCV@1&EA6 z5%Z3BuU(lQ_B1Ix;+=+dGR!6h7HIAz{7diNvrm>r5(!6JEAN0JpLlxYpftFG)bbel ziAPd4q>C7HZZcV|$Tr%ReL4Q>lpI&~MStgw5?F`2abB16oznt;*8;!4YJZpNzv!Hx zrGO(a0@s*nN`5)~lR>!Ejh`?9!vdIN=|jpU>Mx+doErG6#3mI80S$y~1$BjrWjOBY z;hvh2ijv3B+mw+`XLQTufuR65;)C%VZh!&rb~qatc<#=v!UYSB95F&h-yn<3lG1TC zAt50Y0Cz!DO;H9N=B-;c1q>MI>X{epC8S?^x``eRNO4U@hRMS-mxvOk&9K08%VkP8 z%TQgM{WLhlc;)s+nM5*4^d&KT(6h4pi4!Nz1qJyQDKO^CFK~%OgG9X*S8-Wl2Gvyz zrKR4Ax=YRi>sIT;ao?7~YvxJl!9$0!qI-D%9_t&RJqyCpi0J8DBv%RhOyLnc-kx6R zX{idz*)e6IcHZ9Jk|9ve&4f|O4Cv6|#HrJPK@SQQF7V}-U&;tTz!B;Qi$*RL0s{qC z%&2J|E|b=l?b~tVN|h@|5Wd5Q564753%+;%&h2{?L@el?$&`jw=qV^5h7ndTFVD2J z6vehgmHLvvFDlR6&hqf^1><=gItB!sJ$>f0Iuzw=qe@hbRWvqAUC?J(RCE=5g-q*W${Iqdgkf(88q*F^;sH$R=$1>@@sS@0}h zlV;80ABUtTB||qrXpp;B=UTkDKa2;E<%Fka;UU+4J=mjff3wO+z`i0yN<6!J-(<2t zO_(z>VjhK*Dutg5ECT$1ra-}>(YJ$asb+)2k&&Df7aCm5zbJ7q)Gl}@!k`0u=)w9- ztpfHLa5G(@=#Y;%UmDOa_YVs8-lHb?w*l#BZshX4+ zfA8+?hYy0lLxC-33NXV1f}|lBY=Gv_9lAnAiiJLWq_i0V36)wuL-?KzZ^NDu(qU zPncN%s>}1?(VAE!$jhTy>y`;2A+hmsFhb!`;U)cxAw!s}ASY+Rq!yxn3-a=j8Yw?T z2Uk8qjF6Q7iueb|88r}BU8#vE)N)qYn4o;U?~#BbIaVdFnkVP)KWKrsjw-=R5OZCY z7&ND)eLA4OIW2YCj9JmKi8fu5cbNIE{sP zr>eyGglEsA-1GRDY}VsPk97@b*S=kcLCZQ_xpbjo)r!lOENt7OkE^?93aEh2O>e|+ zBU6>0k`^BI?9s#El`B@)Y0)}wZm$fyH=gXVt+Ra-xuV3%?;Q-1; z8DRDyC|Jl?f_z&nDM>FA;$vN0bIqPJ2Pg!Z;j;U5dC;L_rygCquslpm^b0(U0H=hc z#F&^EpWJx}_i^CZiB26lwQmerFqwh)Nl8he zVG$1>-e0>t~?5PjE}x3@4}cP zWMzDl-I;h;pm6r~`-xYFc(g*20Q$^s@*app5p{rLa-j$S`Oa3r`Q-$6<;1I8a{lJD zz+be$TL;`ivdTbQ9T^t>@Z1GB<{nKO;+DhEdfZQo-|N(=Nv&!soA^sl88Hy0a^B|M zO&d1oQiXtU%x^iyeg26`Yy5uW7K}Q69679B^|O_#SN(9zn4!IT0Kyd-Hyk=;W->N! z4F6F>1s9l?)1EtceBgx4?n~m1F8fcKT+7UnUyM++p%Sn`t#3O(>*=I zymj;TZ>G%@#a>p0v>9^cnzL-x*Fy#_n=!|$P{BD{Hkl1>w=Z9t_t~hbRjZaPQG&op zUkx6zXyz(;MdKRiPY?%Jwz zGuSmI8=5&7mPt?%%{QM9yZhS_m05fJ)P>y}H+CC2{PV9z32JNHu+uYX^QNhCnYBbu zwRWlLuP_*fF~h%6>Br35ut~3WoilR8J5?)HsaL)Kn9+lJ_Tqgj*SP7Z@1~^a+?=ct z3^!M?ZqV67d(R%(uPx}?wd)7{J{UmC{kwJ%s)ca^LAe;KpdA}+j&9W}3;5|Y-+nfd zr!#Nk)&|X*o5j!7VRqP)U2-L9jaDjSvvjLj5wJ%w{_3+YRl4!>)~t0gxXvBbYW2MxsKCzC*1cC|rYhS$!nwM20i+oA2H!(Aji1#q$=;MsW z@whX%x$6nd0qI~inXM+zTpsCqHwXtYv=e$<2FFa>fsDeh2yY=>8U>f>EwPB!s9kJU zOx)TuYdUGllzL5Dfdk4SXQYuh8}xdp2`1XeFgfN$KA*hFdZj!zTY7rBi<<|97v6># z-0=x{^A{p4twhkXF!c&|f%wQYW~M7dF89^2^np}RrE1=?*|fQ{Yt^l17J6ANmW$%4 z2%BqjWgRiKCLNZ=NOV0`3Ze$Ak_Itj#lYEwN~~qYHM4k?NLW%$2LAlw&}2-+In$H5 zS=`p`R#S%AVsp>sX=8#B4<8gv(J!}aXyZJDe%!MMbRF}TD$p(PbQThFkC8emY!jHOs9G9Iuer>47m zc)_h_+S2Vdh6HBUTwcV66{l83iI(!<>u66Gv_(qlv}Vi=WlF<%^w6P2-_Jd8=#b6L z)y&w5g3Dw!;~^xm*!1x#Ai0?yV1xv4V~D~m2-61G8HqB^$VsdinX?7TQMhSCL=Zwo zf`XA=XcmEaX%kM$JHeNN*dk8;12<=>l!qwquWZhX~3Kq-&G-O(nb4E80SY5$6JTUp` zS+EcxsmYYdG-gem@|er5T36Czxrmz$wtqUU5#N>ujapQ#R)=K~VQ*!YQRrbt>X-@Y zQ*qw|yz~ZEQlzTc^OnG=tmL4Wlo0h#EaVh1BtQwA+3g;v%$V_II*7QWaq}d0fXFRS z-DWmOrKmn~iZzFf)YqDc9f-&k9tZWU)1ZF!n$?Kwf}g$EN>XlS2%)X159|svrIHUQ zEfd9<1&L?8(?Wwrq*0Nb;GRa4&eH^_#al=3>g7o`W+C6*9BNlq`UGgC`Q_PBGnsr~ z;+Q*OW@MJ9%!9-ZP@A5bF?rgI1@phR>J2my)U&`5l9Qq;LnR&vSZpB_xbsC~K^{s5 zku8YMOoU3$kip_yY!kVSISt}P>|F?g$kRjeQ!Q>xdDsN0IV3xZR96nliJUE`1^&7P z-b(Q(YZriQC`OKJH?Ds*=IeEvw}>hP%*lkI_>|E52mmce#RgPrR3>E*jD<7_z%`V+ zB}mW602v=nS;3)K$U-Us zMf_F>sG`9d52X|VK;bEjZo=fL#Q9)i6hz?-;4q+$PY)=E?uwrct$`(?Gm@fA!oJ#Q zG-5fC1E>!E*=0gfh! zg5|X4lxZ{YO}%iD>Oe(M7vXPcLf8ZB(@kJ10>l86immS|RlilxOQ z<{~Nai{g=C3Mr;FG$6u{>T!_rOQp$(!rSb(Z{7H5{sIvgsOfs6B+GQTxVfG@eYRHJ z`WRp03IszYjmCqZ+>!&x&!gs{IZ0_=4TXH8D%^-w6$~nig)oAO(9OhbJchvl`aa@{ z+EIRF2kkxB>z>UPMJbxqH_D6sc6Z_Uv7;d=;UK@HQ0f9@}amSpZBlY=9{1EnppF z#Xp4`8K z8Ks3dwxSvW?TQ3qSp&ins%!^VEsuKU>XiGEGIgn&=3NhQZ?+znI5I$ zqRk3-f%dMnwMJ)lE>SEup^1Thp_Kt$jPV3{p;XYVH5%fXJPQl`Wy=QQ$`J1$A;D%zhhK99b^M=*ZHeZNR0I zr(BIL9CslgEQO@z;vprok@7|<)sZSWq1?Hb^PSTIf7=3YA5{V`36o&UBwo^~GpE$v zxrntb>&n>7My(!OV)~%~C0JlJ&z(M{xL@H)<;oIYo4${-Mc8{n_qiJlD!Unr5Nr#z za06OJQ3WiS$`ia%R5SQx)&tP0%_b|9I+%e;8&iQi4pdaQSF4GTAZjReG+R(AZ@%0Y zLmq-AP!@*>#lLmS-h+oMz^GnK!bt@)LLd}>KflTqE5Jrim@w9yZV{MvI3%PY8#a7W zY%Jjn7%+mk+>6kxHj7mcMGWkjGj!OhWAzASEIPGtAz0YVoc^7kuiqCHCm<-$_PLh! zaWm@7rVJEONuyTHoHE|OM5z&F${C@p?KWTv_jlw4wlS9UbV5K;0?cpFDFH?FSy6LU zs(Ewf^yu5iub3}C0dA^hwOENVE7AK%W5%Lf4BBB;#*ZBXDw@ToNxY(@5veq!Gq#M< zb_f~{BN4=?UfR&k6BObDZVIx4!~DY}29#$;5tL={gheADDV&5Dc8DJ=8PfU?JvNJp z8v*{}dr4lX{d#UhD@lkK3PdU7uN^-ijhV$pFX7Dqvy_VW1)+c&I!8X97@s`KUbDUiqOPBDMOLEV*;rwn}CFA3qG*x@j#Qg$0Wg1s1^%>F~D9 zP3zYIltl~VbJeJqFIq%A)&f4>I-7Ozg83y%l(d^|SI-A-UAsQ~!Ht3ieB8aTV9Qvt zWWIk%-&@zN?b*I9_})#zWqRh$o!iGpFu-ovyJ_R;Uk}B+hPl2<6@pZjeb_KVg(PQ?n&_d&1;wK+rB9}GNOE$(nc4<_D$1&b8$a&w`ks}?Ql)VF`Wf(6Bx4+2E33JDJS z@$0XatX@OB-18UC-MoD1d1Tm@)ytzJ!mCuQc5?&@4bSJDiebhFbW^7x2oEy)Iz@oD|n9x(k=Axr1gK-f! zGT4_1F>He!*RNdIzk4U-NNiN(ia9gC9z7Z(^tG#2e$>C;s1YN4ymGVB*o?{J0i@KV zq>imx-MW0Sc;Ui7&YHD-^Clt6(GW9PB^~0J;Rf0$-rW+Y|VF)KIkyvibojvQ9omBo6fMtsUcjf{+(KXob+ zOh!Novmr{hvWf1j&fKi2tm2pAY)Zc>WvM7t}!1Xeb*sV|Bb7#&K zEa5xyv(e1Jp@JGP&9fq1q&m~){!p=E#R3KL&HMJ-B|8q(s#T+~Z{F#XCVuBIZ``=b z-Ceh4+creS7u3e)4!RzY?E}9BtL>7R-IWR`V7=uiFTD z(6LMRE{z(WK6$!NulLXWc8rx0hmRVKX~d)%GYOt&Pf6F>)bm#_nfl!%lduyKESr)* zH$b&BvHRrFeEJNG4EYMdAv6z!45WwX-E39r)nln08oBzZ>hNdtep=wr*pHn0?ec|F{>6Gt zn=%#B%wfr}X4;mnUOQ>(cLR_a2lg*t`2C)}`^U_f$_yKR9sdt|=K&s7weJ1NB-7FZq|*}!B{b>1cR>)5UKJ@86jVe- z5kbY?5fG#o1(n`=@1YY)2qYmP36P%Y`F`)3BqHaYd(ZPd&+(pXvK@!Xp0(FrYwcP8 z_g(+>78J7N!~|5%PMy0jd$EgP#}M5URP^ZM|SBN|3jer3``9#zKubY=a34FhXt zJ$f}XsQ4zbO(?OJL`y?pNUn1UtcqPYbTO$FVT3(wDtB#-=sM(ysa!RJZMy~y9|Pzk zA|f;-#Jo~870_wbZTXz=zi$Hf%fg%G^ts0&b+yrv~<=Al7`V0w~K*cba9k) z(eORI(vp*0#4)l)bbd@W)WH*M)U-L};jwGftXZvDGor-cb6K@|R7z43jP=E{ zXV$G)9$mRgohp&L*KIg*q(C7|bxP5=?*wVZv{B zG;7&P=x>8p^;*#_TepSM2417<)=f!DgcLilcVFAKZ6B*%^@jy>9cfsPu$QZwF=NJ( z@4gKS4eHmgA9b^55)>MG>((tBySm~8AU{00Bkd?XN=M-TcLYkD#*LoF?OVTf`#sxtAKS6_tFM3P*6s0KTX&w>dwBMzU!hq~ ze(B}r&6>`+bkUiW+P-N^8_RWeC1G$C&mav$y>xdMO;(r@JbFX{gpI&AMNGw&*&!{L zv*3&vpUHm)g8>==fIy)m8=Ep6*H(|Jm3-^w@}*0O+KL4X*gUUHvqG2 zn@W4w@@3ahA|F0yI_qj&x4bu!Fh<-#MRzZiZn>ii~X2uA7#vb!UvWrvbY z5i$!KFt}v2xsZ_{_`w@vo}K>rr`?|HK`fVP<6eSxC(r2d!-qV}gT%n>tj0@=_f(DxtDsXAsLKC}`ZuL&br1H|EbOiqU5h!uL60ikSkwff!naWrQ zIEc5`yl3x~KmU@KZ^q?sCa#{kAer3xf+`O5YnTeSMiZPOd*^MPbU{J`lt5-=fi>0vW zJFv=^A!>ARz9SPEE(oU2mT|fB5Kelb7e!>(?(_x=3NJ12W83 zG*nYSxsc0OuG-CJr^VjAU;h=We?EF}U(Wrz-+uiK8h`VatpS|ikg$uFu3)bX3r`m# zszwrwY2lK^d`94iMINa#ZDt@96B;lGB(4@2b@BWKRw~d+#2#UYI0+0>CrcT+EOy+e z`C=*!ipv-x1LVym_y-ZT9a!2vK0sXuI}#>g=iG&8k;Rxu5av zoOxMUnZtW@?cS*VrcE2rjO+#O#1etaKzSr~pPNC45j6qc@G#-PYovYW&VW?A9zF8i z-5a%Pjb7OK@h3>=jmA6irB{CV{Ofknb@H)w<03bFBQeO2vDv(H1pp-$_m*eg;Zmfrrsd4Im4x0lQH`NyPv&jxg{rX|zUe zS}VHFlg~UiW&GHd)gz|8`Pzf~>AA-H*RHYSNUZ?K`|O zWKh@obyokf=F6o&2LuKKUfp{4IeqF>o0?GuqptIS!6W+(?AEjyumx7~0+Hdk5ms~@ zQ^3<%E! z3riIQRcmlsHRn#8-oJAvmLZ56rF}yvc~}eRsm>yJv5)d@OL1(~SfRV%R4dFheNf zflk5Pzz*S(%8T1!e1d(Vc;#q3Do0jtKWNaX{sX!P7#Ub6)*(W0E5k>@Zf-31EQ+VwJEZ1;gv1*W(!t%m^hd69_kID|AF!TAIIq8F#&j=p|xeO$zqh ztbB`wh^**eMvsE@v{a+Fx87i4b_3C9c~}8sL*ml%7$m#~xk@+GHK#Ndi8t=%X0hV&kp~LdZANW@1y^pH#n(RoIf0^z?XioQ zK<_oe_+zu3pY7-8mqY3Y>d4N@&B^j97tCHFo;WC(P|E~t$94=$)^YNn`WmDoj>{P^V;?55|f?SD3~YW ziY3Gin++@x6c;3N5yw&BoG(NOaYv^)H^-A76OR{nB%xdoAtG=1J$P!R%iA~q{`ps5 zZP>a4){S20=I49z?9{T1T9ho~q5j9~Px7;+wd%cag-{x+#5EfKJMSfqnhh)|9yq ziGfPYvK_MscwRTWOPG*MctDkU!DL*rQHX&JfJGo!>n3POh$vhrg*%o=7a)oi*K2%s z^w2M&eE9~IgLB2`P^cKRQ?4cNx|azLcOgWC6~-Jw@B*GiPz%T)G=@wszC{n#6XysP zfptA?)_Mha5mbcppdQFA_$PrvfLPQ?%nywwJu#$|o20wWw~Vh(fH+t(8d?Y$je%C# zwcdWjXqBpUUSzjnl_5Zhw4$}}P7H-;j_Q|;tMbhs0>wwtcnQ zb+|q4!K9DTB+Qr?D$4=CF5C-Ssq%-45)Q$_?U1hqFJC^P-BVtt*Fj-P0VJqimrL|8 zhi&=?vp)UeOAS#3cwmH0@$uo8BT`?9TzLj^6pootOoia6PM5xwj=(<|0j7;hnEsRR z|Bs)hZ;xo)qUH!v0D-cmhD?o(yF%WUu)t8JxA;aYPFbc9dK04~8+{yv(G$}K(dm>6 zAX+nM%cN%fMC}%|6pjQRKJI{-F&`|s`RtJy{5`2AE$JRAZMAw(3O(Qu zfOdFbVcykbX6Y|`fmXJ~#N+}zf&{?RnpL}KXlxfU$;EJ7i5{>YzQ%erF0OoqN?tzR z^aZiPjc5t&WK95+!3)D@D60%U({=~yF;`}0hh6m0dB;GhRd<1 zMrNfXC+FwqSFI8$mjLWyY9eYk?;sMrgSSAlror@vz==fowxgKS0s@C9tz62`U#r8M zms7^y&xRkSf<-`C$Q03=K}ivGAgHVia1EC`D1Kako={XCL?)pg5Kr%c<38k&5@5&% z^?Ze@aY|HM1SX@$pAzLzgqA!@d32H)Lm{;2(`@&mG4q`eI#NKym3#)oWh7<9kpW70 z;)cl?wiVdEEf9kl9zEqL#7Yj;TVe#V0)tMG<$#-8UIyzkdI8Q$ya6KM(Ch*?@voP@ zikKCyNItW$`cDOJ)l_M#9ED;Aimo&|pld|_`giq1{_Ai1zjsHOCo<>7S0Xk3YL6E6 zp6%7~sRlJ>zwwIRmWR&F<`YxV29xPtVE0hzRb(IeV69M!#bLF#R{0y7#XpeGSXh?hqk z4koX`4O#;XV_q<~<$i>Rm2_or+tj8?;Mm?> zyH%^S^OqGSv_N4+JN6@xw}OWv9*8U@T|jjPw-^n) z#z90D`6epJzW?gzf&ClT9NwzI$R2I&_tK3p#EPjQ6Pnm|Ks>k?=p~PugiK*#n+dIC zARadntGFH^BM}n;W&TXLdA(=j+WE(jpFB8YDPQwGR9Wc_-w#1X^-9szp?JqW^$TKKp{^i7u=V(1m15 zya+b&`lpBMA*%O(TINq*6k_F{&i#*n{A2w*5oxTfGhBxYKwT{@KB>w)TDwxKPGVYSoQy z-gNJdEz!+dpkfN+tZ|$?wEup3im$eU%EaLC5r*Bw7{v`xDX9Xoe%2v^~D zWZ$m+d-s92kN4?Yqi$VASxQJIq6yCaaMtJFd`-%j4I4J}@891LP!CZObydUOv?bvbqmaRGYd6Xo(?ur%5Sj#^(a8OXCNU~ODBq#6QzUAE6)77d) z3>rJxsMlHuu!82z`arfG?MIFt(B$Ow=|8}#(>*b0&}Y-%#6=UgLr7cHTch56|BlV{ z*J$wEyMG^k*nyLbKa(KmxqE2$d zBBXX97t$5f|1W=_ARTR^0C46)J;1Q=s-g?n1t|K+zw*#se~J=bd=z6S9~6sST%kx2 zE^^V(@A1cB}`t#65s@jK9yLbeOuaYyamK2>TEU(%l zEi64s-UvMMtNQy7fDjPikkgrTCjnhCGBT1KK|B?sqN465Cy8f`Iw*7U;QpB}j7P8Z z@$vqA%Im`hKD}h&!ZHE=Q^$;6^UKNtyKT+tRg=a(7akZ0-o(enVaC83A0-j(6~xy( zNUQT$Lw2dI#m3@{&Ke&>;L24alM@pqzBFrT_OQ5{&A#~4Pe%}{=goOKPsTM1SH?&9=AJE(k3B60XeLEtm2KJBmvs9>3^@*&j@O^%WnJ*Yd@S#*7#?`Q;bb;#@d)-Urj)B}q*}+_mAIJD4*bgogy4 zI(Zb7_9+t(SS$MCrSq`pj06b9fVf`Vkv|I`hU>FJs6XrA45@uvnAo|kj3Hat}DAJ51dju-cGy&f)H`nhxj z9vOiWr*X3hB_xNvfUQDwMQ?9^Y-*WXo?brj@$qoBY@?wJLkCM35HHibm8%RIXG+qo z0~8XiF4)5>#^gzL->24NMoW{U+Xfz+q{w%6?-GJcG z&)#}t-s)c(G;1DGzT!vkP4oEp!<*M)8TcVX28C9tKQ19I`8yT1CNGxAO=L7L85RNm8Jp`U~GtzhNV`^CF_Evn|-LNfQvOa^ z{r!Chj2rvjq*q6d9C`ZmX`CZzE~dxIua6=aePG?@CFF_11R9okKQpjw5D2gQD4E#= z&DK5Eu(7*`kt|OTv28Xwjl)o7S(rF@+_93xC4G!V(j2BU%Ka=?!(LFI)&-gi(bP@nMBp2CxX@ z3cr5nA6LoZ@~Y$?Jj_D{|MbySo?HKS2j9A0st`37BfIM=+Rx!?TX6&Upk4}U)eVZy zxGwrv--|2!S1%|n_ct8@*U#*4+PVL#TNr795g=o5a46t%<7zD1aVRe_=VPi?uO`OG zlJ-y{0g=#3Z}1i%!Nx*@!Rm3N#&#G$6w3ykDZFCUwOh8o`Qb;4zW;vy7oS1&3Un$X zv(UiMgq!gwx;XKai;TP;dsT}opUraP+BI;Wc8S7>^#~ZOu-?OHAea#^kV`NAfnvcR z${BQu)BW`+)7I@j9N_8sY_IOwSr0G-=8(c_qZLmg(jX+CJ&$X?j<6c$-1|3gViZV> zBy3UudCDRjB&G-Mh{_}xr*zK1-eAiiIgvH)U3&N36mzLl-vKiwys&@k4oohD1M$G1 zf-A)c1kJr%WK`UhtN8V>YkDR2LiOlcR`x|PyztQD7lVe(NHI)|p>&EaVihNvvup++ z{Io{Xl2xm}T>10GOR>+59O3DvF%=Y~-$@P)3Z=Vr9Iz8p4{0j^C$%s_e3F*}K0))M z7oUs&EPl@FB&OQ(qNEg1m#e`p`m?YG3b*NGhRZXA%%XC+h1krvUWI3*Yy86GCG{6IgKvuu*L~%Ts-Hhy2S4OTWJd8N zr62zGBT({yThv2*`kcW*!L?hpo-=R0-JE}D-`=>>XL|H`62m_rRg_?2Yd{x>H--Qo zRwhd85n{YGkY|QCJKm7#gqw0dEEDl<=Y@_k^Uc7h_F~k?no9kVva=pe)Sv}BH99|@r z6ZgP1Um3>1NB|aMc@6EZjM=DX(}vC5?78R08OjC4#l_vbd-s!Bv!K`E1}I3gusQ70 z@98B!Eh32`vGG=|T7|(3R(e?aF+Aw(vf5xJKw2F9!s#9eC437RYzWoFs>Or7%2TJ# zv~1pDK>w#x&R=wpn9*vz91~NcMhzG5;esJRP~jj@?z*#pql~HoEY}Cu=R&wtbJX=a zc3tawckN8NO1i)fSS&go`lwF1it+K``ilw|{_FaRA~JFUntxPMe7UQr`nLG$!YNh^ zxW!)-o+*NR&Qp)8Qoj0gd8y=)yA;<``k@4lK#AkHh47WI*Z5r6-=F=-sA0oeRgcv8 zc)$70_rVpaSj7twQ(Pf$*;iD7I004<*^$#j!b%b>K%6NA^c_C1>$mUb0BhwRYw-Su zAKBQdgJ2eoTpJh?t_cbxxJS!Yt%eSN=5)+?f@4s!@0c;Y`t%_sfoRO&Gf|>o)dgu3 zh7vjz=RFnK0|z{$2QHM^+1YQs`MM@OUE|}`|HYRZ)vNpN^cfyTFJKba8XO6seo)ds z|6$IgapUHF^r3rj@S;VFB_4=43W;C1hy*c(mc}6t2-9a4S8vg4F*`vG#Heuj{J9^D zAEEK}Y|yk#|Dl8T?LBbh;GQYdCgTuDc1CQaVep5HdiFxh*-lLwW5D>tn6bV346tj+ zOn_^W$QkiY!f=@sUMyJvad9Q3ew0nHNa8dh58fK{9O=n?Dpj5K&G#C20O!UUz zfBWgj9|)*lpffVD^Km7J5DCp zh>LwVnQ99h>Gx87%LEX`3HG^T$Ic&p`dOWN^>8W|wT}o@0vQ(e46zdrghR)F4?k>{ z2wIj5BtZtSh`$8*!bRgcX$A=$1hgWv4^UvU>Tv#XI&!n`lSp0T<;OigeDA$-L4l)R zc%FK2Y{aS^e4$%9hczqn0j)6k_}R6%0t&DKq_QxVwC<$PyukM6D*Y7>g7o$Yp$iX~qJEO8lQNv8QH9p2@qb z|5QTDrIXx3gpKP*B<=e59xjkj1HjGIy-H)1{CkJ|zpq_>PbKOg*R5m`!1oPJM@PVG zjaOip&BFwvfL97ZJjj3}aKQ978qGxeL$j5X-uS^81-slli04O4n7ulW! zeBDh1$VRh5HAPh<;5Ge|LjUm}Qhoo%n@w z3qs`u1ebTJc!4t4QRiT>fnHH(b>eHSAPf~7iXI?f@{D_ms5c% z!)6B0OgXfxLE)jG^dcJ&=K{9kr7w0hT*4#fz&Qzl!bg!4@=16q@Ky>*^Tl%=$fmil zUNQnWYZdx2fz?u>06G`)L}dKYxaH@?r6MO-5uk%hxN_;QUJW~XhEQs>Tl{vB4733c z)Iw96zqQgOy)Hfe*$9+4jT>zUJ%wq6DrU7x#&N5hYbRpSEK&Nv9S#@sKz)Myq9BUe zXyXAOLl_{71IrjXHFx0;gyn18q?v4p3hV+~(if-Il9`d=&mJ<^C!~u6GC_KPlwc>l zqQM}e(^h~%eYdVX{LA>!5lA5pKwSn+G++6X`ywM13lFqQJNcF_b6~?YG~yYu~xFM~gu#AhVa=~!Yf_0QjPns81Vn;9)Hba|DRg%7kAN#2$d=nD(9Gw8mqX2OBg1hfcZ-F zOd&1f68)P$X4p4-7?!@AJD-i`ja~u8xRbjwvY%yw?e{Fjtj-Y z4@K6^K)(cEP;JRUlb)2gddU(ab(!s~ErbmK+8y@e`*v!r`3i6xW}>5jYU<#&sKDSO zoW_Ym2b{JXpqLi3#{g=}f&<$eJ+_4&n}7X1{^CW6FeQ4Z8NYdMX}5r;EVG#&%5vFhWEah0 zqX=_RW<5xO`bPYq0t@smsxaZa^*|qL5ys#$jg{sNE|PbVwK>BnK{+(Drcc|v?pM4k zQCIm2_yofsl|gXW1pvoDjz(p~P4IpVQDYqz60W#AtsVt7BnRyQGod*3cmol}T7x(# zxOwIB%Yz4zX^h$+2Sf&-hyj&A02(6i)DOS`io^*KCPsis`spB>BKTqVkf10ul4dy2 z+qr~ISmXr&0(?oAVJTRgTWva>yOl%|AeDotDNY(eiZdd&P(7T30!&egziy*^`D0u;=AUH?b@v-fh0c(;ZVt=m&vX8AegEX*mcb zt);06H{;f=S~dBVNfI%JVHZKk?ynFZ0+V(N@4^xn*nzpvCQ6Otx+e z{b}yE>(>7K;9gqYS~Y>rpI5GO%gNlkZPUhIR+5V#EU4_Vg+Fds`*To$Ur0!h0z3PS ztCyDiIA{H@KPTUci>@AFFnX_8vhcw6P3b9jQc~~Nu36jDv4yL9oVc|RoHj*F~R(bvleSd-%6=Vi;5E?N+G z^=i!;kz{kHRoFo;o%7Y!b?ZUiy}S1`Yt=lmMifj3x4`%^-)hmL5jdPD4(z{p=2X(H zn~Q&%mzt6sU8CCWjq6t~USzZ8MO3MXm_&xf96S8msuhP0?j!I(NKoLZ6Gzwoys~k< zI&7-M#>TE#zVflUb*ojW@ZLMqpBXcb1<_|8e^4&4Twq9u(zwMA6G0(M2x|Mwj>3D% zDgmtkq02%fH_vJ{o1?=0$jT~<7FpVr9{=_cC~+FMD4{I8m4f&4@+A|-jNZKAHzGru zb2Gbl>2&hMkt!9!zWC&WrHdDWn9Tp@Mhv+bdm$$+W#Z65&kh-U>FjA+ZuY1rdM4e9 zM|C}Q{K(Kw9nPOQSvfRt{+w?KDgq5HewXCFL5C8PGv}!j$4B(&>c}^fkLJaJ1MtBC zB@rGZbAVkHmDuTgZq#slP8I~q_<{Ye#a^;m%^$z~a$asOTuQ>NggNic@I%KI-T|f9 zW<(Wt*vCHe?9v4bA}dzhx^exqsZ-JE-hKP6@4or6Mzv}wcM|U3OH&GX7s3~3x23kwepq?>!wbbf)bh(e|zk(A@=+n{C7t6?0)x7GV6wK zzWm~yabr#H?#Z{~KYnMLD8s@f$W9unH+d3-P(oKlOD)jk<>Yi~-E#e^RaL`74({IB zr)Be$+wp*%4&p%6e^K-ACMS;_F~Xjg<7UYl-TU!7canr12K5yDD04m78`&^8b^Jur z=H$ABO*Sw3 zzmI9b+#f!iHG>REJd$Tej$HKPyr_tX9a}cN{rVJ?=#2FAzMVR3-LNh^sNAd>@8l-k z7GoP#JMXX`WNL8-KP{wOMhChdf8&OQm4H7T7GVhm&w?mUVU@Q9V~@xcI&>q zd+*%2!`6$8zwFEMt0~pE|LyqrAA18-hmv)${RaL36lmP&VWE{OEZVRMU17-=vjYvf z#S4~VL=ooWyJ7vh=SGZRe$;8%dip1yJIvXS?K|Ss_VoiZ{=bpnyjvTIBDeU{t zWfW_g?~WZUpU%8`;bf~uO@|K{P+)qN z=+$uTcDNS@M6%9+C(PXkn=Zx?f$G$;Jr{Ou*?iy_smFVCXxFoAckAb$gj6hl?A%$C zR+Du%am}h#M-Oc4)xFo^Pd=RT-pnV*jDy?9I`QCvebCGx6?W)AVOZrV1BVaax@_JH zFHH9I_WSzV?|{X%`%XEHenTct>)NRP@v!R@=ZUy1kj^t_XreB$7)eglUFRfsrz_NSad`9D1tX z(=$Jq_2%SBP$k)^Nt0fE1$SP*0AFoD!0j9H^_n*Qb;lk?iP3?Y0kK7`EsShls^OL_ z1atMa!~wUMe25jW>|?Po@6@q_Fv8k`ix)4Z-M-bZT6ML@FPR!iHHp65q**g|19f`O zS`8j++^iK$3(P~U=-Nq1cVLMx#+-kB+BD1@s3bf*+&44`UrLl*=v%Q5^0cho^9L7QxvsV7jxnAgqJ42J$}NTgU2Sm`g-?X zz1h~(y6gE~ZCbiVHfreS?@Pa~q4K+FdbDpRMh|SYxoKl!Vye}MdVki8T^l!=;BoWK z!F%nxS1zjNDG_EQUT@`#OPv*c41PefG+rwl#rOifN<5gr~I z<}!@qrcIkR#a=$DdQGsaMT-`ksZ_BF8-c`f5#W|>R{&FNMcTA$%Xtu#Nb6@#pF`Oe zyBDfsd7#Fe!m=H!)z+>P4dYWy#fPa=uK|se&$(nYvt7J$nPKV55V60>PNl{z=G=Ks zcI?>6$O(;GrScUFp}{O8u-Q|l z9Xq$t)9lR5FFtyI=Z0UsOa@JE?!B}$c>NyTy8p6r#b@u&n6r9~M21mEL`1}$l;rvf zocV#URAljj2;$(k#fj1lO4mS@wD%GR+%gLbjqLc{>wUcg*d}5di^!`<35g3w~e zFUmYRt|9?6^%^v)Q>*6a7bY^Hg8KKYzYq%hphs zeFqL6)~)mLBl|Lv6MH`12S*J&cK{vI9iUfMt5yxku;JKIy}=VL55Vy9^V1njjaqhG zxp?t=6UO^{dN%IZMRv`Y^O!wowB>?A?0EP%-KtfOfJol3f1kHc8NqZCAqIGV@WHIO zYuC4I-BBqlxNXNyKyz?NAoemTcW;M8*R!}9;Y3A|3ca&Q`JknGG1OL7u&9f>V=BJ`6dL~>DZm2G3TFZ4Rb7N1G6?ZMQC1(0}h z!kkmY@#DO?*RI6w*tsh-uv~}Eoe*h^A;R*^@e_u?;63~H*KgUD1|#xr-MSSR5{#Iq zCk(v=1QuHv`X@g*q@wDX^riGDQ6s=_xnv#u)P?for!NU%!C=X~&3ta(xl8={OGl3$ zHJMD;<8Pn6bj8g6p(st*wc))&7Ao;YWWkL_i76Jq!K+uFek)h3KD>W_PR4`pf0%>O zUyBwk(cOYV!cfpil+K*&*tz@0ZM$zJCYU@u4<9?3bT?g8at^E-@D683$HU`?@8{%X zXa4^C@3f#@hxXhts#fjrI`v+d@O1nPrp~TOkhw@$Zs1qr^Vf*g+Bg%^;@-`x8TR*q}ws4Pp$fSHOmM{CX(Gy zrx+-@YXZua0Y5IsUdgqZTeokQdMS4Gij}t9yq#OOUpjQCbI+aty}!R7vgUH^HA}vk zojQ`lbNse$LwbB1@VIz_}`+*V`8W?8hk2MCQ6sY{{l?;UPA9eM9ci zg2YloPVy4dZi&YNuBp%}z#vk_!(wA6)&2LizwY0&7ZhZ~F`Z;RAfOME&i=G42e6qG z3wqkX7$PL-S-pl0H9r2!R{lb}7#Wb%!)R*VzQd(cF?)AzPftt!{PWK#DOWok-+ujN z+>PtXXGa7Y&9H*f(nxl3#s7&)ln9qGqg6CGIr(n6;2^(%GV3>QNxgAP)+KJ5x8Ir8 zwqxfX7cKsI{(_j8b3Ck^oE+=jv?|pi7^02qH>BQ4;RnM2xXOz_QsSp%P2kersu3u0 zz|E9qQo*QVlxNda)W}UvzJ@?kLW75kFSi3tha z`t%F08ZqkeUahM~{x zbI{OXwv4=|`VSGG6s(pk&g~psW9X2luU)&$F5XY` zzwg;7x=o|{wHr0+H+UG+TXZtCG!O#}m&hk5Y#vUX&)c7Vwc@9r+DAtX8rbjZjmvPg z!$yxid;U_x$ZCt0Ee(sVtMJr){Nz_pojB2_S@Y*dKij=`U#qD5=#0vtQB@rh9MI;8 zCr0&q>aBO*>C(O1m^Y_>`0A9_QMKM2Gw%6!XLabxlX+y{RzB&ldtMFYu56qC#OC$y31n?EjZJMhO&-G zyL)%pXP*a_4M0|)Rzug5sNN#fjN}eR^=BOcHz&KgMB{QNXuJIUMm#h6?1c+WYSvt| zVpT}(x>}uX_1ZPZwr+=Zt68sJx1mE{e)(mq)q4EIDUZ;QvVrBW+xYN>7mgl2EYe2~ zQdx)yW1qPc$4cf1uuT0sOKtl0NURc+P)JCo52_2nBUskr*2jt#)rnY4POCXPGb=DC z6tgsZ@IV3dGsZ~5N-^VwcVM~Aauo|OVTfTO*lNqq%plNqnX*BsRD>E4%@MSBS~GJp z{R7J4Q$pqD%yd@D?w&r%&yk3F=uHw;!-%c1(*CouGV~q>qn8gM2~pGxZnimJeY1Jf zw)LC0!PgM|3MWIS!(zFg<>~DMeu+7jJ@5X5RR6NU9v(V@3^#4&gZo~d-Z+?}kkVx7 zlZT{~#4yli#UL3kd9j-ojnJN(asOUGU=R^HBrFOB+ICxBR*uQvgsvz`o$v}|>vg-A znidokB!9YTaoI(M7d^O_-lIXC1J|zUj6OUte(Zp$)@kND zf(P&byc|}@5DTjjjso^{wN{o`$W_YPtoQHV4-5)H25|6DC}GdZp$(qiWI!e$f%}ea zJC`k4vTW%Jt0=-mfDtX2;X{THu>gMEV9(=OnBhX$h$z57UPmBYemU6e=VfL3`T5Z* z`lc)vd0Ko!b9kBvNL#ON9bcL_u}kkK`H5I_b3Hvh@gHRKJ|@RpMD))F@g%v98vo4Xwex-h5$mM zf@_Id;pOK~J0;YhyBJzjEyL3jQ zFuW3jNf=)dX2jt#dgJJh=8wv+_w-@olnfO_uR;v4+X+j--GOf&34a1#Wy+Mx&dJ@i zb617=7-PD26lLQ98322(SRg(FtjH&*Z%;1~R7g6?$nu3CwK{j5k=${FlmXD= zZ{2R#s8OXVRl#T;1QjBR_(xJ$hy>xFgIa>g$UY`$PWSMhR3`j-qmkbX$g7?p3k1Eo zk?=qKu6|y%I{x;}H{N;+Z02emSBdaIR7o0X`IT{ntR4SCSGZ1sa!W6j zK#NQBv*%lKva|UKdwQ8XjRpu!cBd$V?;Z0~w!JvCw)FJW0RIsDxv`dryAg{uUQl2N z7z%EJHIye{9k6V%X5YARqe7(!Pm?zYGJo#e#b3=H|I)-!&%I!EXdc{8L%TG1`ItPt zgpZdQi}jrBgIOImKvO6XHJc!aZ^z&A^70KN6DEe=5?~2;PH7K1a$bI3{LSlCDwHPy z8|T3b+9gZk?LkD28h`l~ z4@Ll%Ty?^$2%iEWgC#XGSbWA21@P1=n3vp@-qLefX35hNXp}C=lM*x)vBCf$Tftw6 zPeN_dAWj0}3Z)q>ZcW9Ew!}Z+r@)Sgf?6IBSuQX(C9W9-utrr4mKz~JNz||{aJ-ly6+iF!u_wCiWdZle^S2G*OjGJ)q(v6|x zUO>@W|J$lw4QutNSFK~U3gZU!iM@DQk2QzbqRZ|m`ZAihB5MoM6K;2J({gB!E`1wD z&wBecjZF|%`a>WNH_eg#`#VNfdZu^R4mGN5`eilqA0-rQ66kYSx2#|Lc>P-4>ecKP zQGV>ezOfh1&_b%E2oHrJ;Sa$Vr!_U{R;Si2hCbfy@yF`CKV_oRE^){O??e@}-%X6~ z)S}4{hO}Yb*;8L7G%k8MBn<=&@@d`gYkJhH+Ot8m_SMUe9s2au^Ct}j_7C5A^Se(z z79V}le_byS9(x-tkpTcq7i!ZCP?RwP4H-L8InmPv8yPTxV8KW1On^#tmM^dcLPm04 zlyurJB!hsaNGxL5u@uG<8KL79{!yGaQ!dj8(bQ?fsHKR0O zD(3=)uq>@ybxP%YR6DR_c;-AJNJ&{GW?W;-&A6|~BxxkQpyk5f>GV>+>W+-R$Wa%G zMT=x8qBK-s z4AT}jpRiEw^6S>EmYmG5KK&R|YkWJ=bcxH#)DTP|Qc{72PSvYZ`}DbUE7q@Hw`|$z zlgE^mjle&UIODx{>(#GwIwt1l^}qkV=%=%%j&paKDr~Hq1VSkr794Zw(%Q`%8MH4y z{utf(^~sZ0FIjXuE)GE?I;{AV!lg@46WK;khscGZ&xNP{yMN?YEpV#aFzezhf{^FR?R%>r$1cJ}e@TSh+f ztcTIFTJ7kl7A<#d--W#}svfJ}!v_y)EY>Fn49xQ|^z8RE>)#WHj-UbwehIfhB_ywL zczF9p)v4d4apMPflTq6oHuB8K{*fjh9dGo)HhmB__nw0v^HFuWD4S$C@=u zPDx`WZDGGoO=Tizpg;n7x%bkKZ`m~B+0n$ju2#ElRFmdAcI;5VPJT(ZWBYe$^0Imj z7?kH>>OEjEs^QTihai;^L>P*=vxDix@Z9O`1l zs!m$>Y&UnKuWz}Ci1fRuvNx%G0Ksu|)%=|N1BVV?xNzaOHEXB7{T7o}86(K#WRElE z%&Atb+W@h}np!8U&p{V~6);^j4Ke|v4l`#>Z_2{pqvBWYNhqc5kXrGW5}Q8h4zfkEdkDJd!3 z0USc%y_=9wp%&fo#%`UxLEoy8_tR3bG1nVBY-Y)S23(qRa}ORoc;fu!Rllrx`<=Jg zgoC9;od@0(r7nB3X?IfSOOCDPCA3S{g+=Vm0t@&-rJMSXrSX(q}(elH0-;uy3YIm333xf}C zsm%iLk;8?Od=Q{4IyecCxe64Wd-Pw0orNchul;YXa@8iki9ZhNkP}?)LRb0hf@?wA zVhDD9;N5j~@p+27(B1V_;dwFBR3-l?S6uraw=Vpou)FZ{pZfdHFZqW$N>BWKMxew2 zH!OzCcBdvU8}QM2dy{pHY33xRPkv6eWDdg>#6yQSwzoGa-sLMeaW7w=tlS(>Pz8q) zlQNV@_AaNtJvHOzO~0UEFHawIV2Xpq7>&RM$6mQGwCQl*Ej{A_EGV0Ga@J`x`FP_F zhZ-j;vzz8YMuy4DQ;ZTtx7GOiduL^5a8oOE2t*1yXlVCY*)!gqem5@OGbqRl;b13$ z4DgKdr6bb{4T00RcP|ZG(R+FV&1|+Aygl>o0W~L`z*Qm5u&S~ATLgNg8 zj`0Ix0F;E@(PU@6{nnc)@d;j`(#+Q2@rEk+v%og#%-%{dpuxnS|X#_M84mM6zG(tTCF z2y^A>7lECaWHuqYcC+}KsS>rSklfNEEHk@pW>tJ>qb+V%0__rj_;ReUe5f8McI(3D`hG-;BEvT0R_g)#O8j5RbgPJTz0V4>M=dIa-JVjzyn_G{v4)b3WQN z;sJ&{CpW81*|O|^0;FIrNpFEMpRfw>9;dEdeg4H4-W>nj{gh;($eFV04fF^8ynMXT z3Nx~DU@f@RnxE(E>yIxWnj%$udV5*0gun@#w%M&&8TZSUEh~s5`-*Ir3D@Bj9=2=G z&QsUo#!eVN<=HX!(o+iv#;t?Y!75UeO$?>I5M~;pYve*|P_f;TS0*r!%{0~@bjR1n zhi@^2XaB%S4AZ>a&`=2^LSwZgU=!PFcT-TgJ^OZ_xElN7#FwUy9g})DneRfv!)RqX zivWql!rr3Y)aT!X6~3b?K< zK2Hs@5&!=i7covv$(D=)gN zunYW2fq(2saSf#({@x={{7d?Kck_?lfpYSPvmiJ)7{I%B{i+owb~op_b1~JcL_qtZ z>S4Pa5*C_t?mV0cWy!PtAntnA%9ZecF$^JdH}hv0ke& z+U&7s&Q_0%f~Lko21C8D@Cxzg&xyq}8Enkv%($CXD^`MFAqoeuis=c$OI-QcT{U=h z>Dq&0m(N`w-X4g~Rx8o8WnJ&)3<}3+(iwj%4gr8$(dC$!@)6ZA(tsV35{Y#uC5ZvV6pTTqFP9jT9&$XSlgjPEII$rwgm3v?B<~-G`cuKKs*Re} zEtEmK;G@_DQnNBmDQpr2kX@WCf?bj3#h(Rvm4+_E_7K9l8YAstO(DSlrx)kBKlfr0 zLla!xEbiYQ%~hYI+DA^6{#jy1pu_<;+Mg?|L`ZOO-BzvU&Yg>9z4yTWYbQ?j?b9be zC->JiYt2?mt2S*k-ku8=%tsYnxNxCYg|Nm=8Ui|OKf+%#r_h!`z#H+mjvYTWYu5Yh zy49;w7lUXqX(K+hRudT+iD@=Udb!}>TCLlD_uY4Pi~aDveK(FDd!kows@=M2^WBuY zZCbU&6nxQwg-*L;(U0@JD}*|%L~l^9Zd%IS z*&lv{^#wCsSeAkUH0U~Q+kgAj*XEquBm4JUKXIaW?>^XjZ{D;q_-jC9q zdGqG^RjAnLv4+fiRv5xz!#)={Nd9{>;r5YZ$3OVsBYywUwQ8iNrF}T-{ha(fNz|?o z25gOP*Y4YIzqXn45A4}{?ZolNdp&`qBW@IQc5rBz8qy!~a@f9%xhSs%_u$kne`2ma*x&A75vD`U-#>!nOR$*oB11g2ew3&PZ&>K6;3 zORdG~t&}P*?7D>aKOv?%`|z3KVvl?f)Gj*Z!dcf}oOj)#=w0>+U7w|ZV6J*E{_yZg z^`*jP^{GhLrZcV?D7}5?DC&3Nsp1P=?fK)T4_#9Fu|$o4>t|Dfx{N^^ZZ>3Q%q!5m z`N`~yH?DSwjC|wSac|C9=u@c%!TCOYd169Bg3jdg$=oI1eZRC_rSPAZF8+4m%3PhV z1wH_WK>V}a-1Bh}e~>kxZJY5!2J8HT=55$frhIue29+m<-s2{Z*5_cENZBHF73@Y! zEks`Fu_(wKyooD{OZ}9MS-FsdZ_$WFDOT#U>)myxNK@s6 zq!@BX6)b#Lcag>viHAqNjArgr~u9ckak_VkYQx>Kidzsh;7HIWs;0c*45alEdF_~GO z=-TP^sc*FI)SZ1s52xk+y>wr{GVTVv(z)1~lXcI}$BR@7T;{ap=4NIF1_t5bC~>b8 z{mra|DJGc<#^K_2ExVF-vw?`&&X3-CJ1{tS9Tu}*Yr^}(N_ZxX^TT(h zhJ}QV9Qz`2jz7f#Q9^AM7#z0rw7Y&pEA-G=B$%nwl9Po4DYa1qgBdJv{K^D~Ket{7 zwJtdu#WqDUxO)@y6$=v~-h~W-L`uAq)UQjoU$$=#uTUPv91UH9iimZO{4oE{zs~Ko8SY*q^e_X5a4i2(lO5$?T!z)t#Xwh>;C&FXf!(?&uu)6C> z2SAJvs?>*@Au`~C2`WO+ zaHlp`G7=1#*EI8&N3V=?@rz=wpDqCBivy!n( z>m@w6Y?{I^YlFh8cJ9%KEm8_0!?*`x&u(LsbY6i$cD+F;A^Jv44=*BH5m5wrKx{lW zzrYZUjXavKAinY|L4YphKCV=fVvo4C@IwhdRmIx z|K9muTKoUe5h!^$>uT$xz`i!~ZtI6wk6i5|^% zqXaDihEN_sU8#iPTr=l~IVj^HAq4s48oqI}Su7vSm|nX^WUboK{hoeW@8!b^7fo{K zjvXhCpQu&4Hpq_Vjup0Y*Cyo^Xh!%8?oK2k?46a;g0LuX{`lSZpjWvH;bLVX@kmfn zX_xk~MoqkZiE}Mmf(oBS7Xs`6L|RJQ&}va#=_u{yA}NJ>D?OrsI0^AJW#lVP@fR4c zb!ue@Ry$PBGltk(-&Fa4SB*-y5Sz?5~AJ;P%89^+21J zZipCQb5^Jh&dY_gj1L6gkl~d>Jbm1CM7ENxT=eSp9ojc-*4!OGA2Cdnz#|w4a&Sk+ zSjhg;<3D5s3LizG`i=1Ahh3U1_LPh=D0o5}nj~>{&9k*|23oHi=WL@F6k70@jN#DhY8!ki4DpV{D;GFOsdp2*n za_%(zGj`BUY&Af0fQ&#ysKt^QtpMLf!lDw730M~M5+o2G#8!k*N}@{3wxTH1pgvtF zz#b8BR6>VDY4GP7y&x<9$tGn+cE*N9^K&!OrE7GAx(Lky>Py=SzWwUU&UI^!=+mQR zjmkTJU&HEyyHG!05qFS5WD&#i86y)sZp6NV%2DDOaS)tVquq-uzl1QQPCh{9P)=_! zdKe&@3fLW$u|h&{YZcqC6t(|XK%w-e|M?@pv?^H#svzp7!>TbE4Q@`G2Tpycs-pVZ z^jK08ARn+2)Z-)^L4n3%A$};auq6^FxQf3y8ab3QWCN*qWpfLJl~~v=UwM>UEKg8Q zA+>Q_;Vax@By=psWF*d~d&fRG?(gN|+y!R3yf^8*(^p$Kwy#j!C!|BBN#mxFI)$a<(wo) zBMu0aW1|fLLIY_Mm}>AS&~qDXs0lHPU57GOffoUeIe_ApEn9CsdgRR6QzJ$WpZ)&( zR478m?$GNcp%0j=cqD;cv}gW7#K`_C;LcareY-eo5mjmjj_hu9fikp_R$vLPlx_(R z3pb^(yHwkjp)Wm3N8leupu~Ajf+L~>3WN!G$r4_|Lb)H?zwf}F-T4+rgC;Gy_JuwM z${j>HUAtoOm8;i6%2yaVY`CXyz`2;S#}Dnto2Ezao^|Wibx7Q0Jk0fk@Mc*rE*W?j zQYT7pPUe#DzrK9^=3~v9Kl#*CLRA3!V4K}(@-W;_P5pVr(wlMDNHP3Gzy9Hss|yw5 zaGW`IZ148Xd3olht=jeI-W>rzWuSEO?f8vr*LLaDnT(O!wrwNDT0ogHLq?AbC|gb; zo)~@{+P60+?T(L^=bl4{rca+iB$O4)7M;5gQ@L`LVI!XL^7S)Yt>@03Ik0ypJ~&+- zf4o+m`b3~cn?HBz=#EXBDBrwUGk9#;cqcx7{!dFLy!wi#uRp>IO_}}G22Gl2Z29a& zMn%=kxieqss)HS`iu_aE3AA7l^1}oB(}gJ1Gng z-%tVM2~P(ZN*RUYa8oc01 z=6v_kkimF?vwQabi!aQ4XBx3ct=Sn*bnE=X5A({GFCP~dN9d1~zD= zdvs4rNb7tv^m7jd}*|^tV7iAyD!{uD6V48gbyGr*U>ktM*4tZB`bg4n$ znvGgX<{$tq@_={J9k@oubqVi$i_(o2&+rz1Rr;ZH1WHGscmzt^#6YPd6*vhLP>T6L zb}pwmXWq;i6W*OMdh|2yT0`vyjb}}s@cP6t>wozr>*lqc*YA)rit&OUA=1Z@3+Et( zAEczLS+(rw-hF+Zej11Ym>do(W+Z?(8*~EVT1}59`@Qt$3}^u_f8Q-zHVqg$T;V%N z2d5=-I_U_uX zZ&z^TYMnZDc(QY6bN)x+VZlewoi%#6rzI!;ylTmjT|4^pAN0dFUp4OD4f$kuGlW*J z`{_$#aJH;bulAO`doUuTCxY7YdvWAuJ(x1-<(wPW%2ln#f6(7WfB@;BHs8t`MBYWDNf#(y2>0@(QoThWl^&%dP&xwtFajk` z<3{fjjhLmoPNH1{JqfpNuvTx`y0unkG}|1_S~Q23O}rI%@$8vSJ-V5^$yEqMK~}qE zWj>fX<+X;<(On@i^78Mer-D;Z+hDQ8^bijfQBF}Y0iKRsx{~pW#N>?|H@?E*<@DT3beLDMYQsNKa ze&cSGoHY|CzOsJVvd$ekuK(?K53;)0V8BH`r6iSCsQ;L6Kmg}rZ^Qw$9ow{uu2SX6 zE?q&wa~IBMXJx$o=3DinYIg70OOunEk(vtX$HtuP(4iBq{_q`5TehU-Ah386(F`$c z5$8Y=IT{UalY92;IdbL7kRe0I^m`hUa0HcnL8le8tIvgR(DN9L!2Od{2%{BVpl~S= zKRiUGpG!yJ|9b>V9B{i#;#_9VsESUf)z{YxfPx6IvWuwK-A+sZyvmg;XE2!JZzbUF z$37wSE&Fvp&iO7eF7CH&TTjHql&w?|h(iqrvkdSRVuXx+G<%%jl&oViA7qFMX}4d$ zbg6p%`ed^edlkt%?1ryO!p%4XuJ@RKxVgpOhzkl0CUpR6XDFYMZ~72g&_1zUeEOe|BT45oN&VpgsmiQUPr zLq|8BKezS5`7P%!*KOGJ$2s$FT)DPk-=Slt&IVPEL>zHdg=$eLiK$2&+IRi(c`Vq; zH-=;*SO}s+2xz4a!+fwg>PDVKAD=-_4aA)C)QQs!pxS3F{P}_6#V_`DW#1Faci~!O zml6`39w7?;;Z?{p{eS<6OD`-Pfs!@?B@Vbj5n24Bj!K*;+_4RTAz@+B|Bt=*0I#Cz z+P_aPX{68+dheakYv>4w6zL)&HUvbmE25$TDgx4bC-hzfq=epUNJ2?S0_i~d>F0dE zH3xltp7(jK@4c?KsDCoXaL#0A&)$1x?zQ&4TD=AH=HH8peUNl->C(kz8a6H!R-t|8 zu6JYNwr|`}@Z`zWi{~;k(x`VH6ckdjWa;fYc05W<1eIunOx-S=$Ob2>mAno0|)lqI(N2f_ikm%RS2zFXTidy zcVgmlvNF$|K80Lh5m2*E!$m9Bu9>@V@#2LzpX}OqfM|H~5$c0ONczFzz{>@T!4I`t zsXvJaOoPUaRQ~?!wrn;T^qG$y9XfhUV=$+sXMpTMLB)2-AWwq;06+jqL_t*U-j$K~ zP`m+k#?D>3Z`-y#=4NzKLc)@Hb7j4t(-!3C?%uxLX0tP0VlWilTeW!MqS&}w+1c4k zSFTd|2guUIyD3pTUftWPp7Wa%SqJcU?qobW}s*W{<|~q%)Lnf5M#!x z4^Gyh(%IE!o8EW!{KdAy!eQMb`n7HTC?oBsxr?3pfQaY%jeh@=A3u1vW3B3MzxhT^ zcHZ!juU@)#qjlA48`iH6t5?qs8j*#X;^pMmlY;8PTHpu)JQGV8Tr#v*^YGXD_9v1ZjtZ@u>VXCHMWRi0Y+^THL6bMi;@?AD`E z-GhhrJ9Qo_p_qC@-IlGUEM30z>o2!$Shr}&;;zlYn>K3LsA-cI2M-ZP47@h@ANipo z5EPWfEl_1J7^cjazI?@s&NZsN)URLSy}J-P!$*yby&c=8dZqR2R+q2az^OIawWeMJ zUL=R@h@Rd1wQ19;NfTg}#$vBu{bBsu=_wDqbHxgRy~mFn7~Zj6$9i=y-@G|<@j~AK zGqLdu`Hvfy_HKgK_`8U=r7Q@TDzO0WKoP$Ni~IH;f4s>+WKYpnGy_F5@L!(+dF}t} zNB`}PWNiPGs|0f!%&Z=>B))67Z8)ULBoJd#J9Bd$LqHb`DsCZ)h2Rs(4XvX5OmPr9Ebz6n^1U{S}!z*D?jg%j#3V0U{S;>?1Fe4dOS2p5g9sG zkduuYxY2BOXw29KMIXkm5ms7piv}Sn&PxQQ#$$sC%1_TEbqN_oq_}~@N)!17c@$F6 zc&K_#Fg;X_$LMzCKYk2+vl?(n0HY*>&B-tDEf$C{VoahwBkNo?M?pR-481SOPw0}k zDmqB-DoK%KKK=;5TnKI)4qXTmt%g=xpB%a=2@8YT` zSIM7w*901#sSV#VmhiY=3Gf_j)CQ-__kEn0B(!Z|^5Xa@1S&~kv2 zk~eC#u~?AS$KRnb5hV|uj`a`lO^CfUwUb|PLIZ@ zG5Hw-OIS3FXgF~NrN{>;*+lp&IICb4q%cktl67A&6vfw*o|$>$RxGZ8Rx)y^!CVgJQcxKv z(DD`CTuI5Kd6@iQ>w+b?d!ze;xu~gbBOx%ZeS7!DMcQ2+d*$)IvJ&L$VJ6 zbdqV~)~Nh7H6?l5qQy`#fHA38Fa}B*ES-+rtz@wk>Y0^*Mq_hIiFi6jdJ$3nuaZE92>{zCtbb!zX}u|pabQTRut{NDn$x6Pv8MKe$|1OILYo;^QquZpLU6n)Ur zMC0fIOss$;8XD?aaqetHs8cmj5`}2+8=)GF6*3qWf#7&?$Rp{r z+Adzz5EMl8xMxrPxP1BYjcb04ueg{%DIe^ygLzKK5+RIT>yrdP=fC$=7Ni2e^df zQ|VFVBWk%;^3MvnAwmF%v3}#WLx&S0+vVc*y86t5&a_@%9^iBHC*L zg2WLMcN0!gz@4uC=k0gWTs%Q(ixFSMO)Df{OL3B%mMfBGtT0j3JKM!R*H^`vpZa` zjvoCeD>HJ#x}@afCe2$695@hs+q7Xr%_oA0#EKX3rk+%4ljo#>U2O*&6AzySjJl z)vQ%3kRC<^EOy}_!lqWoC*0k%b!T4Aqb}W^Ytgm?pbSXCy#uLEhwbdCGe?gcE_6C^ z3GdmbH-#wW(jM2&ty`{LzJeWczzh9QjZv!c4sY~8p2+Kro) ztJNIy<{RJ^VTxckP`hFEsyel5UA}q)q4mluuRhMqTC;XpdPYja#!Uwd9PDtaEmrHf zGf}6bj+F~5->X-j5@pJ9376Hfdd;f1xVT33>p5&zACe`i)o0F}xpd|7*s)`!s-yt3 z`^2de6nuMe&=9;7`@cAF&L^LoJ#)HM_Z~P(da;v1dvWmh+wuDcY2u&3(TmzLQ-3^+ z&)y@&44~lqKc2`EMX=iozJHJrg6aIvpZ;ARe!oi5hoTwyYtFzQp69=22mj}L#iCw# z0~F6)8`ix0>dU*fY;{^Kg^wQh?%L(jg^QKSSD5qNcN>;2HhP>}HmrGb#IQS8uR5P( zzdLN$%P;niy>Z!Q$$zVF@8pDg2Dc;n{E1;5+FUt*p=_zr%YU9dvUd+;000LcLs4@q zzm%xFdG6GxF6|u_i;qb^aZtbHm>Wi{eR!V$+FERik{5i`%Oh0Ig4BDr-W~k>?U)-Z z0&w9b=EtnktXZ~f;_yLKUGUMXzZ?J7^eLa4+%_La;U{muIc>r?(cL|c>o=}oAqC8h zZp(({3x1q5v3Yn?Rbl?nUe6snb*yHc>K87arPcxjiV#E$-s#vGx%TZ>M(o?M-C0;@ z$xQ3pt<%*jm&3}IpZo39Evr_Tb(;PA_ENX3Wbxqhr;d&o+>bOXT9@PPmtR`+!;G>e zOCH+4|CddX8mh4oPn48&@?;dF#+in&QiDg8@!&ydo!TaUUqV2Ae0@VImd{E{1zW|b zjT@NY4HS0~2^O?fC>y$d#}2Ax zEuJ;Y-)LO9Y?(o?FCE~wd-v|qqepq%_GT?wO#Ajbe5D&6K4df)7R;Gj=yZo4je_B= zQE}+(AHHkRx#PU$Yq^0#2Rz@Sbt_^EhYcHX`S`K&Wy?;OI1z;TdFJ#U17Dar^-GS) zd60DEm%T65s-f1yKgfC>FawMxPo5kR5n)m}`t*7J*slj_SFb`SCIREtyxfITr;hvL z^HHyk26#zpHFd(ThTH)x zPX-RaF3QBZ9XPbObF%SK39#ZQZp7cl}XZR|!>jiu7Ugfvx{N0 z);&%8hzLrU11NE^x3ZHG!z))}uf+MO;b|>+1vYBk)~z>Cx~q2m1}$5)vOClUgSlq4 zS{Z4X92I>%`s2?(lki8arewtm=CIO9_me?+l6FwI3o0Dd`uer2*Uz2Yzj?C&h})go zvfbzpKFx=L=9|LVvcFBcPA-kU(4lJ7to8Zx=f{nG3qM7w6Qm?1qK~&|+tE(;J*_6Z zc^eeShsh~%v9T(nPm}Og6wM(tkswT*^~eGH(Oa*-_vO^)&06935fBpEZ}ix2$G=U| zl}VF6ty#a3#$g zjI_nq+`L&{g>#eFx1=uIrQn_$~ph=%L}`|0k?9PwD}u$%MKcc z6#D&I5GlV2jDLye|ImEVm!cW?E6>0)2i%g%Rfl&QNkk-zKNVj14v@i6r#7s=Y|R>A zR}6DnP|gHyX!K?)RTHJ;mR4&t;XX*HqVgv(lld1bM&dy6BOy7r#cD}U3k(d@=?u7q zP^&@7N~)?|uU^BtwcmK_ZQL~!JExh1dfs?Ka0CsQ=OaHiFSvLKzTLPU-K14>*(Dw? zYCp27s#Pf|DOIXh1L#we9|GKgfx#xDkIF*Q*=)a{;^Zj-n-sSV_Y1>D{5)@7>u!C@ zRH%xx%qNpSA33zgPcvt~+`HG|8@G%`BSTima4^;#5FCPkh+eC&Qlolu!}=@MtOH~b z4;%mwK4IM3ZNeKZoqNM(wX_OvPLHTArYgv_*(^Gv2{CaiCbm)STJYY$J@q+(Vl0oM z6l>IB<;p+2dDDi|x7o*GD|`~45LUhtV+Z6FQeWUzhWMM#`v*vs{r`63tlx-fIam40 z!M}mEoTs#`AimPL7op`o{&ub38vnNv;kN^d{wZk~;(yOcu%Sj%kAq|FJ3z9A7GBTbNkBm8>r1VeZx~5C^0}~0!bVq6=lT3M5;V`>~>qP zUcGi~-E!%~347L~6_gossoJ-1Z?{`RLrdMfc^&he(`oP4>-qhMk32|8F_?_!&R@*P z%Cy@oL^t9i0%*IPxD+}U&7G5wn@vD1$f`tnuR$Z-YPCjn8Yvug;){H0nHf~S6auXLBy~>%I6#wfl`&@4P zbg0Ir!W+F+yY?!7pAD;*8eE0T7tZyoP`PPyiH{fR;vWGNJmp7Jc>%1`L^@*cjZ&Iz+N0X;K$}5x#ytqrN zHCC5+c}sjF4$g$_h-Rf80VeG_-Z98;&bLh~v4I3|Cy>3zD=8zI4bS86`UcIke zy4=*zKas9?AO=koXQ`6GhH>m&V#CHdb zezkVpdZZP!lCp|8Ae~{>{CSWzFLdnOyF*0QlgBgX&efTGLqfx5EM2jC@BXgg;e(&= zbLY+-=wy}0;4%46pFKAyxcKnl!&6gJKl$Y2X0_`L95{I3*tZ)t3J0b^UeFWe7jZ!) zcnTTAwpXfE`}r|%zW4gr@LF|0`1Fg%c?C|5cKoFGex5tGX`On70-3me>kU5s?|%K= zvN>}*HfXqZ?fRD8dppRl<)&_aKaMH-R5SyB$r(WL`+xcQWo-Wx@-FLnTrsdu;&0%RKt&WqKm}tRYM9ez z&CSg%UaB-@xnRJ#Wex&PBtxEfd;>ka{lJ)L>SPv_>XEYFZgb}3+2UF^DTAHX+->#4IYOG5w$KaFHfh}ko#LyPY40QH#Z~> zVeYtEXJ_Y9gT}|tSA6-DxESsn&5~O+Ffc&L=_-aoD@`GnJP!K6(tt{w5!V%FK@*uKgaM~St*-U0h@&U*cJ4xvxAxmudQ9@Wb zr4T)%;g51hplFaRc*)QoqE{s;74xrgQl$vmhjA%+S(qImbC^ROU5J0KjtCSGhce<5 zDmkJ_Qi;j1u$Xc#WEkR^jXyDM{Dt3gBRNF*EmB7|+0VQYTdT~j98k2$49G*m^eMkZ zWU>{7ynAQk&-wk#aokGXTNGV+3f($=q0{9);+_)2VEb z5CQ{|^;(Dx6)Bu32LmVE^uv}R+7*&m+`kIsmHAHg(&3VNfziNG2{R0zw_`1lng zOkDn=1b-tiz;h8HN~JR*Ys|$0XxT1+ArK;2VyQlff~+<0%Re}T41^p;ls-i=L3+>} zc=8V}2^r3QxNp94CB|hZIS72YK}RjL;Nn5-mLIVqdgD6w0kLZS$Eiv2)`SrzOi z>__9H_0yRo*wS0CokNit3a~=)OExYFHHiK%u%LwRs2p~wZp^@44u=*NOD3HK?(!Q`}qn&L2lD+S6j*dtmqZ#^$RRoj6T@JLbLK!l@54&IBq1s)hs zCxHr?Z^Z$Fe?WiigXoD*ZH6KYBs_++gR&stPXLvu*n(DE0^|Zi!5V-SfYr!AuqZ0% zC5j}A1HPhQ(l2Nbe8g8Rn0NK;X`u^5tHtk8I1vu#gEC;|Rb&c5} zCQlH}uG$s3DKGO8N;G?cQ9zP&83PA%WrF>gBHEV?n=2g_8!UfS^3B5EunUl<4!waa zva}%mWQTw`R}%7ECIH_a;&=2!j0yqbo;Q2ewX0WTvSiLFoXpV(SFIlDDUwgF$V^F@ zN?8waEyl*ZlUv69MTKV{m!Z0a+;C=;v|@4%X3931K}imF*hsY z;^~ush=S*Ws>BR}-3mYeQc8&z!6yL@(cWB6@I)5)YW0sl{&@KCVWA#q3cXgU34=DA z19c#KB(a~%Y2XvRLSGhCR9d##uADvPv=n+_7x)I{L|`Z=Nlz)BS(ux5{>&LbN6|!G zTvF6jgbt7=g2wa9u3dL;#V}e01Hmj|^!Ykt+Vttuzf(e3Kx#tTRg{{f#Id(yjGdv; z>Z9x@KYZ}P?b~q#ddj$HSmXo<4LGBy%3wiK(bE<9{Wu>H6A0;N^}4bXSZzLur}#_0t4o1%$O`)Uao=JP1o}C}Uv<`5IM5&XnAc+8sN#jTkbRvv~{{7-w;_(6~h@ zSJ)D{UeQ)G1Ao;Sc;%5jBn?R*)C=8dpK#vq?;fuI+Xm z+`Z%bsbAtI=vCE$B{&3P6`d|I^1(u&$^jl|c0x!1VZ8n>JMPus!tPyr$GrZgAht`= z<`K0hMmho~#rIi24OJT73<{{zH}DnQ5!iLPlI}!L7|_e?%qLKY=LcOLEffBlTqtzX zzRtoqja8G|sI?estaXZ(o`QZKc z+%6luv8Rd$hfIPaL5 zXQ_I|m~aHABZBn?S^{%f4KXaY0~0gzM7|dTg�BsIV)zg?=J#iQJ61W6}+#40m{C z=8Ytwb=XvPNpn-&UA=aPLmZJirYj5shL`pabap46SG@nt{La3}~lJnetce z-+#Yva)6O7!-HOXo+wIi5hoi)YVn z+q~(-(ZfOh=F+7?&zv~6W7FE$n>PxaZnLj%@!+7#7te3myy58KL)4QgRl1buocsv9 z2+E&~I<|Jrs?(=V7AqE9vP8+P8`p#1cAI16s+H%@o>YF$+_@dsv~g4L_rShAYgR44 zbpBMu^5uPt1woj5*ZU9e-&?<8`JL+*tqxm`#a6X?&Fm*xYgR1ZwsGCHE0-!%s8~E8 z;KHe>Es^VQM_)y2HTd}j2N%C`{@nJ>8&4lSLdNhCp`mnX*P2ze>(sB&sEKgg&^q+d zYMcJe)HV?j5uG~28O)tCJIFt9v%a#cY4zb#8OBT$j zUa69wuixg#O%9vo)bS%5*R3+>v=u5<5|&1IV}WBKO!(z?K1__?xM8i2!L(w@;^-@v z>(r=vCpLD`{CTO#NmVPBQ=70UsNIEm8`rI0y=Kk5+jkn)tA{QOVrHeJEL*&A|L)yB zTGcNH_O$cyLN0}y=tY$ zW~)}IB5_REoUdItvvpJCv17+Wn3Ry>*Dqh%xiPY7!v-e3E-vo&`t=(cHmFy}UGCcL}ZMDk*2x^HWaytoABtz`@k)v$+ z1x{<>E6?`?2U$0`C}XcCX{+QGMAwBCHfdE?E}U-?PMD9@RahAL{g0!E4DuA_-8dWd z)`%BJ4jbyR7hXAibV%PGYPZd0Em%Ke+LM%ISpUe?E8iMA=;DbZzAD!TFAqQb%U<{o z6m?i*avL&!IA>A@rru9jHTAnsK77~Dpxv}=!P~=!PI~LDkO05ci{?z9@*xO~O8({> zudQCXs8*HA`?v4-aKbxgl{z;yy-!4^13Pz>2?_do@@KY;bf<%ayJV$R*blH<3Q&-F zSIzLh8vFX%RV&RV!%v@nII@4Abt{(_5Agr|wb6SvMZ%aQCnt>^H3}S8+3lmdcYl(a zUaL~o$mPqou3Ce3|M8@Wh55O~0(=_Qt8L9leVmnrA@szF6AvCd0CK==SqZVWDcXu= z;4eG_EHR(0jaL$VN?bdvwx!d*{qXCr`i~q&)tAp*wx|=QSvQvr4Lfn=lCMrvke0Y( z-HKz!jtm(z;6cKz$Jvi3PMCmax7BJtd-FO=>_=JYJJznCIAo80m4R9@wy8aLW3Gpl z3med{zi&`5OLjP1_y_2MGG#(1P8biFJ?_od8bx#%{mM($Ld(+czWHR<4+CH5t9Du& zHfVhQ%;_erJJ2~4*~*nG4;eCK@46LljUA^8EIx0}&mL>u(bLCNM$@DT6Pq<_28@34 z`Q-cYcOO5_c;}sWED-e`?U`%WP-q{eKiCqv?$pVs7X}R0!Ri9pYKUM+Z0HLj84L3g z?^mu|6(14t>QJfPe&hAmUmYEHt5d)3okzX*%Ea+wA0+nf*`Z^?y}PAKl}3#gibKc^ zO~XbF=Fgf51C*4Mc;?ip&fR-JWQ(m*)*DzaVMRQ8VQD-wrhnJ4R;|jx{$G9i`LeCs znzm?GwOr{blRx>;<+^&Azv|QzG5ncLs>SW6+*b*5DLjU^90kd8=sBh11 z9U?HX0@E}moCydnw~es~<)ZfSS+#m4H!Cmk?vecm&cw!#O?`b+OD0@u8_~Uxzj$H z7+$SHkFMRE7CSL7=*kA-WIR4LkIClI6Os7kNAESQS*LHuuB!a}?9>!61sxs-IAV5Q z7%|4K(syXtXwi?~cx>1eQ337nHm+#)QqpC0`H!a1-i7gL9o)MNC4%&w6hDwu>`GDl%B}*c2YBW@-{TsAxU#K;ftXxy2@kvX4q{W+t{M&fBXl*W4L5r?ka`nDB+Thwv zB9iYVn5l(k_YiwZH62)DwaO&eild3zRjF>%0z*Jp?FK5np=QhW7M)+EDs?@1F0Cam z@%AlH{<*fT8&?eN->Dr;_SNXCH)5~1>e|8K(-l0hM;ekg{VJ9E*3X5r1j5eKdozP?1+kiQAN z53myd4aL!zI+JEU-?(crnB$yI8yaXr;{CE^%c2}Ao_TIerd)l|q6INAF$WL-8WtMb zp<`#75uZNc^U;N+O>tatI0-Lee|maqnKI>AYS0J01|J!rkr{+I{7Iw>fR63dDXuUI zT1j7oNWd})dUB}Z`vRx0#il53?@TQAQiDP@FvW5hLQANBzKLG1d<1v3RjXEIrhYkj z!rw1|Dip+I`uY1uMIAeP{u~@O1GnVnI#M3u%p<;fN|zu_yxghG4Bb#XaHLte0{(}J zhq+#l6DPNF{(`yp?%g|l@V7iX5g5W%<4H?+DQG>$A zk6ZwD=h5XwQI)BTU$1) zcje_A+_O9S+=b3PdlP(LqGajh`|)|XS(bwACy&#L2L%QP1?}CjD=Q%h|7Vm|#gh)y zRfGXBsnVFtzQ%IpZztS=LB<)6zX()wMZs_!LRcP%w<2Ol{6>&j;`;*C&JRx1lsm?W z9B&)}a>T*TP3zAbKPoPKcDqKhxCk^WR|E6;EE}>wp=(5hINh4?rp;8oe%m*1VrGaq zJ$y8Z+HajBI^4q~dSIXJamL*5zvZH=6m%NhoF9I;8ykZLOx{6pY^MoY*BiXqOGHW5 z>D><29x&Aoh<@n)^z^jiC4!3;3)s7NPkQ`aI*&#AlaD@Z9}%%;!@6|~7GAx45r?0` z+#GjiR;4PHxoZ?x%gV}n8s@{e{uOF7n5W}W237Q{Xa@eo8F=P^8;%!kk#`kfGp+l$Rr_V~{pH?Vi(IH`QN`$uR-FIZa zL6fI^+cBc+8=rnQ^}~Zi9V?ei<}k#Pzs4 zO>5O#xo%^bnsw1uZ8$m_4diZx6au_~Fpv-8rb+XbS1w#cnU(SjYNFrKpwWB@#>96{ z%8yZAQWg?W*Mi|>Xwf;LizL#L_)01T3y4enkIpc6_H1~3L3v^(rF^Am(45OC6y=@M zb#`)3>9h`vn9>s*;Ei8?JAL7@RV^Ac?K^N#TtcEsYpmI@aYX+?@4ot4+lCEsW$_6u zh0|ktO6L06vx)}?@`gAZ_-i5{xHQ!=aY&T0V3{QxpEN40mDq2?JgnA?diAyIH)6tT z)mgq~UDl(WQ%K;J-8js6xt0_`h`U-|ryC_RqOW6!HNSAO;UJm&<9*ewES{Ffq|%;I5^`xp8ljDBqT^D*#+sTv;#YVwipl`#F=RB1v$AvegS$YU@pgn-Byj( zXeTHV9L5hERo3l(keE26Tf~7QM}tDkg3ma9|%YS%sC3HBa#TXmyy$ZFjG9W zWv7?xr2~&LANOrq@A$14LqMR7h<@Dq+!AF8lMPgMFAvqPTU-*qBK-J}14qb$fr}#5OPV z=VpQ3WQid%o`x6~v-;;LZvm$fOlZ?5@p;4u0V&{3%yIHvqlD2XC#ZFzc z{jbl!v**Vx-XG|-@ER!IqQrr}lGsRZbif-Z>?sU^pD)Qb1Zr45vztUyR)^cy&je85 z<_pPdBpZ~3F%ppuYkl2J`U6Hk$Bole)t~$ikUoutOyj3!y~03&u+%p`G{? zt=2#?RTM^aQhte|3vejV!OKGYx!Kpm2O(W?=Fn*U{R2qP1afJ029O${=Pwj45TJz9 z(5THO{1$Z(=ONOCK%7}q_(nEBI64npX7 zM#j%a!G|BY z^b*tK9hn|yM$Z3NJjwt1i}<&=z*FFUisB+%)v6*t?my+$DQe%Ja^?S%i}Hy5lg5p0 zi+2~a6LXr&X3ft_Nlp%e)h-n#NB|zWJ=qy4x%qiv6)VDBSY29!L1Om2s)QKktT_qs zx5Fw_Hv9M!&?i_dd686<>9yt;SPLGLTO*)YF_b&%!3h2c9Sa=DfI%2SCLd*G1qTH9 z7}SLY`DyV9RPuEieHH2*_@$s)A;k(FWfv4LT}H3h+i@y1Xi!E`XYmAA73AE#mr$W< z6#$=1;feyQB>Yo(tOfb@!U8l#AA{b<*WaNxD|tBVTCJXxSoDBG?hKsr5NZ;I7vv@; zCRC|i2WCSjrDYO(MJxHiry$0MAnmItq9nq;Q z{3Yv2e#x-15}nDp{7phAS#~&7cI&u!&{?I!a;J)vZkdJO zH)fjcDR7G%EYi4@N%+%#J#*_C1`LMs8uR*|KXtN0v!+A4bnOw|Y~HLNHRO(RItCAZ zu~T^X3+>x=?GzCo6D^V3prgFUE^Le!PMv67t?Kap{UT~r-?wuIFS_Wm0FUB2^7WUW zKNlX}tx8kIM!TS;?@3X47p{sO(K^CzR)*QheQ?+YF4*50*sliF#MY$PBo z;Xf+rcK3$$yESUkwr0(jhYh+DeM9=opkX5`E5YE<92af2wlXztKF@G~^wvY$VFs%7;m!v_xRTCdjbt(#dD2wkIi z-O$NByLWeO*q}?}hOKH;9X@zqT=Y#+arN!p!UY zhb}uWUO9IwX22^iPyKqb&TS!5G4|HYHZ9wftJfHx3iNB-D!kgd(CU_giLbpf?we_s zVsF3m?UZlc8C#g24bG5yi2<>3A$^&!lQFjc!)odHGyX++bWz5=iM>PIIYJC-2-_d+@M5CwJzz-;f0bGbGc-5{KrP z0R#Y2rg4W(J1<>38-4f1k)vl!{aR(q(`#HYx30Ht-8!sZLyQVa8WWOtBak3hTpC;M zgx6nw`|GbR#oT)DtEu0PAD^F_jSzyC?OLKz#i(P4@sI1$turMhNorB6TIH;?RKaB7eqkA0 zXQGZ#baBwo5e}7pz@Wjvz^PLwl~-NVUd3PAV>asjOvcvX;pJ-Ae)2d=*5~|5On+Qf zy=TYva-pT$w{3&=*QilrOmj3|yLK&SS_)Q5K7aJ-CojJ;rc~)LyzV#=n;u63F|>Z^ z{25hVcE7>HF$4DR-=94fFPsy_Sji%dn>n4t_u0p6tX-pKv*t~+9%bMnM=LPMct@iG zqsyXHB8r+c4JR{)QKzd}wOamzWOx#dUOVZ7_g@)3resNqSaObxSr}x&a1T1KDmQP? zpdogTy5Ea~lszZq(Sfv3_-R2;jmK=#)v8&&Ws7EyGcvLLiX)}sZ^QZq4CgB?H*V6T zM5$6xK-H^MwLDCr0h)P#(uZTlycQbDsD&a2X<5byfx;M1MyYah1`Zic=^3dU6VatJB`o&u-bKm*#bsN;W=;UT`KU~&^6i;>{*_S| z{rcO_fKYqS)CM`rc(r;%Q1KE#GCXhPsk(mM+KtQ6`IEkS>E#>~{<3~;&J@z2} zZofHmwAj0#W8t_JxVV1(X7g4p1z^E@hwb{+Yxi!PU9@!BsP{iq8H|*OL4$;Rhkupg zmIAaKDfb86H97u%*&4MC20y~}eayuID^z)ung+PCN+2H|rcutZX6Ic$b?(ab%iA_? z{`l)}p-8}M^iGNr2*69mZLm)5uv>3ky%>A-^73U%Uj5)>Y@38dih0#*UnEbsosdkl zbF8wM6CDqe;zMiKHu(kQxiuzVQ%J?~nQ7o7O9)wS$dB4pn3r|__%Ana-P*c!>&KJ7 zmQ;h3Q*hY9XtA|Yir#yIvoQMFb-DD?Wv{&d0U5ZRMAW0?O6fkL5U`BaX>U!)-K;)!E7Z{64txo+aOKaqWr63@+B zwz+cUO5Ya-y2RP$DPckvq{~Pony2H^K+#q-1Ao66c;sqc{8L~Jj2m&msag8#6(HP7p~d>{r1$df-Y!+OO7rH;v3vsC zAiFpQ;*sKV8T5Lqc!WULh)EzX4?jph39&Ky!7by5Q)sn_^EZ7{GIr6XJvaA@FQ;VO zy@zu&U@iV5)IiYVq$dLxPZPydSy)gwapJ`M_}e8aRc6$v|8f8rjp(B_!D94jIY(hP zR5>|0l<1@U3iOjhfbeAcfAGZV)A#O=e(UYYZ@rO|_7F@2=tY$l5Ej=!E^hV@+;{xc@w@j23>-3M;ETDB zpTNK~3I#)PpVPbSHc7P#j4CAzOg`oUOQGn_D64LFZlTRTs5nTBP$7*q&pVwphn3rN zb3r~a?F`oJ<6CI6(v(z0P-}AYtwF)1JRZ@?MQ0{B$m#YAD}VgdxwE$u zMvtE~_4QYuq^2OM%7=yBj*X^ff@DZkUP)T>{;-fh6teFrW%5T*|2w_BMSXpCXW*Fw zZd6Jo4Tz%R>9k$Dcbodvm#BVu57QboZtC@7LpcsAUVOm7L3eIOr>AECO&HBUpc0{F zA4bPwm($s$sBq4mJC!O`p=_h~@d10Vba_Af`1yD1-ZLXER*L*eOb2Kwgm2|46*JS) z*(0>F*uxya@zCMJ0P9N^E&{QlmO88y2Y_8hKNjy7bXq$OCz{YQIs?@R|UAoi4^{ZE?Er6gRxRL+q1a5C0l3*zdEe;zo zd??EBg$oyi%t0gwQsZ3c!R-RyJ}N&F1VmTMlq;Wd^QN_+Kp=(?g4T`aw$9L{Q)h%o^v!Fy@kyHsnS#QP*btf!iGc76(D(jH-0`@+r~L_WWD|+by|yx$*I3%9mrjObDH2<1=!DrIOxg?$*5r zm$`ZQDl&>Rt(B@)fu(2K>8kg>h-{D_*#xzVwxSvMd(OZ!2iyWsddfDUIJ)jeM_aR= zSe`tY_5Dv(nzm{Z-YP9AY3te*`B|i3dayonL!)*bOO^=E&B@-oeFxreZQ6Gx#YyCf zrACKk!-{2rtt@>!JCx=hg4r!#B7jj)LF}SjPIj zZjBo8w_{XpyFsU3{PR!b)OKb+TDfqMDlnvZi?)aq%j1mf)cel-$JT-;SsAH0xsPm? zygfTMxLpM;TeVRIglt|s$6(7_y6|V83gsI#ZLTxwtxmTCof05(*$gUM`n{NIr=q_9 z`b#wPn)Msrx^!;!;`yu)6#Novo>65d5qa9(Tee5(YgQ~(1^BjV(-O6}fH=s9i7tG| zbDlg&PqpRcSo5;UC<43ErcFDQzwd_C%M7l<r}sW1_!P~W{5U#(7UGO2^wID^ZAL>O&!Zm(+p_k3TA zdR;UF$_zYnz>R)JDREw3aAeWk`5o%iYu}*3u8T%muTGuQCr^o9?s6Xc_3*&)KhKACAz}9W z?@a90y2Yiaqg$6R?b@{Q@+C{JUA;Vg;`pTd2_y*oX~BXevwvz-wbI7O4NI0RAz!sc z08z|<_>Jfdx`Nb)-P*MtGje1xU%yp5c9aS&vuE2@#E>Y`ur&x4bX%3{*rELc+qG?5 zug z`k_&!3KaBNxNxC(T8pZS7BA!pYY1CbX7AQ5#*BK&Y&5OezCEmT>FwLMW~M&GNPq{q zSH0zosDlUkv}@C*Ufmm4uFhVtm;yCChabHAZkO<;jC$jeC0)atE|@nrCi>dc3FFce z<4rz3Kh2r9c;TD|70Yg3w|eoS`P>mMCt?7QCT2?QDacIk+_vqgmxc%W`L5o!qh#@t zT=2w+Q+4XrSDZAZVCB;epxytbaf@A&pQ5d32L6UK@W)l;-?01te_duZ{Lhh-y!O1s zq_|^s?^YKSSV&G3;0rk+&L5<}U=5y?{TTb3Qv8uaab8g8DPHSo5rf zM8TVV{Dg(&7{Xrg06{w!6D72Xj-*bI8#%c-0RjFJ$Ia$)W@crFln(V0x4;+DhHkk@ zWrdVrP{MViWqL?L4ZQVh*WtTqGwL^QMzkQ>F@J@qhCj}?6oRLI{=Rfo{40b(66qqm zgXCJ$I)sH$<>y%Z%{WIAg+xvk&!Z;=!Nmh5Q6MD0B*Ri-Vd%RM7pSw+K?**SLd4*( zbnn{Zt7$(qXcCUIjCg>c?Tg^hTCFzDCF=_pgTG-h0OjHxjn{S%{<{~NGsm7v?4p1MwjE*uI)2_n!WYV5v|Dtn?vFP7p^6^?_)A$ znA{A;iOf-4BpEd~k=dmw!FMI_$0Jr%(nZRMw0hi%FgduT9T*?E3cZk7<}-Jnv&DN- zaorTX9@$8|D^CD*G^PJ6wv+{j^j?C$yhrk&s405H7aHR#^h!LCq)o4IQ4GAwSJLsa zbWj4vB%Def6lsuav6Gb?;R0MjZQwHxo3tkPLdNK%T(Z~ch~9G7Xot=;K$$a2 zd@=(PDJHH$++2?licfit>7w#P@#e@A|8y)uLXkCIl}WCWoUGo>Di*_bBG)Y1{x4?W znbWv=vY%oEkEfOjh~|I*18*aiGoERk9!sK%2zU}+@X!j7DO<4sU%4A7u0V!fUA$B< z1s5t7eN$d-Jw9gk`em&*OFC8EOb$d-FKND}8L|5`M z%DV@IK>`3~6w#GD*Rs>kFF?>$O8QCe6sppI@7y^~5tx&IgtP{=d53A%`{1D96D8Q9 zb(=a32y78obWZW&9fzW26Q2H54^?X%c7R6OBu%(BuIxF z0f;Hymi*=vc_b*xqdXPznLSYU-dime1VK{9i#dcj@2AmgO08^Z;rS!X` zOCn*DV2CK02DRbSFQ;nFW=HNk;j|5nDt?`R#--|n-y!sM-L(_mI z9bkCWW#7AP>+Y=^d^GOc*DjtpdYJPBUJ!1KfGv##S%??g;5kjZX)vWJAlV?dDW15ijxzY5JW}ZFRVC;XX?b*8n z%7e&zN9t)=6!ILZc|hf%lvk8TQ-Kd&I9_;p2iPmG91BU@zXgL7UE9SwhOe@n7fFNn zngA(bio78}BDhH6s)XCIyo0=5ygWj|$bP{t{>Kl*>jcf^G9D&vnLn3TQeJQVD4YU7 zAt(jX3l^i=I|UF`cH7w#Q6Q9rL*gP$%UnsKw?&o1?+qEjpMV%)52*MBA^0vwm`N5{9b4luW_0M)>D(>w@|KLW=SxKcU!W~+MMU2 zj=8Ni1RF8~1Y*44g-j=jSdkV=@T3A9f|g7nxWjq${NlM&7tfz_*lqjv?OnWNA=8L> zkff9;s{Bt?(1(JXp!5V_EbPw+rIv@!P z3ET2G=(W=>CG)r!+!KEA%b|nZ5gK7XrJnJFWNK*w=OH8Bmu_du{d-&1uN6_JJhwE> z#Yt|U+!xZ<0NgGkoS$@wHj!!sh$)l~0)h@WoYav(lpvesuF37;Fi9Rjhvm^ytEYZ3 zdG5@g@NeY+o;aQX`cJboS+xBXW`H;G+1g<4KF=wjPgJG*G)eM_zJHk#`XaNIHs2=(gHTqA$_7>}HI-y+Ap? z0CTVw>~cB_A&5oiV<93e0bPU|wmBVEoySJnLZ{0L`~VO_raPU<@weX}(o5^ero*Ix z<#1v$(XrKTmL;;lXL%!Bxmz%Z>GgpLa5_v(0rSF!c4#oY^U((@BO{TDvZ8m`Oez;Y z8b_fdf))BoTp`iWu|fcxJgmY?Go>EajISr}-?4=RFW@O%f(?;@vrs_V@f~>bxPJJ0 z%AuV*bx=JbsWD6o{E5i6VR{gmCyFzVg5+Dr+(6e=WOUrMWt(2B>HT~!FE4@E(8;Y5 zo*Q>Y=FKU7JTi|F5fM?>uakRS21zy;CYu!QqUFN+i>%^#5aR$*0C=(!*~H3(Kf4)} z$VjkNo(Ji$Xt7*`Swc>LvfOm-fG$na^->GNn2Me zd3Yz5$q}K*0*kqkt~38AF@p3(+n+cCV9T?$3Bf0&3}i9BHF9mWN)@i%x{0Fp^2k>+ zGcz}?TT9@3czE-E0|t<;b?c^$RVvoH6BiQ~d!t*oZs9Fku3o)5IWfLpzX5d{wNMml zb^Ptv9XocVrKQxaS$*ix!5-gY8#hKK#N2$8oTxUMM~ogF6y$&X%GI6Qw>zx%=X&&M z)~YpiX9y@)piK>{4sa*lyR&8Mj_fCodOr7D)7EYG?A%VEP@mp?i5bMQw06y^(jorm zPaZ}MojY$TLRg+XdHnqOiz7zA<{LmY928G&+heOaA(Knt(oZ=)oa!mIBb}|pU;*JYwpHIXFPZS**$pFYh+13eJbkQ=_u00 z^zPj|q-2;p`U<5hOp5_vXTDgn*uS6t&CEGJ!>&XfJxrMkUtj-Ue?3&UL7hQ^2kzN* z=-jz;9U|Jd>llF}PpH;R6SvxVa%gWs8}GmQ$2pMHki>6WL|>@y~s^D_UqBq z^vn@M1`iuCZ0UlZKbkrB0#kqf{Md2hxS&^G5v2ESzy164YtcXdS2G|w$1}CD>V|0G zML?I?v2x{yZ@hMJ=N>09;E%JO@7nR|<#Salm;3R%shd_UH@V&0)~*{ja_GGqS1gaR z-W@jRr5F3h-@4(>&v~s+&xi4GX0`j;>0`q?wU4=SzGCST%jeA+{z7kbL-=N@)fh#| z1T@vx&Yl?4y_2gTC(ze8esI6|xSOa>%(;*%@B?UaYR{Na!(G-~yx+#Z*gqlqhSp~N zX53p&OD^6CX{kv+PyUenO_Fv*v4;t%;i9RGeLVT2kErZxay!SpI%?gr#nmg7JGg86XCJ<& z@z_%m<0p<9>BxDkvgC~F(K#nA*{pIbp7q05W8Xkq!xeJomtP_T5H@IzqUq6Vh;`I2 zltoxrl^gHxo*mol*|?!nnKCC29~#=GX=Z$kEHaQw%!!yv3&)Kay70%DN3@?c*@!PL4O(U&d^ZQuU(waa1ULKiKZKXl;pDr+8shpDDDL@dn6f&F`@ zP5nBwM98UQ2S*GUi2kUA>oX8F7%d|b3PhKYN%pbiI!zm)fL}auWa_xr*DPHW#-Bg_;N||$uUok!B+&2k*GKK$yw>1$-MDn_ z)t;RT^B?(|)Z>Q^KyLt#m(5@B#p`ckMt_`^HskXzP_Gq{BRVtBg&mcf9dH~P_qKH_ zUw`RE0-*Bq9)CDu&})PG#ay}Q%+GzZcaQYgn=H0w{xD;6Se{8ONhX~mL-MuR!n*LUyk-J@R~$%4LF^JdehO{2I~ z!{f&Z3!E`+I)p=`Bgav@Dp#yD`-kbRBHGSdwu(1?_=_)eZ`JbnuZM>Y9(>{OzA9C# zy*Xh59Lmo#zwa~n#i`#+<=sk8PCk5a|Cl;;?eaSj<+D?dUi)@$uh6tfr%s(JlrFw?^@_3M-VQLCuU))&xU2Si8@oU zLWS;~+xPF*-IkYEAvBaMD`uVcNm}Zrjhjy%Ki;cX@8v&y|KT^^4IDWNz4UW?;i)rc z!8NO07y#g?T!jiFM!xjxngwr-d)weweD0i|T~^DX)2GQBHE#Snt(w)_uzJP&UrbZL zT{P~i=TApR9a+8qX#F~MRSs+G)@=#&l>P~|jf`{0-?<%DHtfZL{mlVE=yaC69A1y4 zySG34z4m)|`M8$Gt-+k{Ll=AW4f8R8sQm#;YS-S;TG*}F%#A^m!JY}PhS>WvyOF!@1ZU`T0h0U`iG1_0z% z!2jOAA8#&S79ULN{|VV)z59acl)~_qAHwJ@%$3j;!n3J;eO9lOXC@~p{@DJ5QL%T-zCO9B ziD#mYJU{T|+24QPb5LgOSVlBIxCd|1d2{CgxvdWEx_>UYhWHtzTGOEbRxrgVjJodyi-(4`anU6pFpQtsSgNi=xyU^2+lGK7iv zp9tMwq-p=j(EZG5+~A0qnP3so5;fX(ox3`y!=g6CMBjRnc)wYdY7NSlo$<-1(Pz$C zC}pP6hPP@>N^6hKRHI&#mhC#bwPw3pTdRJPl+<)gIng(7wvXrpfyo+zXAPr?b?CXzA19!&|xpZIxB`DP9ogK5O*UQ80giyb)B;1KyAD=u3nFR zd;EkwtJXZ%wadO8JJ2+RNk`j*aK?zLa@T0y+$W$I4m5XT??8?AYS*rA`SSg{bfXKT z>$BwKP5$V!hBfPT?$%9JV9iK>^e`m@ZMs>r7Gf%OtJ=2dfTE4*5o|zxM(qN3gr*jX z6oaojF8*$-4xQ8{U*?m#z!4q0+`kvk+s-kV{#e)FyL(sV=U=6MLknZq8{2p4EdF?y z%fv}pyflUkALayKTDF`wdzQ^wfOu7@?VpW%J-k);o8#ZHVhz<`Xpm(Gw-5&w=G;Z4 zNvvyb=dJ5kI&|)8rzW!27+S8pPpPnkd-tJytsJ8nywO`Ellv8dFwYS(Vq zs-?*{0FUnKHEN|kNM(LwZd^aOKC)5yiuEg2Ik|JsnUkj;CMD5;qyy9IY+7yewh?%Q zFk?K(xbo>m5s}AQOp!cV8g+Q9Rx&WHwoZfkZ925a_|Eb58#Z{5@(@G$`4dr{JG5<9 zy~?6R3+<1yaIRqvC%pgOm0LIWZ`$_h=U?E_&QGb*<#;mT59u@an)cYdqQW~0@1COH zf65FzgBrJF%n;WRXh)TrQdpSGnR2wkUT4IH zgtj3oQtXF1oiV^K@j)^+M;vX@P4d%Hiv3F$h9pQ^8huP%gf6P2o6D4f|zXFG#p}hz~DhcyLUfz;%IJqdiS2acrmeO!NrRu z&E>9LyEgC8;j^ctUL|%c%A^Msqr)AqUR}Lv#TVno8vFv!IMQM6(iblp!%E*%!e)G^yY40BvMw45rhkjz}fH*)gT ztC#E5Z%BhOQHTPSs%oXG`}Q5!xpm9eQ@$#+wO~zc+qDyOu4M&Ye|g z$?4Rlo!f;Mkc11axOQx;PoQ(|to zy-GhoXs{XcrdkSY`kH*`pbSj;Gj|<@n;GC%dz9MDxD~<5a0yXh`BzCi%b&Rr1E^8E zcC-4m-*|ltZl}a!sZD0+Af_-o=W&?Lsx=s6V`HV~;;AY9kcWy#36m>*;JL<4#9-#x z<|c}CrM?gTM~DqE4{P$XA59uP^7HR!_Itk1@xy!Hd2=joIaa6h#*G_@V0E!#7cN|E z+p{NNo|c+IYvgU0MwN|;(i((Q@|x9u+grAWJ$u}}aqaxY3s{Zr+=+{hj^>5nmE-lZIqX>D@P6|+Z5Zf8S=ZtZfF0h^`-Og+ zwrxFiA}T-U(SrGN_*3hSZ5=9i>9S>FVxwJ_LXXYTt5=_0dk#EEO2(jn>HMY4j7Rd) zi)0Xn=Cy5`);RMrj{UOd=BZO% zd-XvDR7=7t&{o~NO$U|6uy+3J!i<#Nn>VCHUjt7Z9tXKfaC+0LbQtTB67Ewip>5|* zA_IUSkLLLo2EFy^ryq^|FL%-@Ksdl-@o_fwsbln5CWkDLWj_lCMeRB-m!oR2!dco zkuKQQiiq^4paP0C6_6IX^j?yXgpvZJm)qa(b0&!(x~uX#3B%l(GpEk^JTKl;jz}6=iv`1wviJ>8Mhva`wfz+}vC^d^SW7 z&dVf$T#!g~nnlzr4jR#0LFi8h!|;+qrDBz%bkOCqkwNz8XTJY-%(+v_YKITbhU^NZ zY&$Qa-9&JEP?fCE{(G=$Pe<@3T1> zNmq{V+4;>EOU!zd2;DcXrMxv`I5*~UZJ!R37Wx%=H?#RAzW9+EW<0niQ`}}jSzWj3bwIr*@jvi(C@GyMp#>B*I z+O&yIhZ67+!F2QY7fs)yV)J|8t^;o78`!Iu5ici38Ssp-JbGQF>NUCz9sc~V5skxZ zy)bS%lJFe?D-id5s#EuO_wc{7_hph=E;uG>@qJ+2SQci-!+mj*7fSxEE1oxRi9Y zO}z%MJu|xJQ^VlUQ9FS%A2eu?J>`18egmK`;AP363QfB8eyV%-$y49ZS*(-Z`*88X zr7a_ybQ>`6OzcJ2zj^Z)w2N%gyjhF74IB08+aG>VfsM!mU@3K0Z}bJ>#!w2G`1ZSV z=Phhpr(UlCkDN!OU@$)Z5+=bT!4vLp7)v|Nh?j;wHDcnFR&DRM z!R^vr;E9(BHgMGRw-UHA831Q&{(8ZpxYuE0F4;*Y>GveyC)Tf>q)$+c!!v+p$ z+o0a!rAuZm|1zLdS%b;?V3*DZ_U>t1Bm7EI(wG-t`1s?G8`iFu3($)5S}PsP1>t|F zi49hn55sK22c)i4(6K`YL3V~1^L%(~1C}%BC(;UG$_><+-e8@BpaOwA(jmJjkqe<` z`2klAki!FCNlaS)!G|SFl%U2UnU{#mC}x{<7BhRMbf(j2wTt4PEIMeZ-C)L!s!P}I zU{zWlp3jKKv837Aw zr!6n-`iIb=Pbk&$6GSml znGK8Q@%Hv*f`PaeckbNTAFtuq_@m9_!-86@s}1_}>#6?!0ZdDb+;D%22(nyI1)(Xj zICo{GXIU&JBHF+eq9qO1fZ75XvOsXjsu^i0DaAZTmzbDHj5O_I<0O-(IXvb2x!+Gs`I-CKjDUy$uZt}FLB}$YS^~|$ufl@X( z%I7KHs1m(n4TNI_EPEBTW9$)(!ni;`kVGNi!s9&e2pyEI2#3XtN{cWPRO3=YY_|v6 zulwPLz_98rzGPdTHy#L#M$9bb+vDp&0#@X!L_XiY=D6#CoAo10V7^x6?nCS#R2MMI zU@~AvjSjmD!KScTG7V(4$xL7#0wRJM;Glf|z?_;NYSg%5;xQkE$;IOnKCGEP2xQ=c zxT`Ql#M&}dv8M(dgedSpM7vBHUKK}FQVLuW0_n8m>q!^S4jIsI=Dhh$TC{{Egoc1! z=r)KF93+RcARSnYY+d4(f%T#SA?z&+Z~+*2Fo8}HY6x4;v=LSE6T2{A+5f<=sSYlZD3oU{-HBpy;XY9h+TB6R?X#9$BG zcVVS5=V1X)k<_6&8f?P`oh8f!HZfTfP8#8;cz&38Ndjln(P| zQ9KZVp>BH~`#j8DKR}|%ya4I&z-mEgMSPiHFC(X5ipfG={J~ve*QidizUb6 z4?zol+O8NNvS$uqeEKn#t zEfba*C+UC<$-oW}mK6zgrW+X$LRAoZM+{hm76R1*pww9j2Gnpt@1!Fr8$=W2b;?>w zLcvq0)LCFxNTUW(KqTEKq!T*lAth0A1WcjGB%uP(Rgc?7dbbgKSnMqY5P?OmX$w?Q zuuRcJ(t!NbR_ZK7K%g0Fgh7v5c+qMWF1#yvPbnNi1w=JQstgkIOOXT`23W%#^Vz5f zO$bdaP&L6>09c3~C5k;H8!ke+9C#y9D;34`ORbb~rqrJvfOzwHU@cE6Lan4YiWYZ= zsJ{r!gbTsGUsXjf0+No`= zn%{o2iiyWefX(5HhYeyi-KkN72OHJDw_4T59~p%BRWOBVFLOpRf&WDpqIb_;TqIt; zbbqVX{T}YhD^n)CE(>iweeea~(CK#V+SRI7je)(ow~7c~_2mjEuU6Z=mbA3E%5 zq|a;CtbMRy{q~LP-&4Kn6ORmJ*OckXB$mo>fPE5ZhO%{6uO{8sqD9|sT{|{xH2L-M z%s}c46bj_HJ^Q0}HwmlKzh}?;YDIjzdKKvrs))vgqB85&e)nMg`VTZ{(4uaGwoUJc zJ9TQ@X#A_=MDszRB~zWYe)rv)_Kg~}Z``m&c=bmI4m@-ABmgjN%A^n9|A2~uZo+;7 zD~bcuBqV$#V#W`4KrgAQEMEwoz`8(&f&0L>V5`DhDx*O^n*kOX9+30=gg_)o3^B7 z+h@&~YIWwxev=z#8&C-Ffw+o#>N8{VOA!rg?mcmE@y0b9mn=N7|A0lj0|{7o{EKaKF}yWpc=83$S4Fm89kQ6*uhm3-PWBjg2S{}7)-A$CN=3m_&B7YJJQP8VWKh-#HNOMRit&|Rsb{8H0jULGV$p0Hf{m+!a zr-oefMq0+cA2$zvbTDG#nzbToHfy?L#}2p;s)6-6s9;QAcJ{C#TGsMf4W(@R&b;<28o4=V(?gL7;vmJ1GU)w*?7R_30K>z^7r zTyL^sxKaJy7TbQ>%sN)q-EQ6P-8*!d>3#e4v%B5hdi7>j5dl)rODZCVV3E`1YcbcZ zUbRKj$fT5H_%~1sHjKfOi7ITFCo?1M=kGTRdGaab*|lob3BUK=pEhrTkrFuYc=qhx zt;^2r)3-Nk?cRO+lkMJJyI~w*-EsL3;|_ZcQ|9MwtyQfWvBB8$ZJ!BLy zK?%1f&}yYE%^F1}5m4A*tPm2~bKszlKl}iX9Kv`?djT1KpzHyV%Rwu&LmK{D2lq+J zpWuj==66!^v@o~AoK!Cr{>s0QveX|J{Lu~k-EytUy!EwTO4ahz(vwE;`ldsEsBB5V zkzeP61hDj~lD|w(m764$`>Hnz?-aaL^#5Jd10n*uyAF|fxLFCJfd^v~UcHv+gz4PX z!ow5dFJo}Q20d&3q)S)I)v9GOTTy4TSgpmXRK0p71+f7PuFa*hvI(iz=VV+z5ViT} z(W5I@u9`Syx-4(SCq3{4f8scG{BV<&_uzgoF)1GLb4XZqrBnPvh-igFkWLOQITs2u>0BXv+OK@X=NB+4vhzX)vmg^msSZRYq>&VqrS3>}LSMB* zT#f+k6$2)6=ste z1;L!m4ER}G<}<<5(^HLRD~xBJ-RWa6`uh20Wo4)>I_4$n+p@ppNy~ig)rpA-7yZkY zvEm|D=B!u(3D2$AOprVyJ(bciq!f@Ph)+P^rTB|1one_hc28<*x{rSV5COs>VDa}0 zggX{S8`{987OQV|owBlC9sk;uxHz9OWih*u-8Gi)NK8>k1Hqdl~>1IiMi-sK17sZ zsDn-)TBYi>^Jg6PTn}EuQ3at<&^dA@6!tN0Q2oRDJsSuLPUR$eaNhY6=* z%gIGURvcy<+;+Xkw?rT+$xeq2+h2V0XQ!u?2=tR6RP6c4mK}4><6pk~rXBnCM8}OB zIr7yJ!&6hQBPM}|m07RTMMp=6R;+>o6&fi>8MCvqd5Ksid0E+I%b@Yi9v8LHS$zU? z^AM1tz94!P*_k<7{9M`ABRA1caQg;V+Oqq=t_znQeRkaVp~J3UO`;uyM-ee4+@Fh3 zHF|#^e>lvXtZXw_s5ekt@sed&JfQu{HkY?wv8*fyO;zLJ|RIXZ0R0No~E=Rcv6%x;!$%WI`IVtP<`RGbv zl>uuiEN!6-#2uP^-)w2ut{t6p>iBW9DC?poBb+zo(x~8yp>X?g(a~lUg5CBrCys=M zRi-# z3BX&a(q%~i*%1oIfz^oyZ2~vyLuIX0fM_BT5me=oLZt$}TEP0YgJklTf4}gXFSw9Q z&|TB@7#Asv6*k&3Bn?W_TG{(X8#Rpw^2{Z&?lDYg5Y3lh^-O`K20a&W)kk#?it ztaV7iucH6&+8)UNQ0|`gi{Kbm)u<0B8Bn)P^VzdLaAs%hkJ=f1@Mj{PWTYf-SiKS> zz*enV>3l59=FRowq%N8_+b^_IL}WwJuwX)|Jw3Quk6nUD9o@HQ%H)Z3VEwucc#hzb z*^OOGjheOME?xjMfwMZTTQ8dVfmd$ou5CY@JFurimrh{i=C$8sBwW0wRcl><|B?lt zn4H;*=FTcsF{E*1Q(hNaIFLk?^6YMBPGZ7^6Z>|)J8_)OV2P+x`^M#qA5NdBw`ZfX z0psj6dIgpZifGej{_L6V+_c?0x5OX*xkLAd^9=eQR<6!SN^a4jh0eztC*#K4ocVJ; z^A8P?Fuo#s7M#OJIasf==Ut0WIJ|%NJ2R$Hc7r;#)2>|k$FwQVoNPc>n0vj>H!vXL zzI*3<`kpH%dEfSp(ff9E>-I2eHXFWOm7I9Bc&So#+q7LUdzLNZ#_sK#FCN(2vuAho zf6Yq1fN%#*3|x&T#a=kNH|otdCbL+mSFavnCoe^xFJ7&d&dbkBhe9~rC-j5@aWf5= zf_ED7LKV5@U>LYl2+<_g?z1OAaxH;|ebBvp4T=I*UCT<#fwh9r#sZYQ1z5=lQYB*` znTryL;?q?fsD}K>Q-OQoYNcAg92%dl3Mr_imKA3XEl({urAy`IzM72EnBV@Rs?u~t zz2uT=t2ekTY=WdvR7&e4eij}D_X}?nUHpxEpkUMY)Z`Aibo zZbNQt$i0!-rDf|Uh77Uz`!8I#u}s+@o{CblOeC+`wd?HOv5oj=sGv-mI{nP~bCKcU z!$*yrG;?Ofu&Nn3xf4e`gMWInkKenW%v>;gcEhmBpMUYytU2>wNMUiA#WJN?0C?&1 zu3hiYy3OFn9>uc!v+vfHDOc{qu_HUTZS%H@(K*QgI-6TRW%~4UXD>7huldYVBVM2R zN%_!9nRz*rMvvLLeW%&m=e^l;KmFk2M%Ai+v2x`nbLJUwJfiePR1QF;!G1lhU5l1O zhCFVtTIa7_SE5YN;h*u{z0=3rTX=pJB4CvlrOco$O(SZJczVS2Pi6-Nmq+<`>ZoV8 z?bu15OrG)PnRDlxM${fT;+ZKkXW@{Vr2{fzh3%@aezS!%*xA^vI*%zKh?A%d#hwa-L*q-Crn)^f-%N)d*_Yc5o^ z@^5JMl(s9BSFa1HP(Smj7ir|q?=V_`pi?bati%_1<6v`<3~8EDrU1KS#}F_3_z!vF>1QAm+7yMK zWrGnxISM3Y%9IB3@WIV)o+C4dFfk&%krq1%#8tYC5Ex7}72IVof#&v zOyMkkmq0@JR<#r$ErZ01teGSihFe*Lq7#X*NCe%`QAsTM(+b{_;?TNLg;5q2kb#oA zF~BTjBOqRsK>k388Uv=q(bfS1T8vl=;xDwGqEz z>^e{rOrJ0vle%bpJ7iCCmqS|$2qja6a@O9JXMgHYtu7!ETJEw}jDdbqN>)}{>O)OJ zO6rKZrkeSuaxO~zf9e4l(Yxvp^=QFg5wv2JfK!fQNM^n8rY2B;Y}snHDq*kA24I?f zeA!+z0wSQ3a9gN&GoNrq0TPH&94aVEty{oMB9;+lSp1*hX$~>&gvnDTO_|CTCsK2_ zmpQOhu-@Ve_OMgO2ksZ(4@$$wqQQ+1juOG;b6Im+*s&DREz=vxH2{iDB2S4AKviAh z#|}$wuVQ7&weQkZgwtTIQXLmf3@=ljOIJE17#AUeAh7+o^7IeD+>;$uQSGppiRdCZ_Z|$n8%rHv=l`{ks){$FBR0O^Fw4yla+cRMId<>d2Gp`Ah$_e z;?a?C$bNx=w1}CE)IHCwD-m4IZSsZzR}yRWePJwux(K_oXqT8wxgLWJ-AXSrA%>^E z@zx{5hF7RmS^7}?V4#U^#Q6z$O+O%{C#*fh5}&%-hbn6tF+Ks^7%mXhjc$zSfGdtz zal6WGFGHZ{6{Cjk6fz}R%QYy>o6uPNjR3X+mx7)`1!w|8qh$R6G73T5SGY{A)V5(E zY716tN{QknQQpe`sgRMr<$aUHSLmJ}nDPa(MESajT>J#g1?+)Ob(Z|17#1?EpD;g-?J z`+mc^Yw0Ni20hA#93JwR6XJzKS)eeN*eDZEalix^4#c?JM-LoWxqMl}h7Ef6>xVY? ziqF4j8rj&VM5*hkX?+I_0@a<4Yz$kB-T@#Z2dDvYAP*++g>$FBTeV{RE3epbm(1Ry zaPL${ocJLThHJ+)3K19V%8{WUj^YGC1c~Uv)D(Wsfzlp|A2>l0nxfKhA0MDr*l(zd zEC*D1OlD#GRGXRcN|lgz&{q+kPhtXYz{WlET1tz1@691@1f?nVUW6M0@MJE;2E@ep z8Lx6Q2U>$`X>7>>uy8j zNCxK*oiKY7n;wlz>0xS3_AKMB~GQm4um4 zF0iJp9$kL)Bj{6UF-G3&Ld{1}Pp%&ued`V%MnqvSB zY>&rv;@HuhTesqN3c&$#5!jmySC6Ao+Ya@Gf4J}{3M%8M#XByW1 zq0o@v(9nvf4sO!3AO&gFb?}B5=`NNl1u3a}uY6qAkw+cR78`X6Zf#I_}^eP#@_X+`ikIhK~y?QJxS?Gar;|5hF^+vTBC8kuWhR zEF<5A&jv=A5h!~bP{k8QNfhHqpx8av&pWmmi}{Wj`&^^Q#(Q>dGnEV&`RwQ>E$%hC z?8su_{gg!_5XpolF1av(@KcnC!zLsc%t&z^X`UoVp*5050wVrDY4lBYPe5FblmIF3p%*)(9PBQ6KE`Hm=D0EkIXOPKIr z89qe~Pzu!e6vND$3U^pS*z@eKjem8)vSkk1Zp$3pzIDv$!)AlSq_eMIz2coI<1D&7 z(LMw=U8o_7gDY4?&o*ldiZXq1tEsbE>ad- zEATeCB^X+j*NIS?5XO8rov)(iA z)mK(8S$sJzM&5$Z(R_vPptlf;Ulo0GHHxnP<~`swY0{*>c@zFc^@&#W*Ihr4{XXvxO)>-_zEDwi++ z!`ihcjvlR9r#9dV0a&_d9_;GIwQDXMJ;XK&qW}Ov07*naRFvm-+{n(ZQmwkdma}5n z(q#)5$3{ojsZ|$v!ZGWf9ov>ITCj7+PQ1C6EL8@?WBHz$mOA&-nIV-b7x(weN=aHc zZ%)d`AwTN{AA__0Y2Vee(^adcJboH<0p<+sZwRZC+{BIy(=@%mXw&3 zot3rns}(V)Ph{ui`uY`T&-47r;|u0|_S2?K-d6AOAr*j<)k_z;Y_=Ucwtl&M1PFODw{Fd%g$o$#$`vaBeKD~y%a?!Nv~lCO z*bB6`Wbwd-^X48qe4tw8O1}PqWC#kv)e1k(=OJhJo~@fs96o$GAz|_S`N>z}>(!1} zw`R@qFP0GrBCJv*kOyycJ#ph z0mGhAAE-z<1b+oh86VH8&zCrJb2k0>!|Ih^n+=`{<;pHvuwd=lHKhXl%LkXk{52^l zar>4Zmn~k9dE-WC#qtC(!28X~<45a8)G&EmJAT@{BWhQR7A>>WQnzp4hEn|Q-Fw$> z{Ju%!CeR_-EznTAfKEsq)b-6b{>99>{c--w`B_8ZXis*Ioj?`g(}G#sUs3iz{pmls4u|R!(I&^@|e?UPl%6wws()16UTJf>39cpuj{=7?H_%a4n64RYdOkAHHAz?(}Iv#rzLOZGEC&cUc&K ztnm6oi6nGAyNiq-xOO=~mv)`V<@EgisI8Mn4PUfyo{z8f!&jbvqHp&V3+9o->w_QJ zu<|RD&H+!l^qn`U?X~!ei>FU{;hAAR2G`a#tB3XL37;y$CB4oLc(V**#h{8ivTy(7 zkxwt3zaY@Z`u-~~5ANIN%O#5gtlqCa@%Xw`U!fs$WZ%BQ9ojjvGyTk_F$4OZJ$Vct zW3y+zH|3dO@ZHH*F1<5pg282DG+2&sz{QIuyULuOUoTra?6F6p&z#M=k^20gfrIv*Zf54tu3au)xWE$Xqxb&t)#ppYLqg`yeE);#(@?(&sSy0Z3(ucDc9h7IFFiW2 zN`+v=GEJLA9zArJYRAXL{!P`I{yuO0Q9-L4%h@) zQoG$a*2s#FedxI{_jm99*pR_|)GoFHbF#i(zQW?~8&E88>Eb1m#*YJgKpjozj0pi$ zBg<&sv}t-=Y|Mq|vCofXs#x{53m2ltj2S~DgCAs9h+QHN!pF}2`^=a!ksnvCT-yBO zkJW0_mX)(?LXrn70nh8NjjPw88%yMeJ9gZ$ZSzx)K00y2ge{wX?EFxtwQE*YZQQ6$ zo7TZWeWfIhv=#Y*IA zK~k8hBCl|D>(Vjmr)`h+>b`yR4`o9`qPA`6_HftzyLY#5dmp5QS`6sb^QCdGyr-&0Cwr(xu~7n+q7>l@&*yDcm@vYyKl#)u&}VPV@BHy`VZfGt696Y z^XJcJDLft>(RRx zLym-+wT_nvr=8QMOm5V${zGls^zG7_2*Slnl{T0~@OOi0s?cAlXmae-mC9`_x|xthkkwEnK2zSPD)JNvSm|PIJ_#6C4Q+vv=jJ#YSmH{NX0sA1>FAA4i!XCSa_!tXv^o3r z?$dYRAY^{*eBez%ZO7bQQ~R!J?On%lBV&~SPFU%cD&2F`v#*XF`}Ckk&+Y%&NHhY2 zwbp%YJpTUKI`8TcjWgq}dLd&(E{ibSjU72IOhk{)gIb;kA53Pa-r^!Qq?b1_vv9hX zW6%g|P_MFr$+qX}c@%Gm5H@b2& z4*#^FUE5YPP3Q5h-}-?pqc6L9`~*zUxh7^NqUb!cIa4A`;;YX>w}9G6ZciRP^8NCa zwJV0zu2?>5!@9ju`@I}yx0g@zHtlna-g$a+y}FI#&L`;fKJ3!5_5k;tI&Ztd&!P9r zfn@2O=a21c@<4}lqcy`jz^`JMcliqO(Xj}|-FjjVSw%4Jc3q0O@W_CH^{ZBW>DlMO z+su>%Gug=NqIv;l%hE@$PmGJJUbzaJ33wtz)v|um77nvD+hDFzvq`OH?e!(fWqMh| zYSg`$kcd^N+T8=SMQBc^g*Uu6+bgh2%?3K1_tk_1D;p6`hecL3qCz9{U!?b*M)x*z zn*8lvzG1a%)oj^97g!?8=u^FV^(&Vy8|=1ohYo+g;zm?_9ff_s*!~ONmhS zd+uq49~YL>OtQj4FQ1Cj!;o=4DA|j2N498@t~c9#idC&!r}=$tbM%&6 zop0S*)o&zSvJeP(>reOHcVA@ns*7jMv?bx4Mr;LMeBt?XCysAewd&;w6YO}h)SJo$ zmxU}PC*y|Kgd_&OfUiXuZny*f=)qt9q-90NozVk#AIB||SA-=-kKvVMu?}tb4d{0xCFOyR9YsS;G`p12n88rBYIQ2NW$zw~w;8Z=n*6NZ{^)3| zTeo5T>Mz-@xR&R6`Sr$@Y*d}v@%<)OBO8-tW-6$av0vM zXdmuD&_iIi7NpYnL5GW_fDWlExK3lB8f<()2WT)Zwj0&0g-0#;WrBGbEdd~*sIQ6D z3_T*s(Lf$Mgm}$`?L%QvG?_8EMCSp$X_wm^SO%bE)6nCvArZ>Gnpg^RQ~b(GBR0v(si{ZvhrQG#tGo+xc_n$VAW+HFJ0c zR8t83N#3+U1SaUmOLe6xbf5&?k*raOq&}>2ULN|?Ywx_(yIbd|?>9X2^a#mH=RR@l zI6ITNfPh1X4z+k7h+HmTy~=o%E?b7LOLT^4yn<@ZMaLc71B$=6gWGpo)0m@75WelS z)WmI@zONV*G-}i+W>-x7d4i-mTmTWO87wwQ!?M#^x>7}Bxu7*)eUWqh>XLa21ViB0 zv11n%VKLA~q)3!^v#5apG0aO?KxEm5`yYI8?S}Q2FJC6Y@9w>OFI~NYIxs5H2*t^q zr8p9pPd|Du=lYdR8@@ZfE9(9Z9f8}3h#Dag;m?g4*|m3ng9SgYW`s6~b2g;PhJjT-msj! zbpDJjH#67a?C{V-8#iu2YQ@r@cWwij7JWPu656EsJ-Fm9Tej@EQzsl*nczK&lI(Co zt7d)n3F{5E=|sB4sNr=>kD9hP#6}u``>L!<2{}1Jeo%v>AR_LfI0#Bh{9)Q6y66rw z8XGD^C%w(MIHcQ^J`Lb0xq{X*%Ab8SGxq!i4Mtgd(0>BvkXuHYm4kQ*rxL6TL<)c_(mQC@O<9#gVL;Lq7TuvxfvP8X>E#}Xib1gpp7K%$V|O~?q{MUzM{_A^1zyHoXaMul|@GZ=6 zw_2+XWs#RM~tlBC=w~HSgaQMdl-=xEl< zh!Nq+Kwp%2k(-E1YYSF_YSuGtjArZ`iIVVe_g5G)hs4s3_f z6V+V9<)q~M>ehRGr5Z<;-Q2Az^9va-M&%`NH zVag`G`PSTpi<;K0*K@#t^VG+n-xjs2=l$&)R<9ACm^|i%m)K5(F$CRUco)A^aJWtHZBaa^ zLerYHzgfSbIEh#9+42Oo$(}3KG10xF>&1MqOTp&fR)cs8Y3Wx1LQR>dcxmm+)Yiqz)hR!oGb69&CQ^u%RQn z^yr0cBZ{F1_U+rgZ7ciCtS3avS{QEmghcQo#!zqpLNKW?kFYppl~=A(r3VHKe7sxF z`|oWTRH+K?a8X>GKI1Lc89n;+@7ZTi>9XagPM?8?f*n!2>$9%X8@~JQyYVl)AUuIA z7yizeR8;pr&;zV973c8>lKvf&(6(RsDv^aMh=u4HmQd$Pxq1ag36D!4juTlmD&9Jb z-DR=Q+8GYiB>q!9EbMKl<@$KT>T;W2L)I*NWgbzd20KJ+8>Y^!W1~{m)QCu#{CZEO zL`Corr9tih3@p2uoDLMGmB3p^^a6q^n2>m&YvR}qmOR{hh`tJO z2t65Tsd$(&8rf4s5aLQsO(}!R5tiT-Z_wxF=D_m?1V{t}J46GGLzeeUV2^?QEnz>E z3b05m6cTBg%W_kB@St9>o|@h|D+x->%_e%g$zt^tk-o>7o}M01vNVmx{4pmx3!w&G z0S^psh%}b2vr!LtNo;c@_=DgPf*|MywS(am-NekS0RLiqY=qOX=h-sTGE0^zjjlP8 z8SF-+tDU;+>|8&V7j$DD{JqWUgKY}V1`y-0m4|LO)QH87+MtlQlb)4dFJHD`!Rl4t z8hm_37e>SmbgvS)pdByFwhT7(QV69mqNu^sT||Zysk3nD9WtO_7b3>kgJzNLi^ zPE?lIwxI9PbBCJI9Q?YASDP!B6Z^Jr_4W5Z5dIBnmXVc(n;iq8QI$a}ZHI1156DaT zOAJLFVt@*BeD7Y_%eghXrPNflx>&BK4qb z-$1?5Y9p*O@aRNZ!nyZaZP{o zt^UJ?hgJ#$*=d!Ze;}j*)E14&qT`P4fxC|5X4%XXAnZ3gSQeKjH#03JF1ljn%2q$z zBni^7ZRq83Vt}J3I8CtK`~s94e4uLP@X7*TvR$Xo|EU7qaJ)T@^h zs#XcJ`S^Gl%*Zo^%b^rRq8aHqnd$IVe*OW(mPA-#cQ7$UAd7*!Y@m^g&XAgz^zG-1 z=gfR>=+ndc4jKUZAaXHc?FSD98=RM$aN&IAiXkrI+TeXkP(}isGmouay1cCPOEEFQ z!4)jUOMu9E@Z30j6?RkS%DPT89I9EYSTV3qr1AzVG-Wl-;sddY>`4MlZdZD03fFMV z$bG53#oON-9vObwnRDU7h4LX)@IZ@=rXaOgG8M}E1QZubNl;s! zxseN?s+f_LPX84zUK}S@=%@fU7>gp<1-TIEAmR`pcG$AhQ$_6(h8>7Snql=NKo2Tz zkk9xF=R!g&n*IFfQ2+^yX0Za0i~e3t#-%f-Duq>X`v$s9RvN4LQPGvQrKj3)5}?ze z6v%3Zo&btd@Xi<%1Pvetn*+5#s0-^WY!2l`L3YlR$)E@=QcE*%ofQRQdtsp1u%zCKyZUiUtVSquZi&K8f)6A_NX0H@zUt zv5-F=bD+Fv7WyXyOU^=UVBlq=RW0vG;y}rUZBN8|iZCO5Be5_5P-ZDY&ILyi$6e49 zcgp{F9mmc50HkCtxLxxW%xw`7@z`UJw~44Zaojiz(r_F)d)6mCyL9f{xXGELM`7c{ zGF|W46Sb{nc#T25d$x&)So_UN#KUk)B92f_^Jabg-hFjy4Cv9VNrU?M<#0JM$x)L~ zxK7=qSI2d19ND2!gBI1RyfAu1PU>}UR{vs(E#g}3Lqd1Ef1jQW8`an4+2?%tal59? z+Bc5u7}?C+()`=YkDs9L3euO7`J!oOL*ob|HkZ%_vj0_b*b-P|I)TECt>Th*yW z-B_%PKm~~`;ONm!9{)B`Q88>4hVZSkKa3?@)`VDG+;jT?7} zY}CGfWXFd5HGFIO6pYdL@7md{N|ix9y4_c+`o`~8({;l3(rmpRlaQ#bo151R@7L|& zwzVSGNzqCa0eZ0d_qZlbdc8}NMx7!XG_M>oX5`a(2!)g~4QcpMvi?8bd5bN@pLg$WS*7wo+Ep{0%#mwB4}e0b4A$W&Xt%6ht^a_2%_Ab# zfAdKa7UdKCgTFrh!A1=pY}AOJ8T-uBx#>5cd=GW*vU~eB)Ek`+Cv6uy6(zT3 zfEChGeW@6oqp**m6;F+#g2Kv4(R8v0#u+%R6F4I?k-Tsp& zo3?55#-#B;jBF7h8-=Z8KI%OW^zOa+#t85U$14W?XhuNW8$LDUEI9?5OcamkKXoN*RxLzeSY$!pHH70Ipd9YM?d4qOv5=8 zD{jEi?aUoJZ18i_CLTC_bkvMV)5ea>%e(<1DM^Mx0k`|XUVVN#dGga$tD?SL{q@pC zG)SJ}WdmG6OAq)m=o;SJWXrkeP0_Kt&&F@qf0&x}?$*gGFJsJ*N1mNF?cmw7V<(TF zHs(QEym7MLNoKHMEyV~-P@>9|!i9eN-mLKP$<4)^= zyKYd`Y!Xo>%lx4upD7<&3DIYG&06MCB~PC?X|Y&l&ibTJpFYeHcIRMlP_o97O_!a~ zr(a)W2)%muXJQ>Xuon)DN$znv)_$|PW{cLf>NP|R+W+w<&PPRMrKRyeG^3f!#OlQl zw#h5d-=|q*!%DU4TuZzvZj!)tnRnt!Y5kgS%LSD|ivx8955i@i056Nrx^?S!x160m)(RgDglwfo_D$@X0Lujd*6UdRwgK_3PJ`u2};e z@ZCGM>THrZY7c$-^+p76aNk}s)_A&I+qda5(|h(CXmjei_v(qL;O7JTgr#KELMxrVli4R=%WYMs1ezq{{FqYX;<%F zJ@60SyKf(gKCmYWP(o7$=67$@PF+TN-@bhj)A#7z51=}*YZti4ph%yhnqf2`Ql{vp zjUy}9jYv#P1f1_{*S1t(vGw15D|iTaOJcg>x2J$$UI8d`@#{lls`Ky4P|~iO53sf1 z5R{ca4wd1rQc_!R|35#0B^q z4sisCWeY|TF&EC+uU(J4_Z|^Vd+BqrM>ptfNIwl;7K7``rMNP+B5>E?GMK$B#xhmI zu3nB4ejFL4$DNX#T)k#39$}4Lu}V0s?6u^SiX}=5udb#e++R{c+}_RWc17)pi;aHo z{h5#hJ?^KOnfRRYxQ-n;+yb|PCJZuR&xH{K*%mBWI(+C;@T8Y6#Ru1`tG8HP{ImL$ zs~U#1zj7(4c7)Z>Kf`VC^$jdhsZvVPRhlH?0;2(EhdNho;-$nrTQ=<6 zu`4zv=A92dVbj=UGDJ3Oc_Mx8NjQ zs~*cq19VMiLx#EltTkxXFc@B5W6~vk0Wed9h`I|7$zNS7Ar=qviy(j0#&0fB(EQd? zG<+|(SCHdx{{M#({pOoRH~;5);I0F1+1nDw6-*AQAu{Y zeJy60#|R3+QJp?5Elqr4usRlPHUp8sF#1!^Wlf%(oD@>2s_6cDq0jBh2xBS38N3E# zcuV-f0|(Auj82P>uiLbRQjZgvrp7%Hl>OZKGfkQf0>{N_MZC`HqPA_%h>z>|@WV{$ zwDcQRbnoB~Maxj)=A>n$i+mguaC%ZwQUU`55&I&4z}nFtd;RQeczn@yL1Lm7`&ijs zycnAj7hkJc3j!}tH!5#bQF5S7ny43TZX?PudQa5$O>Dw;>DG<-Yw78k!~0wnvE4?ytViuAG*>z)?k7Fs())s_!8-0BOurMU_J5K;HR3(ZBloIr3 zFV0Kv@9&#u%Rw=YJ2rHjb&ldC{Ndus06`?a08#hu(yiy_-AA^bjw=&f{@F1j$RCve z)LP(4F?)`QzECc>B2gE>UieOuhXSvWEL-3 z+UCf!yAT%}K5)NNzbf$e8vf-8i)IQCg| zW)P7pIkm8MB3f0uV=(b-eSNj!%ZDF*3@#Ra-OtD0mW%5?q_SSXF3?$`Y;d;di5v0M z=j~^)JF;|?kDUuDANKsn(WnIC;|~o= z5`k8@e`V_pU1tQqJ_h}=g$o{fVyMx#n8EDh?Gu=tlZoyoBL)4-$jmKPytHU<3mNeW z^eL8;od*d4cH!2uvUB~5l|m*Y`=&HVw4%K_Jlu8Do`YLXohuzw`nfUBun`NWhXz-Q zkGaIaC~w`+BmVL~eBOZ(<}V*Q6`E@xt5S#oC4W?#zgv!ipDH)WP|E?LAap_f2}pBM zATp8H)qPT`2FeY}V&TD}OMd+7C@A9gqJN`4+Pl0`_^0kb@%Z~y#zjB>rakamU&-IJ zDSxcmDp(g#AeiOO${g|dquxgSoKHWNJ$fcRvI-*CCTZn zPrZ?HBQ7qqa%JWn))WqxJ0v9Z%!%X34hS?BfBrPG7NYTjCZMV)RTDnZh2zro>#T7+M)b!`B^0P*8C4@e_`mY+sWBmBjSexJqHwF_f45sRQ_+Ru(`wrWx(p zwM#sAIwd*LjD@?>Y}3G%5J9qtBGX6{w z638cnH2g^;6#W2HX_0&n>Q7o%cxc#H@Kb}me5@BXSsoOEyN02hHLhF(S?y`zBhqVS z6=W&@Ybh1POX2zX%Y~vV@|nnSQ?~i*0PRKXY0>$v>VaFn(z~JMXbZETh84ACWjy`( zz?{^i*IpZkPoKoNXlG`c!EVpGcI|rNRg`sek`ps8U&2hTWy_X2|9~Y6=6gAuOBc@b z3#rhcaih%ily6skNyw4TogO~Db?fncJKR|*3unF4y5FN_j7PLBeqdO{2yS~$=G6Bk>6Zuda35~Yv?i+v-zcwWZp;Wgr7W56=fVu-Tri=~wt*2AL{0(W4q`8_Rk zKEBJBF4EgGm(Baow^H?b4IVz?b*ZxwHL4D%EcgiMJHO zZ)^M=|7T~UUX4AsbMuBzXMNPXLx<9(OCSDu-@n0UsP|{sCq;l6eQ5Gzqe&`ov(HAym|iaoaJ*r^$V^L8QC~HEqUEHU*Z^` zIllV*d{-=)Yj$VM|Lnbh(9p(Bn!(MBpF#LsqUspD2tA$@8@+Yoy4kbmG;ZIiM9H$) z;>VmnSGQJ#_MM9yOKA-Oi0JpRzJl9U3njG*5&l%Lvha_$xOTF$fFPs*OlgHC^j68p zkTinTtYi4YVFGu6SI)?}ctkr({d@1osQlt7{5V zVjL|iEjJQqoW6EeZxo&Gs2;fcWZdj+!HhXw@iEcI_C_IF?b`l+nGRl_$sf+@@=)Ig z8b#_{Ik1c4M~#GEeP{XlcCA}}JoBSb!=IWpV}^Hdg}L+Rfh=cEpL%QTnELf18#HJ% z{DtQS_vnsCq=v2T9ryYqEZ1ZL!X1jVg)Kn+mNlz4fA?)bxpF;w^cwKklcGzfM9c_1 z5iZxSUw`=*UyL3zmRcZ=_;T5@$!|<$I+FvrV1V=SXP=FFa>$2oO!g^TcJ{(=v4RG9 zXFNN$cEe`XDu>Pb^s{G&J@vsGQv-u5&Yr*6X!6d5m!+8k8Q_s?>)P+Ou3zt4HmG-x z-u)kY3^DyT-+V)mSr}t%62-_6%ML)pVOzOy-nbd>qCDd!G#a~t-ahYtI&18(A)if~ z92iu7-r}YB(mHVBz>G1YBkR_wRJHoMpU!=8=p&!K^@e|$67!dQ0U1K1gqRzC8>tec zX!j2r9^CLvvGSo0b??`6@RMlKty%Sz$;V%|7eE}bVkPhd!P9J?FJ3Zr>U3Eoh}^|( zwVFSfJ$uy9Cq9}!BcNP`ISW6>;OOLuBX2%8wpQK9>S0x8e)93C;ZJ_}#zf!p73a-g zE)5nkNw>*D1dYLM>o;uMxZbB+`5xW64}AP7m~4(+yLS$L>~SGWqH+gp^OBm+Y6lRW zy>N2Nl@e<6KWSv!Sp5_M82a&wRhK3Rn=adrB$ljH2DDsms)JGBO6Zi5O>5b52F0a= zRJQ7^{PfyO1+S};)Gd{kmd3mZ*{xo$jI@kdOUu-dy>J)&3A{H2kFq7PU#h zw@&M_^!lhPx>NCq4XOqaFtA`s{Lw}WZ>Tnv5O=U+R;pAf?r43+HXwU<${d3jh7Oy} z#@^r`&9@+(8PyQGou~+}z4jWh3j|Yu2nrDI8e-}K{|L5{CL=uy6GVitzztQRbU!~o zQt}FE*g@qfVhprpcuT9cmXZV*j;lQ7jFbeK$=h$g-Jn4OY(RJy1r^Q`&|V$9%b~Rz zEN0EYo+J(DCa;rcR#vvRx3^{q!izU<+<=SMN~A0;&2N5IJnHnoeKTgvXxON+G)>5w z=p(9}HE5EeHDKZq#|`{`lqyA6Ajv`iLaQm7cS%Iij7aIyJe6AAT%v+b@KC;OK{r@n zy&gAAAAR)E`t|Eg-rkH?fzgUcb&8JP>j6<8XFL>BFIM>&@v=TscQ_+cOuLblmXg}_ zo{(b1*ZA}E>-Wkny8q96;AT4i&r|-7KP_m860a6wfZ!m9;5q0dOxK@n2B5%O-~yJ= zAR~x@DxQ`J>Lu7Eb4>z^#f;|3pU)mv4&k?X^3eQ(t=GRh) zPzbc{c&4C|f(7a*f-_jjK3<6uCAikIRow-95`z^kl^;CdGqFt z8aC!WX?R!s!*Hc|*P9aOk0!0duPlAeh3HV~b((sQmJ*W5-mF$DtwgDYYA7Nm={*5$ z;f{GTGduf@=`$uxnS!o5JpgKpW+85CijIHU16mKN#bf?gr4d4zQOLB6EZX=u<(~TV z{%KxC5C45VpqRUVUv__0PKv3@=dH9l$aH1a>UHTl-QMfEqvum1!%G|S@lS++KXYif zKvw(#lo~Q>7XXStqTrVlB&ea)<4@(!q8XSLzT4>q9t!&j`XT8AZ8-QPnjxatp-R^v zO)E=L5h;wHJVkMvL#vfm6YdwLB!4zw1^$G8l5#||L}a33xdRdS1Jr zo>~L7pITX*xruN;U(Q=l0TEu)KH?MQ+E4+cy+F2@2r;IJp>Has=<>IEfRRz!jeKV6 z^G3waHv}^SQs;Vf>FIgt$;l5js|alue=$YJf29X*pCkX3V*h=G@GWU*Df3h|b^sdz z0eiu-u_{;CPf`1mL#~mKaGzSqZ?4EuU28dTM%hE*DVd2yxE-B2 z^hlIcNpZ^LqP?4+gUTa66AjAM9r$r+CTFb_&g@2VRvn_9f43iak-EyOv_<%6SwRSi zRYgm|>d+brdEwAXp*ZqaMF@4#>Ok{VHRQhJtVXJEcTNbNmYveYrhDjNJ^>B2*;*ib z-YPo&RS(E#B>(8%f{_Bj31%UpNg=3gkjMsYQD6S6p8R({|I^VgfLy1a+KV)A@R-;T7L2&*k2%KY+O9>X(cH|my zwk-Aqz%^!Bq{gI{`}vs9wyPJ>R|EhhS02MGsZ2%0EW9W7HPRxqv03)YZk-Sf$y4nY zAQBgfBpa4I6z*K6u-xKW&h#q9i4>VFP1I>b5P@si{0+Ka_Xfj3Uy&ORXfXmPR4O%b zISna1q-OBul34gWlwL&CB|WSxsH9}bFkCbKP{rF9_esUJsKym)`DiL4T&ot)arBvV zpWDIeUSW~k;>|zwCEPxCcgG#omvqvtz|0mmv96BOm$vVa#*H32M{M-KIBbNM&<%U zTU#_*9 z@Q_EcImuHFt!~5-6F{T+%6|xHM1Zail~S!xw9P$X^QA?SsXA-zExf~1Rmc3NTK;Ss zlBXv6=YlkpUYJASyV@N2-vXDCO@% zgz3U=SDjXL`p@>j-=5>vW}b8sS2urOYF=_1J`H_{%Ef)sQnCX|6Vm0jZ}7Gr)=)#c z&HNU)RJwgAy72>e7JP$iRf})6bauj;FR z{XJ-z+?{7_SZF(BQY4Y;JC{Mg*$p9&47#+;+-l7$0^Axiqn0msXIK4KTP^edzgpnG zy)adMfPh2eu=1~k5f*?0iEz(>Df)`d$3nwv<822a7NWlo(L2we7mXU zq|}C2A)V9|?4vBs8vIF)u&7?x8>yK|qf|?O)vm%eX|zT1pQZZe8Kw5t&*j7?k*%Qa(QL;z3dgDQN+3NV%v*O7~ahJ^JzxKEtI5g0a&;Mp9 zDX8Vmlc(U`o$$Xl5E`U{uo}qT0_cCHmD)F;^T;M@?TGx8lbX zhB6oPZk;YFQJ0oxLolb-_@bM}^k${4tT?n*Xgyi*y7ZnhkSw@S z^#32|0X8AEsexEY7O^@lci)PMf_`2)4S^OM{m zz^x6GnzQ-1#`hy1lUlr?IRIPE#N|)CnyQ1uD!u&p3qQ7-i!Q9sKdI8qX;eG@^+~!y zweb(0|8!q9f>gImdvc37~|Dm9} z>J_c`3VTfZNms*YnZ7fJ^!jXn+l}I$%)tD$)UQYCcQVeueQSUIv!2vm_)oX@pV#((|0(;pYO<(N zD41;8NU6%>_m~|@xa~hTO#k~0{~xd6-yXR0VLHE?wIwo&fbuN~7K@UYJY&VjHn?a? zh&u59WA8iwtthVlzwNeHc=X;B6nn>t*s!5u!yYwOj3w3>OJa@Ps4@0fK}1Cn>Agrt zL`5t}mEL*%mc6_8|2Z@FUceZCn!k_0dplhA?at0lIXnBEGiS~@vpMK#Afl42RCV*6 zZFIKPfRVk}dL;UBMYek4$2b4&`qxWwe_j6nR^a~;@iX<|?SKeac!CtXCMECw)$e7i z+HWB%*%HuobWS9-)f4|m?d#xeJ|)qTS8DW1!w1Ln*b``eiHWRbMWny#_X;fo0N*B^ zWQPattQ0^IOF&#cLoF8(`w%ft!Hgu$uHMdOO#;o2D?}H_*3W8y*B8O@e?b8TQChv zv0^~#D7=tx;OyXpRg)fHp(vl0sNQZhpUr0yocz`ls3pMt6JKh!d+|qXN|?g&C;KI9 zM2D0jn%luRZlyl{Tla$PAV1An1K;@*^?u~=U)SzGJ-Hw&aQ8P9!`IR`4W6@CfDS7d_8z4c$ zF^5Q{iTO_-o2359pdZ-kF~MRYvfpG*sLeKA60*mw9#A*7-@}en*Z-S}0s`vgZ^Z@U zD-cUf?4?+}6kug~Bp^=0?8y3mXS_v{%^jDUt2h3 zWTBKN`6!W(DJMgD;e=pz7G-~J-a+~ukpe%VEH3u-d}tZdO)KH4c!1_B+!p@^sNt(2w|tI@uyP@TBJ34kGR85) z;{RSx%YtL=q<}0L%S=mp1?~o75oM$wsj~>wsF5HG;k6I9x5Wk^>^Y+GS$L;v!zeY3uhtq%S5@K6KJ6<< z(K!!@zQA-r$rrYR08eMAHvBo)W zwQeAKd1=e6GR(EU@#fGkLJeAl3mf3~KI5=d$%I)f??tyrYZu8PgEpcGj|92EfdH%9x>%L!7 zxo^8>hqo(W5z?VMso9%w1j6vCzqti)by*7!H4fY&w-g|@@Xxo{Fa}Os(y`M%cF1Dg zS8Z#cCNqqT1SQkVtyl(A<624q^};t#u7op^tqMh)jWz&7d`XdQB-7ltqgG{xsiD5LEyLfRdy3;<*jC$Qr=--FG@fe}#7I(Dt3L(s$jI||T8&?ZS+M)USNeKKcZgQGfh3gdu?ZiR$I z`VoiP4cYRwY=bs>hU3bbq_MorNO|dKOq|yEKs2x|q7lYQ(d=fr!;9p{b#Sc36kt0Z zR7cXL?XhRENKLYDTK#FU2q>^M1!U(CVwC5rNaIFd%r665)KX-Q zB|t?@Nx4CA&3Y@Kt8IsEF*rd!h|3m7>|nrW3Y&!zJ)bOPXJt~Lco6Ca1@f<2kI+cW zB!-L4p%#B`n;RLN{dW{F*^}DGXPl7+oeS!%uddGa9hNAwtCZjhDdeCf#6s>DF9Gse+sbWyUV|K z#&h_k3|zMu+6saPJt3gLmJ|@_nu2N>?;}=eB8`|;Aqs#YKS5jpars;0=q82yZ$gbX zP02rG?89#m8kaEI%yjr=sVd|bdaG$Ob7ZVCgb=~8)>1(2)~tGkf@Tx3kiawTRdZqi zb;Kr~jOIa42q^Gl3NTq`EsRbBhOltW=GUZP8o!ylR*6i*81uS{rJdR0mgcav}DST zq#KEd7LF*tHpFpBF9KVH53kjs#(~l_#48^^W#cSEr7%`}l8O(PFOxIHf-!T34X@SI z4jL!l6)zp@8-5y^wP*y=&;htjcX0f)EqVjB|MLqv#IL1*GXD?_QhtDi*zM_{&{}$z zbR-E1yK61eUE{#5smjr|Z9gqvBP^9H0B6TU5l- z1uO_n`7abuq)@HNCeXLID>WCLvk z;YyJI=x0GU8z030b;h>vpT&$5ttvGvAq$FVPc0-BlvZ1U2{6gm$Z9?=G}%&uf-_($ zer;u7@LVZW;k6VkSj&Jy5R8?YY!S$Y%BlLPSUFSpPD7=R6hsu3;vqZ=0Kq0(vm$a3 z?En%o@CaygPz7mZ%(hhVxl$AEX4H;=GSWz8#8aQ`GDvMMRyBtQRSfZROa0UUnXNRH z&(~ilkh|n0AGXXhA6dswYVC+r$KPwv{9j6eAEAOUq+o_Hwdg@+L*fRjQnmZNP{xkeb2R5(Rxtoq2)S#}%1 zm*Rz;^%)XOpV>$z6S72`7T%X+_*BxUiV+I;Ro3!H0+>pPQno;<)LeqJNU!|!h!!$< zX(JTI!GH`Kwuw^;wQme|6-)~#IfI?EXFFogtTYCEys4%c$$W#FkfUiiNtc!*0ab&I z8d~g}ZyEKFw2H_1gMOzx!zoJoav}lZ(HP;ARaN<;5nNsfiL!968==QJ{iw=GOsZfk<}N1@*u<`QB3Lm|m$@g$j6S1XnnVt7AMJlMiKwnD8Tlp-9RSSQp3j zWP*Y%`Tr;gubPFs9JJ;Bl9CZazz<&8HeBQ=mJtdWR4g2JxaGU)kYzC*fTQHBei_;T zs6bc0NLd*NSp+i$0!E8c1iZN_2q;(RLAi`2GQ$v57TDb?=xRy4{8UvGC<|(*z%Q+;#ND2hG+LP}BC!=#ZPCTOQV68hH`Crj1Bg zM62M5ZHGWQ;c4N1m9}}rbR945dud|jaqtx=*8*`cFm2e5Ft+&o_dVZ9r%Nn9oJfRV zI5iv#B8z-N|40qD9YQ}Dp*dLzkBj8OOe(a6y9d@}Nk|0Dg7*l~)q~O+%9S5TYkGoz zrUPCf$04q354@B%SuIxZ(DO&Dw};IwQ|#yVJEBLblu8|1>eovuwhgCj=MD5xdVde0=Qws7cjpF&H-VK z+m;0=O3wOW$%deCPfmz6sXGr!Y&MPQ;i$cmm$KUxzZd)BQVr+y)nawe)Hwbz4zUx+vzkL)<7U^v?ta8}7U zgfo{Sr2^b%#WD-U{3Q+y7h#bWG)YZ4S0(+ZS%T(@QaVlDJ_f*Dx8HxAZ@tD4!4!d& zY;ZE}*R!TP)awd(&gTXWYSCe@l|L-Hq5F}BA3yhjmo7U0obnB;uQ|FaaCGy-56tTS zDguSF3ofbOq?M_J!sLR`0B}<911AgKSi%@*(i>JUpV_|;VufpN{ZkSFHGI8-DUKR^ zLuG64zUz)fpG*)|n+)vex%WMI_F3mG`)cln4IA3*xpz^c#w-gW*$^(4B*+)YUIvL= z?WIXd3=X_2gw%MarO8YbSVgr_*{N9n#y=h()2Fv=BZG(pGpfK0f4k$&YJe^%!y&Q;`r{TnNG~KxG6HFFw+F}4B1huZ1E~D5)kyM+|w{k z6NpDXo<(q}`If_so-yParn1Og)bYOoaM$q;5Hx=?Uk8YQDS{*!<6OnzG)BctRut9=}l5egu^`fH;JV_vI)5wrJWXcEh6AUwd=q`b{g=m$%t% zuZZv7c1{m!_NV7wIIP=ADL~RgCl1Hu88%&}5YH4Dd)UhYpvWLVVs_w0Ok*GnaY@X; zK=~;vB;xyY;?6(cI)CygtEAwr2Or<5(;l;D&ak4fNEn3|#)A|@0Nbf|5-)Hb@@^5; zhkVx&>4q2a;{|@oPN(pk9#PT~Kar~E*qIbMk6~fc(QK6Bpe^GCqZl<5$XAfM$(5*p(kzA4vZoCjW~uWmi!DGO;Pw_DYXIC^5F?!A*H)F~tPDydPEU|dpC^PxMS2JeB8a63u+aX-s%C(EZ#Cr+hML!my93Hmu$I(4%{teSS&noo&;cH*3(jvF?3$OrEysw&&JZx<`6->B`*7hZfBE1Ofs zj~>{scj<=pyX|wpMHgSzpyS@3O_?z7(}}j9nm2D=JmnpA!l}pi=xG)<5oTN{r3U~R zQDt6I_qym$wm1OppCn5VSyw2aC+5P<2`*@HgV?>KjL;EBcl&n42h+xkOqOl@e)1=V zR|)fK_^J7GKc6viEbQSCM;&8^V$AM2A2)JXapR^xd_Hr=(1CAE`4qLfD^5Sj*tpIt zDFIT)_3J(E-G0vw9T(MRe=PB3{NBA^AMn&;M#gR4s>P&%?@t;z;=>7(Pv~*>oT+zz zF@EB`_uhBq4YxFD*+!{~GqAv0;}}`QB9q3Cd*#W;jZ}G)2K9!${?deDL;4LL|K*oo z-t+qlP=6_G({|R_QHIN2fp(AcFPuAn-h&sP%haLOE*DtAgjG?>+&7030HFW0Kaku~dsV?1)zxtXhSARLp zNT-Thw5V9If;CHf|3kt?`j(q-STt=+IA%wk(7-;geYT?XsmCAvu+OV~9(xpWHr%Pr z^ueDB-CDJ~?1BsW_I~x9Xa4~Wo3v~(WxzXAfcEM0=FFHp>W$|OHy&-;Dp6Lwh_WKd){HPZOsGTQBAnebRA~>mw6v%PhKsGW5t`uJj4Kxo7bN_e&n)6^Wqy;AAH!MX~fJhg|^-E4}W}r#AvMl3_m${#-t(dyk%@! zjnQG>F|$9OI6K;;C5kD7`oEV+vY3Y~!S~8b2R#0W;a1-Hmpk77puemGx~X|ze0Ip` z7hLt_3O>(+bt=w5$&TvqaoWHgg{^SNhZ4I7-)CTwEMKr7WJOxE?%&j^YGGLd)l~{8rRk zh&5_9aqOsO&6^(9^|*tMIw|ZWCXF3oBFHT+*k$)UL{|>L&zSoi*2Q%q9rxR}Y1`K8 zR;*mQaDGw4dPg03Sf-%FwWD2*IHI_?Xvz2Ag~fV>!N6L(a+Rzt`k6=Wxwq>gS|pa|rVArssx3bE3SxWiwGZ;5eGfa*EnT;2-q%YPelLpO z1qB^;-Q6>tE=L~KxJ3)>ffg^AC#^8evrj$!cjsJW*afs`#^k9>zy7LX`Lcr#I~?JE z=fe*rPjtdpEn6n7@H3D8<=$fs>Eb2W%FvrVYbI(p4O+F?=kO!#l13f&*gIXiv10K; z7CX0uX_*qqP{c0?LV+JoMgpn?6sS`aP=eHKi$zgGY29&++O(Uoj_lTb%z$_1fAdYK zsQv**9M^j1-QIcfo^sP_(zad40}jvFEGHt7;33lqM_?M!_lUDVX*xxPCCRG9nq@0Q z#Nj8GFI$}P-Db@iq4q3U8&1nst%dbAty};0aK{4wcJ5zr$`B zI}$O?bX7%RLDWbk4cAL-+!SuyY~_m8c0p03uwJtk%?(rnGhV!+ys)rj-gn=u->?=i zMT1baBJ|I9`#S{3!A8`rIE-@fg_dEcO%gPM4=HthuEZR^_mAK2@l zgGqq-Phm;(zOTPxg=35dDHIf8{6TDGEyP|(Xx#{TRZ!TiIPhFdUH%(z+;#cw2W`(& z0A)9kb8C?|vEshNwmNq?s$%te1iNl|`F@9WZMo}S(fW-HE3$6Y`UAThgU=BkpfQ>2 zJ88U2nT~76yl`PW;~aYAF>Y1zyKfeJ{C=OIAAj)O=W{C8Z|d6ZIu zo6Y(1t3*Xb>H77Vs_GWaTlMYR7yC-gW71CasBR}DU4OuUffGgyd-cWV^r{}h>ux8X zQdyZCGjZz1^2)hi&l^2y+Tu0qx}AP@(H?suz3Fwy`D2F;nmuXU(+}VG`YSK(vfEyW zEMSYDdFGkP6DACPx1SfScU+IN_Bo`B0JP`7_S$Rx-+OoM*WVeH@w6T3xX*q@QHc?b zjT|$+tU5V*;>7XOKds-Qb;pCdgc>z}^VL_Uj2k<+Uq2%r2d|xig3c$N9BI~K=%^88 zm6gLMOd2_PCN)g@R?>8=Sdq&bD15yGJ=sf|DT9Rk8&Q&y0Iw8LAWQad@F$=^-K0Pk zAF_>ytrNvGO#Qqx(9s@r1F4;8{eAW?XwuYm!h0Tfnh|bbSPk|$?6`O;16~{9Lc=Kp z!F@BFunOXKaXMUlaJQ4syY(-IQ}q6Wcm3<0yA30J?w{^EuWS$yr+K!ycGcQ12M>DWii_{P>;k0Tdv`nO%4=>;8?jR_yu_$V z4t?og=bv%*)i>V8gYUoi(!}@PGn{&cU7Ro@C!c*@VbfOeb(@~J?N96Geyt=JINLLi zIpx$=d+(oKzv-UeUGmy(w~ih%0yV;f8U4p=Z|`+tccY@>l{;?zd#~Tmc;~gxKbuk1 zq}83T_BHA?o;YaGBR5=YtX{T9*X|cye|x&P@dbB3Kr4p4{^q|PdDN(1-)P>x!ic0} z4X(WJAqKD^ufKZ7MLj=88oqpSrDIVj*-^J{ zH9fS(<^3sk3O0;M^ULOLbQ`s7jH5`n8Ck{81wCEe`U}e43;xz-3h2HkJwbN}K2&$p zx_{hr)0t=W-2JHTaXVVmtrtP6+R`E4CBbgw)=tL5g$Jt1rW-G?apqyfLk&}gT|d1^ zxkQD(jwF#^n3-6*l8-0>A1#n0N^(>i@Q-?NG$l&cEzOwW5^QrK#R)pPnT{Dr6h&OK z9tJH%MtsAnrRY4DG-~NZ3e#4k*o`MD%Stz_M5(%7v*xae453I=IIK9LjCf^LNxcSk z*hmp7D}ckIXhG5rRXQ=2NgEi+^3^N(j>Zii{PJV>VP)Tc?h?<56=dA_#!cyXyk1dp zvR-{OUSO6{$Br7+b~0YRcC~Gp^&2+Ggo~=ILbSJ|nMB#z<%XMRSigR}q&3>r=%0u1 zJ{zw{mX(IX;lkn)IASt^k#=k|R9#xWZlx8*%CynSXd@Wrf^^b~S0^`Z3a68g{q62A z=6>C3p97wL`L(n!*j%x08O(c8<7Rfapvoy^(+pM)&>2q%a+@(9x3r;6;V?pW%>Oa7 zL_w36vbnE7XpBfLW4&ioEL(cZQEeOkxR+>Acx~oN0~z;^-+6Goqfh{r16>T5Ft{|1 z>I5SXnPcjOD0Q@L04y>ZNL?~UJQB);>N$m@iHoR0BWW5NsTGo%lso&A@n20H)v`eWEq%LBU!%DBVcpM6IVEm5 zR88j1>LbA~rC}D0z_5~`z$wer_ha>qom+r|tHfQn_%nfaNh4yk0-BArP*R35rSk$S zT(7_@Y?8E{a#dTIZa5r_ih`!rPR-p+dKI#MAHjcmK;O5P&zsw6*EVaHEuB7Lv|$z< z*ZrI-qa=x-9y`tU^`Q^W$I01*D0V1eIpSRfdr!xxG751514kLgA_u~X=R#gi4!Pw; z1tD0R4En&1Ndb-2V7G>EeG&>`ui^ozOP4HZ-L`EQp)<>Nws^|kw08Bzb?ZCrvKxzk zw74=E$D#ya2DL#mY`~pxG*Yh+NJQ~Wv;Xja!sg{1A!RUDG;IvK!9Hb77)mBG;gI;u zBNbjs06mQ{zH3B7M)}(H#RcJrU6}Dwrk7f>ba|VdcN5lGd1577O2U-fcB~XA&a)?bBX4G;qqKNmV~A5R5A*KKhJv zd)@RW?KOq;(jssfS15RMm1c>PSQ$GMSL!l`sWZrkpFi$!A{6GXI$;x(u$?I{-LQJ) zidL=K6%^KktO%6QMz5+YU$tsY8>k%(%Ue+6Oj)%%VdB&YAFY(U8n?W>*Nw+_KRv-H zicUKVzj(INIa3pHSx#9DX=Y`4X$bS4a7?_O(@Mc>p_BYkVe?rorq2EiIPN<8>T6ZQ z1qAVcnl?WD`1NO>cuJgK_|_%2+}87)bJJ<}_CMdcVAL?9r~r+Q-{1b{6MLTjX75)g zJoitagqceLqY%QIt;j2i%K)zL1p!o^jE{ zaFO=N3M6NTKgJQFMLuAgY(oP#ulRVBEyR~9Rl??-0U%?rg-vMV+SQlzJk6*sH>x&u z=zQoyFT83Nz%M%)JB-&6@!<_4%Om*80w39>2E=COA&T1wX>c!s!zmO3@h8iSfH?$F zCL~Fb4jh*u2{iZLd;5)nk3M8Hs%NZUdq~goZ@%YojMZOz;UA-4c-9Dq5SjG4|Doeg zJ|)gz>6;f^bKUt@_rjnb#+jkeXFN3{pt0}zkqGh-nGy+s8I5pg4kek;xaq0GhJN(X zM=$p2=g~@Dp!87MsFKQXBqORubGcKu0|X30K!F{P0)V9A)THM5lXyJ-?BD*{>*`BK zO&oL8Er0C&apQL;PyTqqxGQhy_2!e0BvaK_Ty^EU)29uXIb+E5Nq1rh z6N?<#<%kT>DM4^8?5D%N8}c}MVJ{uc_?#f?M&X%X0eA+3ob)2N9Kmac!S2SwHcD+W zcmhej`0Ue1pM6%F_U#rfT-d5vv#!S+P0Fm!isBnVM*&0;2;lm|MHU?Psv~YS=P|bD z@)L#1^|lNT0Cg%JMZ}zTHQb9K3JkC8f+oNii^cx(_+x{|kLy2r?847JA3kw%DjZFR zqt&<&%ou2#J8nGWCL#zPJl9MnW2!C4tntelLT17kK-1E0Sc*u7aJs6TzQP>K1`8mj z3@4eYs`x(#4<0;n^z;4t%=qvETdgF7Gk%7~F8rcLf~UeA{weK=iR4Udl;oApK{ z)ESm2>$|*UvodRJXEn!pIR+M&1Fm z{1K0>U|fh?=R?48*ZD56V-i3(;w=J~GPUP_WxPp~7Vxe0>Nh|T=!NaH6TbfTzk+oL zt`F;S%&U(*xqidOCT(_t2ZSN5ms&Gm(8oufd#)WTHZ9;<_*wA!s~Ho^*R6^~3a3t; zyvOdlo^{q4!$*vsGiT2JhaTGH$YU}_)Q`tMn>2CO^eH7x8=cNvU?AYRmQQYjZH%jgf^-mSwb*#0zxZ2kWj|>eA0x9jcdb& z;mK2{bn3L**=L?Le$3dJpUpb((1W{lJ=Qjh*w$_C)R9xCPVneBsVNefYt<^$KHe-2aywufC#hpEu!c|9RuJqlXM&o$>Dv z-S^fDPw&2SYs9`~Wo7WcaHlxYOS{RX3%+Zx*PhX$V#-b67cdk1nL7FRq zZ07y0olcB=?%CU}y2?u?zxZ_eP1pV5k2hSCj#qs#eexeKx-eoH%jVC&^X$`;r5jo{ zY4r6sb3tU0db61s*&%rlfjj0h{;D+_#!dX>(Z~NDjzo_;=2!#|Hj;7QkA`J~wIKuE z|MOLsFPuNWw6yfWD=)sO*X8rS`zBFY@!%yFty{bhCx1^oc>n7!{&Tku?N%;dx@prU zXiDWoR--b3^+ipFVTrBEQm(&f$x>teh64`RFO9m1Va)t&_Pg)D2f-ZMFp|vZ@p#A&_T%khYlX}x10XZwrK;? zPb^!xBJDVP?YF7=^_TKleh&vNx<)Hhbjgw`YB66_!-5TQ+;zq%%%GbHR6Cciwl;vwEI; za-<|>qjzip`qT$Tb>aL4yLZ}^?Sol$BpYDRLqkPC5K*v65!6{Wp<%~LHQ=0m+0_?c zeDR4VpL|o#vrjnxyz8&JX8n>ME>WmsZc=+3IAbmdb4}XY-qAYsK1jDr0a4lv6-FC-=4!!lh2djPal1~O3ru)~& z9>u8g+{u%^oi??7>sCv?`?m7|`=8PCoRgzPE-EF6pppK->!Q|zexoq($i3Z2CX=kL zy78tPx?Xfa^Hwb}39YWEeC&aHjfHb#ZQC5!`5?w8=tkXWBM(_vjYgwSz4!(q^tH>D z%^fqg@26jdBe6~EHqM$lz5AKxe!JkihAo<(diohgy`~s0CDKMwy+WgY!ICB4?}~Sq ztg=s*~HA9fIQyMgg!-_&p_o)klCV9#8zC*Hsr@ef3AvXPk8AnKz$v z?z)vLBk+EHdezc}w_ktV88_T`ELJB-h`|@O6^mC5?DuZZ^Dh+RLsrbt$`(Zrz?Nxx zrsMwm+tC6eT(Hy5yS6*%;BaC6q#5qmt{rYh;Crv^b?JDDxfx2 ztyS$4afJA%af9&{+HiOpKH zrZ#KWtm=I5fh`a0eAyXi+;`7k%QmcGgERC+gEk_-vt$$pquUU~VDapPVdIB?LQ zA#c3)dI&>AB|?Wh?6t;Z0Eby=1VQ0gr#<#O^zb8)JNqGLmt8t;TDuZ~#FbZH^T~$; zPCx6cF(Zc|USe1gI~Zy%V~p6vWPGIIW~!?EPdDDswta`|Z~hYl6*mft3qBk)u>bT= z&phY+doH`QvUCHDbMX~G0})r*R;PmwG9CO_MRwn1&o=uX9EugOYP!qL9agVg37OC8 zdA4VTd!F3=U$4GgT~)?Y6xVF0c5BwGLT_;g>yg{Gx&d{zExm)%ep3nv#;TxCrc9Xv zzjwj!e{U8RUUl7dc)tGr%UKpyDNDb<;j#-)JO8ZTU42blbfOUjxLC1F9yJ16k0Xvg z4x>I60fB8GN|ZHF{WCmRZvYzn6$wZvBq)iPA{-PKKkz22E5?s>K5@zU~I(0It;`sfDyQ#6C zOyEmz+_c%6Rm*O>$7FY|X0G_&j3r?z^de@4VgZ;wv+eBFx+2F2QKw zE&)(jVe}#x1Z3)Od3Gqt5z&Tiq(Gcd7@l&}xN(SmFTV5=qjAF}ix!KEJw;4v)p6Gj z-+cGA=i9Nz#jcv6Z zWhR4C%9pH&9(VwiFP-nLyrwK4Zvmu_m`7Zi+@?D-%tE@1`|s%viKjF1qMqqon@Q#q+=U zV%7)!-omxFsM(16I1X9MZ@S^yL{-JDH{ZN|{aU6J)z$3mWAuLG)gKlw+*DfD_uco5 z!omX&IaG!>v5w?1L{AaaD>3Orft%HXl!*0FdSttvFoiNU&YU@G`HB@MopjRPhaK_5 zg87c;RaKN3r4_B)cldX;KQ$Qa$o=&Cf#e402!95S`df*}N_UmJ8D*bHG;Hw7?5{x&Ur$>)#+M@ORg^S|fe0$O} z&my4~$%qn0^9M*}YVV?`W}pWMKWtaBOc#wJLxIUnhYHfDNSo=tZ@u>VEuR{N1?>+w zq{n%eAV`Ogmhm8yX}9xErQd#CUS3*Quko+}{Rceph*(mZ)>X$GRj^;@7yG=2C|KDj zV&si51?4zwaCi>Mawua)(4)leJ`zFZ!~Sz^~6)pyzt_rJ?H!v|M|~QLA|hP;zbr!K1Q0Q%NK3@Y#M~Q z{>(E)24h7odF1Kt-Mdd8GiKzIk5JK8M;vv_Kb|)lGy}H9rbbO9M7zbtA7PYL%#>rt zsS?wfjGe+iEqkq*_NOyuKKshwnDjK<_YS`7l7LWu>$lmQ(ZNmpD;8x3i+BhEF?G)@b z_#&*U=oG^>BJwbBnhIQsy<^6Ymv3Z;FyD&d78TIsg+<4$EH5uF zXoTs!S-oNH#m5}`=I3*9+eF)y6vW4*Cgdkd*R5{Uuo)5zY7Sdyp3~up4IExKXg0s6ogkO)vG~Sy*%ELdeNE_=RELLlI{1w zpMU~85CwoOf<|qPykd0j8`V}Axr0-`1q`NwCLAOchyfqUdSRplA!Xzazne({{g&(` zCWD60egrJG33DfFb7Fih8?yjYSfFsp0KkkNp+Ve;ihLN^amEXmG)5@F-3)@~Od1I? zbpV=ilyvn?fBs{S9!qZyQgiZ^v4r(YX$cA^ELAeBAfhqi zctK$Zk{xtqh0cuDb7CbZgrMKDvLb%k?RSb1ztZ3qw9i;D^9)giEjq--0g0VR;lvP$ zyo_oX{G;ZG;3$GUI{dRi=9Y4x!qx z(u@INn|&{-ugyv*v~ckI%Ibk=#J~YibobRUUH?WAH}<8Z>HlRJW5sZCRWY{1>!lWRnteqmf+H z`4Dj2b-oMyFA1Q&NHzK8j1TjdbKKNgvz!{ibt44F`QE`Vz(TA?01fT~Mr6&USmYrJ z*Uu6MA_xde*+4YTrw}y*LZFRs#OX9%s1N~*%t7j=QUV{$;29wZV2!L|${Hq82`QTy zJ$V9O!Yih#K)86MfakcP9x%UPA0G0ko+O|k_*kmJWPvRoLk5;LQXyhIHze91a;_KT zD9d+obRs*)XJCLu2L@TaltnUKJ;{sOBCC)gL0T#~P&HDz3@+kq3p}JS5nXeF*ggY( zB9EYz&|b>QP6d$MI2QOzhSJ`o9|~FQ(nAhR2)HSyBvm=!U7Q`+B8+s(xe7A{N|bV$ zF0gtjCZjl86bi@-GGzHzmV9jnQ6ZbWA?ibm65=D!iiQy!AtI+X3};dU739&}MZj@G z`yd1q*zqVJD5Ji4s?L`W^eIpf+<{w#fd*~_wK10jD&SAC#1R>^0#Ghhivb4^OTi=Y ztPF~vn)N}fWk6(13I%QyERvfTbMrXXk)k;$Pl20)#vqi^(H39?{{S3`0meffU?yz8 zV2@y{2v{s!65^u@%tDVPrR>dFAcVb+r+_(0#^bCyQi>p&Y8pq_Utvn|a>ekI_83LL zV&QFbbv%au%-adsZ~n$hi7DyN)m+r43P-g zStm{7qe_*-K!Pz6`4mPmCuLc|4vA`yPRd_?1Tvrs>NGe=KMz7cfgO(mU|Alx)d>pA z0+(Qpu$f?kU>+Eu;0)kX-@QEL8A)*qO_PQ1lLRCL5k!bBCLf?D*P9<#SfTP1UKR8d z)ek`*1w*PDTr&3;E|N?|m;hq3F%B{lJVB9Ak2uuZpkw=~*&YyS^6%j#C z&g2%WdXk}>lR>_v7`A}F#E>Zi07C+SK?{V{Uoz?S2qx9TO9|o#xL1`A)uVoZDS^8X zN<9lw%cI2Te5B^K4U!@jCmsOjFbW zfL;DWCy6Yw32unU1E^Ao1^*Oed!r^=jMgpt)T@|3vN))9(*=2WthB~wb0B}A2&1E~})Q<|zRQIrlP zMVUHRko?_@Z)74oF!}IT5Et=`YKE}RLS+7AYefw$nVm@OR4V{jZV8hO*0S)b&Pl@q zDho|aOWu~G&k+;o71@f!Ng=t(4W~RMWwAVvucKrnQXLc*vxRaiTP2|qHPmHsE-5wX z)I}XX=x$ZVi4Zh@+fhI>iTU!j{47ktW+C&D!AU@Y z+Dd_dziMq3ofLnyK+{nUv_K6=gSRww8 z;)2V70=1a}0mogN-|qI=Ho@5}0&ixwsjB9eIp$WsNcbywXp;Vl)Y~U%f?5R>_+=Cb zIPPCYry$jDM1h>%Mi#men22u+B|kyK1C0pTR#9{qq-SD*frj3XO8$5LTH|$282+Q= z!DT>!T1bI_@)|;Vs)j%6~^~66mLL*>3qn za2`;gc2XeVxNGNo-F{o8gd`&77HbQ+$igl$iuJ~}4vP^~fyKB3e;1-_BC5WqFmuEh zoz2n308cM&=GwXSRW3I88&IH@QeZ2Nrk1ubXx8>e0a=jzaRA1N)SJj)I*2nQz*=k& zm8mqhxWw9Aq5Y3p@4RBkk24Lf0}9ke3IxDi8{g`7+$N>}hqDY>NEBI#mY0Y{qL{^r zb+#BnV|<6_)eKIKLc(La!jzgOZ#DQGbf@F*A-u*n$;FYTq^=9=3R-C`mBb-gUp@ied#= zP|6+!S>JTx#v12GTLwUU>xeCH2Im0)OB}D4(OP>5Rd~Z)}4eAn5pr$BLOMqMUF-5~nDJo!_$4}P4P(MI_K?^{cb!Rdm zpFK{11o74!%DN1e4W%3pH_>e8l$~wMA9YF$S@DEC!%uN^L`ohijgFUc#krl7%TdzH zJ|AqvhmU++q9oS|NTV|oNdcaY)guy>t0$*YF24B%y(Dl;2c0>`T+rp`u5!GmPL?Z0 zyxDIZws<~UsN|D-YU{MW^_5hHtp;Sv7bwqm>GZ&iYg?7!hKKjKwA0>4$i^^K$F^JS z)A_(27sRcCq!o49jhEM{HBK~htr~%hoF8wFA8&zQ5Pn+<=-!^|LfKA2SCP+(SHO|a zjU9H>7pFe1-?sc9x1Xj!t&wqW9)WZxLmUcXvHO&`qQ~*Q%(R8ulUT|ih3x+ z=C6G73dc2(1QUP&B0*sV2jmGT@Du^0;2)>jtWR)Q-GYl0P|9M5l(pG0KeyN!fEZn( zB$ONovNMZ1*OxrbKM~;O;`>GRSSwcBz-5GyeA*G;Psw@BO}+JG*@K4dKdrc<3Dy$@QaHI68;Jb?C`)X9Y&$j)`%usOtA%$ z?4|_Iz;1Y+5rdOsGgG~<2fYmJmWRbD2AfEUnFm2(-j)~#SkoB+HPQ;pr?8h8fgAmZ zItn0D4HYks3)3w}okk;meu7V0lIUCPVx=!7u^t;#J`$^f)KUI#&R<=q2pzAkw`{(y z{}wm1IjEwt=lS|m^UdweXI*nAiY(Mk8mmgtrQ%7_D&HH9lX~RADI^yKB zD?)|Y_fm8933C2f3P>*!8&~NhydpWOj%7PWB9#d{>^Lu5P09`S&yn4W0^t5xg8oJ~ zcX;3yFp|#1l$ow0oeV&vZ=q$z5|>CvG2j3uETdY@=E}?B(#M$d$sW>ZYoI z%1YXyPV`Y1D33}u3WHTuxhs2g=W0UzRkmsvP>;*u-z$J@K?hZH*|`AGg^JPVa-nPR z+a6_<6qX+$iIkSNpp^NMmo6%9xOdm%stu=I`<*&3^At=KUafp@=2%?G5 zo9*w6c*^jjM$$4?ZP>7S?V4uM`s^Z2=i|uq988cs@Z^tlO#=BSKm^JqJ}bP_6Z|aJ zs1j8V75rrkQ1J?0G(^f?irMURtmMn5Ex@gSlG7~I>Ks%HB$6OwkOU3E`D`RcqGg2A zC8`}8h&9|3!WGXCg zx6Vg)*lT~w39l=QV{ntUs;Obk)hEdL=PAI;Qk_=19NyAZhmp==MZ=3$tcVoEc4^Z( z2BIg8kR8sj_hr6^Nxut1O;KQn2W|*Mm(p6joNxse@kcZed?U%SNz_PWjFqd_7S${4 z)^(3qLS}rDKs~4*;Ft$8>81+KRBpn#=Mhb(p=SamVmG3K22D-s*b6roJAfKMQ zI;Q51nh4}iN!fEsKa>Z@yZx>rU+>Y{0B!U+~{S*aj5X;elIE`DOC0esnNoY(yf$_8lbJg59fD zuWMFZZfd@6=WPV;aq0G*4^mZfAV`f**TP5?j?Nj}+} zb5CiA-25(yS5PFpcAi1WC6*`^&tLViNJPG>a-piD2LryS99(Bjz)6{VIxF7MpIkmR zi_zqw(ouxupiAE(M#tC0adQP!6}Zg4I3^Nmy(>%1v{A?lP01V7uOm4APbdJh!aL$C z;K&q|PXGlh39gK<#g@@{|MnxNeZR-vyT(u!Qg?#GHlV-`PwE^jW|5B>ot!E02f`1g zcVDs4V1|4I)-2$OKcLdk3%B~(5203cn z@~rO4pq2hfK42^H=)34q0hx}}T4o9&pXMmH$W`jbr661WE%_NSPN6u-nOUi*i8DZm zLp=fqS|zuXsNV<7rJd?yl9c$~BVROYq|Q=>G-ydp<&mthqGVH}!PPZXB_cCr_6;&K zZe?f)P3EtvFf>V{2%jG_YqU;5=08gT`HW0D6Wt{v>-g$RqVGw!l{aP-hr_ArIOn<} z|7>Q#&EJLsc}Vzevio1D_D$9AxZG2!GkESS{_iY)5uzy3m0sO#<(CR52bNH3bT)=|1dgI zzD*<#k~6s}wB@L*%J8kJeSV*oqmN>#W^aWa&sluEZz1P5GtSoVn2 zhg-0!WF??SVBR1)jRfUStWB$srlqF7k>C; zQB_vDX2o(pkpw7M6UXX(-KrHE*R7ESbGjp}um~dr^8i{P(h4~lk){|vst#CV8s?Z0 z!~b^Ay&t^ujw%I6x6 zAOxvW-Hz#c`tiq(9{f=x<2$L;{5hZZJoLcR58S7!bUgwPQi!MMEE^7E4k*i`$U|K5 z=THCjznU{=;@ENHM~@;u7oJ(56ebZlcslH*dcXM3GY{VX^n(waG;%~p^>0|(XxM~D z4+%{MO@;g0N;>A57!U?06+jqL_t)a;^%mAT>FD#JQDKKY|awPcu_caq}!e! zHvEX?htlx^cpSm1%(HQZQNy6v4L&(li-vYpy0i&cBEP0Jbm-TQEhuEt%)(8KaE%jI#V;F16(s zsNQ05s38jM_%d$wZS%F$w}joL4{3oZ(qYBHA%xB#O5m;l0UKX=vQp_KGhPy*21|3A zvR*35^GFjSe%h}rA2Vjmpn(IbD#{N!_`r)Uy|Scvn`AsOe$aqH1Nv_$D?Pl+k(XU{ zHJLVrvQQ% zf>XwW6&I@HJ39(=hwgClp zc;KeH2(PWW3tzs-ZngXX69M{{^f!S6VZceFSMAW^Ff&MdQ;Fn^88hNG66wtPrSsU7 z7UlwZ#pt1`FOgho1e*HeL{cG^xp<|}Zy5sJ@%oy=;VaRFLvf1*5-7S8+U3opL)#_P}Da}P$xZQHjW)_(@5 z7;E10%HLo5)Z>p%oH(gzG`+E`@~N-yU%Fu4@m&vN4goQokO2!HHoQMvdFkwF(~F{Z z$ceoD+P{`8TKw=!Zz__>L5Wy?<71X5Q^PGKaLFs<&FBdI9yOyasxTV0Yo;bO(Yo4~ zXKpB!@ii9!Ygs+=d=J5$ut&`7)R?Qjwp^;gp&qB5#yB`jo^2K#` z|1DGynK<&JsF%)EZ5-3@l{G(n+x@Jw&-vZ&i=4~{Z@m;U5|x`)%^vbG8z#Onbo`*{ zUllfQ>%nz1FY4QMP{a zNADz(31}tUusR|HM;}s<^|$&7P|FyqfBfO4uu!Ytqs_vwXtBd^TX>QYBfWgm;N*Wn z0X-j*X_96jUR)22<=-*oexy{`D^-F_n` zj=$@#_jm7cmhYvSx8J#K`<XvOwv60yUvJv!}XJJvBrcKLeN40!*^lP9yP7>X9|vD zK13{U9C>6{D)!yH`LfA?;dMRgD49yg!lf~8;*_~x&P6x9%VCG}E{{0;(DB2E`fe&@ z`cuY_z4zYxlPi7@X32LG@f2-msTQ?G)+IUrX8t^J_LT8YKI&-kS&pHRk^cPi&ksB7 zFxEoR7^DKO>!YnGieEtpC=gI!yP-hc18(6P?64J9`Wy}{gKLElo%nV{U>b#H&-=?Q zw=Q44yvGF>U2)ZwXB@e|lC+q|pWOZ6&%b=@)t7re_v~MqB5#r(c1olJ(E=)AdKmTy?? z`nbm|M7Hi$Rwm;Wg@wi6e)Z*s^{def6fRV-+o@l_J_z1v*F8B$#?gPoNGB3CoJhE^ zI6^X$qRonR8?BvF3%>h?;lwf{rR&z*an)s5{BZgFZJgn$A81-2Us)VU8*Pzmw3 zhXlZOSt)ZOX)Ba8orK6%!rXH`f8FYJD9N>I-QtZm-Vhm|sBc?Ch7B7vZsIu?T@-87 z$cV)j{jdbvNDNp2#I%nUiwJ6vkT3mV$%5HmxaHMPKmYPu{XTf_qmOU=^Br#5A2ej> zo_p;hoU&;@{p8>Ky#D(9Z@<~Fu@vAx;NU|Hh7HsD$J5XBe(SaQbHA!CEyYqsxd;!# z3wx+qnC9;L?QayYge5oq;sy=h=-pS!r2sqBpkb2@n>Ni`xOm;Vb$jo)xK%E0_fo;pcZ?G4M7$KazvSA)Nf_5eZ zY9oILWugVw-*!8k@{5l@KCt(jMySAWVxaA-|9btgJMKKK%hB` zKpYC8M+XNVGpuO84?i&Kl{l^1?RLN+t{qL7;iFGGQ{;aYRa0lpy8Xe20niU#dF82l z?{x|bo3?IgJF$Y|#<$%6uwjMYf8}3K{pFr;VPV6Tt?Yt=1oa6=5s!%LaF*-~>)rk6 z;~32JfA!@{PU-&oKb~5&XaRPR*WPwpBJItYI`vQ2^b)1)g2G5)ajTAk4vV_J@Td85 z^~&`Y%hvfjzx+hEVsoB62~M$p!s^xKC1G>7cEwQ-F@q`=5X3?~UX5Z2tgcc6M3V{C zY}|#21&ryZJl9QDRlWWY%O?|4W zEAU8t;t_`>egEn^@4w`#UQwkX>)D~Ciw(WQ%kokc*X^Ipp0m@Q2mI^px6P1KT2{f6;b_E(gs`l4m9>PENtTwDQXM2PAhYYbYu9Zg z^Ja}3Aw)B>x+BW=gd79Cv&0J}nlW`*jK=P3Sw_-6Li*Mur z2v0EnW|a^N8+rh`C~(;!FNVMut3{ukJ)Ah^Dt0JRToNm3z|Qe$^iuQ&IZiamAOuTn zTPZi>*b!{nF}1gyuwq&nlI)Bf7FQ%@#&Z+TKKc0i)yq2{eBk);rFao$uS{ zt%YBIu}7z!*DPB$Zukhp44-;WPi6yYKNBr(U=%TYSa{k~Kn(nv2`dt5*wjG+g9k3` zeIf-7o5w82smAjl1*ox4Bej@oD#)gFH9x-p_~Q$%0}2Eb_)iq5bHFXEFHnZQpyweX z5z}zQ4Z>iGi4KARjCFu@xC*?UiImu|F>%k>EMN*gizwShk|1`4;x0;@LE)Q3RKS5k z+<)@`JENiEO6D?~fwdM{j2Nur5bB4OYbK5uKJ3E*By84k_Xi(%s9lGh(QjF_Y{j%u zBSwDk9(8Emq0{{jKi+<)4hcpVGH2=_ejpfN;4=)DWy4S??V;vVTWLR>>aXuyR zu7fzkLRWiq9vMZ24tuWkU?lE8~ZxSG>L|SsoyXY|tP_ zwmy&;9i%>bgiw_bfaiolfq1fE;gJi;Q{t7KQC1Ks+xujP7C~(l1e$eK)w9`1ipV9< zF~KpQKtO@*lmc}QxO3Q~kx;->L#p^;P?*b$(w5em^VI069Oj8P7@$zCN zN>GoxX)IaDNFpSVOnG=f+JW+*=&h)r?knCl%Sdm*1yed;O9|Makz8jhPYE&u-{R#% z2h|ph?;=VelIejYlibu|B2|(+vICSZoH++U)@dn7MQtu(%SK76AOsW$D6l!!1N9!t<&iSjmZKKIh;$|Co#fEG6auHyk^3r2X=w{1w~^t1T(>e6Ap=Bp`+4DOU(rR16OSvMBlht4gax^2o!kt|20A<-|lDAU)X% zaObpGfMfF4S}XXGl0?)<=Ptq;87Q>WTeXqfI#)@P_qHrg1tFk7K!NR;0vbcM;|l)P zRRf`NIJr6NJhbJbwDcf<3E*wT<>ok6#jR`X%cdnOfm+}?x8A7nliS?dDycv&oWF`w z7z>Q@SFl%^yQo^}7=_@QYlB|5dR87+x0Hw@o5$v4z;PC}RXwxA13Bs{ZCx38OqIC# z`QR*|KtO@*lLB=PxYOe6S@i7yDVc8IN7zw*ghiQK*irr?QpS6ph=)<@BaPZpPduRV z6bv=MFK`L&=^q>}zC>8kRI8dihrx@m!LqX$UZv5?#zI{%QEnM&HiQJ2m1F~xm=tz2 zNTSJ5#uMZqSUD0Sb)gj0vpu}@+3fi1`Y4~nony*~-zg9o-dlNR#TpWKS1BvQk2KVZ z1muh>K(=;dT0tu?24aFiIpP$H83lYjc1JY9H$&(WrV{L(hQ>ELo_XwHhT9d~12Buw z;{jHqq!V9(8FU%hPe*7*l#eARxnUHVP3g*}_wo#tplFgwGlg7Qmt3X@;mtvnYZ9? zfX3=^i*nfz4#QTmfEE%@eS;8CUWsX)sb( zVkEHx7l6yYTimbW;6yDbH&x=9_#1S5PZByLp188|tZY0M=t??kvGm6U1;@QcMJwG-2Nn#1O7vRyk$ zhraA@Ni|eT5$w<$Tpmg~igIzHBEy4%RV_y`G3Li8U+%J@aLB=X2}2w1rKp3Vm4o|3 z?vgpg5%{yLL?WpvLhDG$JxUOVD9RbEq{+|*u@91V;SqpMQk0$x&q8E~Ks*(r#4z3f zKxTnu*>c|X*o0m-@5Bi>$)pa9pi(G>Q#~0J@x=&96|Kxv$|I6$Xgg2F1DX%TtPpfm zRAD?-H5XR|;$Vg?MTOLiv*O$_2mu9lKnm13;07=F;aVvyp11jUh3_^&P(SUuDS$%| z4lGdc3=RW^Y>p11r}s507L*# zVdr6pc}Dieb}U@_W!PUm?aLOu=dg0}6n8+9Q-eFfbmX7(XTU=pw-OOxGt@ z4DqwOz6)zhwOv?w;L^dsU4TrWSVjq{B$#hHfELcAh?UxN&8P#n3STe7i*T_v%s!}U zv6+MyiZZ$&ATp>XM)nLe(18ZYY{4|b;jrK`B#@Cseze+4xR~UN!!7Cw!>f39;F9#f zYcyoZupuJ~I4g>aG8%An*(+H*$9gzz5kCY>6HJDI82hX7ES$KEqf9~=KXKy4n-D7E zrhr>i-VWJ9GbT)8RLVxo?1dzQftwO%0}y~wj|?FrEu<|}pPYr?=TZm}gn$A&9tG+g za0|OD%&a&P;oFB3RIDGsA)p4|2)eON4R11;lzDXjcl%Uq+JM`?S6+Bw=Co;!X}@zb80dmjVuy$>-IzbM+c(US08ZKysaJZa{@WCEAa`^Z2zNOAGI>4ce zqZM2k3STUe3^z?+DBx1I@655ZR6qtWGROZS+;*D1m?d*cwj4VNxhR;Mu*|OJm#F|J zjBU&uydi@IjT}6PYO`$+?N37-GLxAK!;(O(OqPTbXUvhB3q@(Zs=tt$y$@9dbgQ8Z z2_%}J#<*l68LzCs4F~( zAHVjmR{&DP4#LVJPzD>}qhY)?5aVVfjSE7=>#(5m7JwTw3@`4_|7Py`Rm*&)`@S3E zF(ur_87y*K&%swADC~bf=d+C~mKHJqBn>u<=UEtB_*F`zjIU&nEA5V!O9)2BnU5rj}#1ux1&u$EMSlo#JKla`NzN({Z`#yf+0TL{DaCdi? z0zryvTdV~t)C(=eTda450&Stiy%a0%?h*n7NFW4ANJ1c<TOKQSG39A3VPWwE|&uieB18&e$R6@83yTeO34cU+uG_;@w!-0TwE<1yI zI8;=>puB-6ZbUY1)DWl>MG@FyuQSHuOff2&7!)&`+0fAe!Z3FTKmdr~@O$r!TfXc^ zw>{g-Wa6&OiLtFPVIV5N5Vy?;D7)PjHRh)kLe=msTBO_ka{D$>SHqdvv=BJk;RP?D zb9xzJt|cCJCeyjHhIVWhb>*Ugy(M)n_+TREL3DW>)dF!K8>|G|^5CJ6NRb@Cxx<$A z$;aa-efkMZG+L+5VfRBrc4V{Rv4VK6WO>1UBgfqC&6_rk8aW&&G#MER;7QyeqXQQV z6162tEXuwp+|;BDDWEbCV`SdJx6xtqF?cN8blMq-K;jE!0nvfqTNo8!78RO|I$W?T zB6I8xtB0C*X!{Q#ySOV;kxJG{Sa#%b*{xKZgL-qqxLYYT@i^^NICUbzjGVY#7Qt}j zh&V>)BXUS;%&A_VyL5?SVJ~+3V%Fy(UYVq3f`QbZAJ1#xKd=VChG**#D%s-!2d`ec zxPRB42MI}ao3?1zq2tLz2U1fW1?SJZYx}QWUS2Qu?q8HG8C{e6-n>Xq@a_rNZQ zS)jCi*B8r{DH9cOZqvGT_}cE!v2)cH%|t&WDs)PlUzw07my8_}aikG*{H+?kUnZ$wA6Y}2Me^Ok6@ zY-|asZ7_I2IS8-{sJS<6+z?VUq*bey4yWyxHR~&sDH|Da`Np*?ZQ8Z2UBBLnRjXrS zZuRKay-fY)SeE>}c1@)U6)V@MbL8-$l!UnadGqYqd!Ry(o$0U_3q>2@3(UGDkzffFZM26ihV4pF#EpkJC7XL?-!V- zXWxD$OO;{L(w33BcG*wYuY}jCR^8~d>n(ol<%asra6n9Zpg25Wy(jM0t!*1Owrk&M z?dmo8@&pbZJn+Wt=uKOX^{d1BA?s@eKrF)SyUhp;>^KTQ{x`YTG(9CAnmw0^t`gQKu(Q9NW5O3oS%gV5LgXXmJ02S1z5;PEUKOdss|#G}39!$`w~G zT_mbg$Bs_<=>4ze&I%0hojGaZS94~;JP5l((wbDjR0E14J9F9GSwjX6vS-`Qo;>!_ zkO4yn4|LjXXO10yG3*6~7kZ~{$-?;`y*Z{pzQB7i(S16#^*GY?_N);@hkQMEc1YpE zutchYuYxZtNR@e*Ql#`1bQj zldLc)ZqJC9Ub%MZa?#*?AHMPCo?m|r3C@4<^qJYyr!gFv|Mgb~_a6u;T#Ow!3s$Uf z`{oT)_zM@#UAcDke?>2TUWjL|2A(;Mn;A!MgCX{I46nLp?*Tx`OpZbKiux#20UEhb!nbclqkRwT)2~#? z;;ELrLIf*|H`Md4y?U8lb~MVR|C+mas(&h8!CxuFzrQz%yBCYe7KJ9Pb(%MpWLdRehV z@YIGHjT+3DGuvU$s8fBf$ICqHtFI{Fo&yKZo;XpiSW&i$fM-xFheMZ@zGUT6Z~r`g z`SPqd`|>x9_Z2r3!>n)22=B7^CC(7E+pM zv3Oy|0J4XQ7ZGHbGHZ7As@3x6%QN%i@$=XHT(?2}QbmePo;YR7EP@z!?b+vVwPd9x zT)uee*xp0kI(7Yg`i#zv8m#{5hYjD(Te0V$!OO>~cUP`n^IGJUmhD`yV8<~K9 zemt*%|Fjx-<}_}IV;Ihcjhj|%)T~>JmhZmzL2^=pp3TVI_F9b^2NfuU^;q7#L4_(+ zI(hOWMhdVYz!Dh6=v{2;0C|*Xpvw&7BKj;p8Z~NMrD^jn65B@);^Kr$fuJB638=T* zvPKRcG31?hYt*a*^k2GgVbiKLITi9t?292snei-{bNUtBqhXS7{hcWOgs8J zfX#d$FpW+e6BARoaA7h|0XXV6Xv8Ksdb3yA>NOfQZZ1=^CUceQHSXWNCyK6!CO9Ry zV||TgE&crSVmVj7Vx`CE1#+x-m%hiU1N=rX^@>KNo8#-^F%GGMvnV>XVo40I1KHU)0R62}0 z=8x2_)FGS6F$~a^uUv(>WQkv|t69H3^-!iv8C`a!4Fe#jlX|FKyH@@B4UZo?nfM@q zToo-+c+~rE&-vuzm&U$UzI=Ix3B23_Aw^>E#Y((J6MMqR$@AlR4g9Coz%vKj=)zb7 z`{c>{@#TUA7)3kKT}cLVL(Q84waAP-0eN((8JUk$;TbG?cTChx zAEGrg=49rT{r0vq#^JbVVkpwIzqE^T_lfB5bY)k~1`ZQThNDS=cMYgFGr`VP= zHB7Y~II_&fOKCc0(foBf5L0+@jeCg7q@bK`4^=kL;*E{LzwQRc$z3%qaE1Q4Wkeu}3 z>#x7o`TC;93PNCk#zY_1Z=f;kh{5Cr9QBws$chhO3gn8odhNo+h&pxa_v+U_AtD?F z5WvJLoDu%kUpDSu_w$4apFm|idi2Qda(C?7Wy_B3QFmf|{ryjzIPoAU(dMv=Rh-gW zkp=n7l)ZB0N_Ivj8Q=xYo%^rHNeSmpo%m(*rfx6xvRceer#(;U&?{H2WoKtUOiDBc z2Nx|~{K~a!o7Sxr6J;?>G@?F>`3%`7ljPK3C{w2F?dT|%!;U=&s~a>^X5;SH5Xe!7zbFTeiUu};mAJ;Ey2s`cu)_v~tM%D|U~FPJsAMaxzlJ9dOVZeFe0kU@hQ zw{9)84hslPq>j;@*e1$)6+vo4jaoI29X)JenKzJ%B5pD;BEY=~3&v4IF>iDljM;G2 zATd~I@x_F5jXGlY*4pz(e6IXsYyH&Lct5>b8)xJGKf(o7g z)ohENtveywML5W8g2@WCVU%G3ln!w+W?&a6l&zK!pu;`@E=KL^YR?N-ll*FW@OP4EGtj4_Um!HpA{o@b+iJkU&o}br% zOtSsiwP4rj(sjDSH&QMqJAy)j*;iaRTJ)yeL$HG1|GzS#J^WjoLhwWU-ZYwQb()^Vzd2)osX9Mcx`8 zDoK=L1I`87p86qi4Bmlb$R=b4^^uWu-zOlz;O#98yWR+Ehi)Pf-kk77YOZzS_zxLU-%22Z_$jM+3jS*I=tVTbf8tfiSzYP|XS3tp{V5<=-OzI*A0CL!#hE1E-s9lE{MetCTx`?49M7B|8V?|8KlqoZlO0`${ zK0o8(3EmGux$)h`f&K3T~M#URGZ}@D_B* zrr(>5k5kejBCciwSGX~9fuSIqfES8tM!GF2>EXk~OGR5Td6`&mk0vW@2GgV- zeSBg{;={)oP%1peA|0Tbf(cdv;sR@Sp%-vNOD1(UqN3vCAHa+l;c*m^t-eK8*&VKY z`GePQ*ns-XKgqQ?I05wxPP;uZA&CRh9IC@1zM8;Con7N9gsNanLTc!_!i@}id1%~& zL^YToBm5Oo9UT_xgtAw zGqPAykNYg))A43yDps__&|xDemr&d}(unfvQU8UoCr+Lf`*aqAGJ2xIMc$AMYBzKi zOOIsDyAjtYA|bP}P9Ygsd!wOrnR3T2Ud$Jae5Ekd9z4Fttmnt`8u*W^foBf5p(?yt zIqA%}bYyp{l7&Zh?bM-c$lkTfN5A>b^m&VHqJ6ppJocaF&gfjZ#48vfS1xt;{7Kei zLt>!G0UB1fbH;?X!m5ULt5vpD$>0gY2Rc(83Za6c4YU!`;I_IlPwd$?pl2r^U3O~J zmF|sd4ei>vXWi1X-WluXNONEg!cuY!Je-~zdp5K!k@wZEZQGSAv3vbWi&NHiqaia_ zXtlfculu2AjbdTd%C#+1Vr>7ecP|_UXhATBtAG`pkI+64Fz)Qs=qo*%R3F@_WzX7` zr;dNqV9SI`FhZNN0^Z<>*tfk^iJ+l9I(Dj6Y43)gyqsCAp0P7r5E9wB>*t@k)-DxR zwS3#q;&1fq9DnJgx6NS{SKSB)l4GJzpwp~#rrZhd+O%rFjxECKR-HBeZNKa^r^n(k z2)k|4J1-sB+O}lSm~L&`l`gVx<#I2(9VQ1U2*tr_p?&L@_N-Q-OSSUt%NKd8Z|ArR zC%l~5GbT=4ICH9xJImtAW{AREf}K$U+C2-|8MELWT##A@;LL!+OX1+fLdGztp{lX= zMrUS3gOC$xCWqLQNaav4WQq=IoZEzy?nDo&QcV)NU3BskqmX@5UKm*11%%0 z({CSv`P_@@5uVaKeWUL4kw~=gDo*W+1mL6Y=A@#f`1E(~-JA$>9#OQOJgQ>gb*o@f zKJ~1)9p|Hp`@6IVDS>%1H(lr``KY>4hCn%4B}R_i9Ll*$MXpFmwXzdHu8Sos6~^bu zC`IZ|9faiISu*$h_-kt5nX73OM$rvjt}!Ery*=%-3s z&z!Y%+pp&$t~Y7gbkd|r;4!)>P{3OeCDr3;*SpV&3l~W%k$z5sTD22d` zOxR8Q_>+mBe9F%$Q>Im|TJ`khYb$r``DN+S)2B}pSmMEplFdG1 z^2XTAv{ckeV$vvDlwz*}1y0?FTC-zUYD&u7*|SK8r?svkOCm*_NTQRcPN`h2`l*ZI zD|YYRzI55Sv!{d#R(WOp=jhR6-kUltJmT88snaLEIW8judRL7egfftp=3lVziEELo zcJ55eOq(@(HjJ^T{Paad;Lt@0hOP9pX)~%+t$yM1)fK;P-?D7k`7@_^nvn*bnXi!U zEx8?ilW_paQLz+UDwEL$g9|Nl)Rlq;t<>Z;2Qn0J$hrFDfQ|whDUv$4`-f+9B2mxg zhWfXE1xnS25X7$~p@mlGf@L|$|1l0N0advhd3fRlCSHQ`7QIu%`;}UnZm;k*}u$Dv9*u#`BYZMh!gOu>Ki2`jc}( zi$sk)a^#>cJ+)h(zBYrod;h`ACZ0HPN(R;VFE#3SZrxb5QNv2rYiAj}hYlVRdGaJc zL%ta7qxeA$fGj32t6$#0nsw@ws8a3W!(=dDu5(9e!8ijcR-jPxR;?eWXP(@?X5X-W1Mqut_QWCaiUb(2_R(b%NzPju_^$_?9kTt#ZQ#hj#4{Lvo$Y zs&}6}c1V|+`eMI)q=SSwVkS*cN@-Me4Tj_e@FwK*`>?sca;__&12z>P3V@=+<_3|APd+ju zq6Rkt-&!gfNQ#ga5g;F>js9LUkxT2MXzoe+0YuL@CpPKhmang=*i<6V3 zoD?Mj0aQtk7(^iPQ8CMd5?IR+wIbK$qYj>;QZ*H#tcYqY2-;d#6CK7pmn#@)MbwZJa}zit-9 zSFKzHKdK&!DKs<`4KN`g!7pEsS|2Dq9IRv^;{F4NjvqW2b1Qo0yoE|fq4Iytb7h8}nS0V#);E(=M_Y+!|Q z<>T+inT;mdrN-&N_OW1vau%z%&1m)V_X-LPy&rdvu21iF+OPnT4xW*cdi3bA^Ovrz z`+4056Q;Dg1jPfblfdi>bsix*a{TJtf(5zPDC?)r7=PM$UuEE3PU zAUk9g{6AiG`4<6Kzl#&zYlOrbz z!L^)gNCI_6li{yA&sCFudyY5tBnow+@|go=Poff3{XY&Zw|}?{h65C%mqXpi$%6_h*A%SF-Tx{}a#@vH#(lpJKs--Zm&EY=_?v3r*#mA- zFkPvS9tZgP$pRB`@2TTFJ2Rc#XRs~7(l6z4Mw!xnY@h|&c>DO#VKcJQ8N}0+X}EM9 zOy45TUzqUG_>_D1sx)kxo}Nxe0)x-7s>2NzSK7sa0OL^?)>qDGEu;>haem<$G&^{b!&@LYf8zb1=S()QMdN2J}jCa0# z*e^2mia}tB(9#cY-9pPYS&=#dkK|0`mPt@pKuGnC-{3|h?-K+O68Q;~D>#$j>V)e8 zwc14$CI|IIdlu|g@v7+m8$8mY(t--62@xiGD^Y7P$g?V{Cn3cQlxU@bMPb)c)t=0a z=eKfcS0tnA$j!-<3-UD2YS*=rY6)pqv_c5ZtK@QGK>(^`wM>fKRpEHtsFZyfTK@jf z9@m0tcjTPUZvi=bQh~pJSp6b~-{;}K2=n~X|5Oe9cW>fa57@@;dI#p3m4kIfA85_(IZ|R_14Id2@m4Pu?0^lWX~i9dFZMB{um?c zGg8xlf1b<8&I&GAfLBVwoN;BP zV?JTec0IgxKcsjl_$8*`U?z|%>u0clK%N-dv}=p?TKL5?U?JXL1s^fzmQ?lm3l;$6 z?nGT@J2ajP55E{1T9VCo$T{m=3KuPMHzLBGl?}!-uy}myR*BG1#t#-FRly2JJ=;KJ zvk3yFwU|4$??g`xKYxMEdsu73)(32;W(2!+`XWV&0R7RoZsInU)X!bGRHAelXb=Qa zrrlP!Sh4%zmpqwSknb!YdKi7XWSR1sUeCavZG@>VS(4Lc-UqL5CQXwlVk(OCbgym;0$)n;>Nl1=zgHb4TBj@W$ z>G-V*D&9j^M6nc{OtX!Tlk1t%oK(q{p0th{6I-nnr8?59(w0}M4PSGj!Q?2qie z1}2K%4;f1^R;bimQn6*EQDxOO~!y ztF|DEsN_g0$|EacVq@-{J9lo%jL(pg)v8rvo1W-`@Gn)T$xOr;r)q|WTsA(TOdq9v)*`O%dcV0TaTacY0H+a z@CXfoO>a@w0Sxcm)oau~eB^|amFard2X_ZJI)_~x7W^-Gmlw{qEn@4sisBJMs!0_Y7OyT{^fdzcc|tm&wsL(M*c-*4Gf zG=KiB8#m#$RD!4_II^73XqY&C#=?Cm>e z-h%mG&udt^2zpR%qkhV?nb&W|G^tSTomYpC|7KpnQl;69V#eq(2M!#7_L;kI;WyuqO6aN; zE5BW|SUkGG_aNnfTQ<94iD`0LkH+;z3?69l3s|^yN4`SETsq6KQ>SaysSBqADzgHU zIR*em+9eB9S&<~zC&EwzL+;Cr0C#1+R9cukr>VdS;+#~!@Cy=&Oa{oxAPLDwCQ+3| zGvff|x)8xCdZkFeTs9o3!ClhggeMtUXg~QSU?&O7!X?Ho@cyhQhUeo&@CG~YjwpPg2{a)VkAEjp?W~ofL3s+gq*Zglg!O9 zcU5ZY_Vcg5qXrn5%KQFPkAIY}66sT{q@V-H4matKA0-zkP*88MIPgp&ZZlGue8VFh0W2JK%9JcoDJ5HZDOY~U+ipMDQIO3G;a2_Vz7ji~#6~)!#fb}*72CKrl zaO?3fIiXNUh|NV`7YEtc!s~=H11#U2rq29#mh1L8tIHMJieh_!RN9$&rx9`YbtR((r$r>}6CfUVZ(H}sN z+s?XBC$@oR1N+2VEI4`*Ql6S1PKV5IiqinW905H+Q}lF!E)Y;4O8}SPQj4-7hBSC) z7>GeYP;bcMz7SDjgXm?Jt>rb6f%+#1Fvr0c*9k-0VV4K>enf5vs{^S3^ds8{rN$P5 zfFQmG{USkR1*t4^6?q_8#Pz#G$^uZ4^rC}nl8AWZii(X8B*)TFWS@-m0Dra0lLkoR zB5Oz-3eGFUM3o_+L?}{89w$2DK~zYDh)yZE0Jj2U$+3hHm=@PEG+-<>9X2kDD_y8F zB33d&GAxm*Jd;aV&nrz*Lh=pNOO$$Dwow`fMH8 zB?D<<#DVhR?Hf=q1YaAMA}$@=yO$1W@%6VGz(*F^!S8xm^919xM6a2sDODXzfOrO@ z9V}770iMx<_6i-Lc@m@pY*@A|BjvH)YW;Nj^y!~Yb!FLbt>>~>1A{|6CM((?iwHCH zdheiMmdMjjaI568dixhBVmAqD%W88vHlzXBLj^OLvhmVq^m4F)Bq|bGlfhc2QPaxR zYY1qY#bc^CMUkc6D5&PVg-W9vsXa=B3VuY5{Ub#Xo53F z9Uf717Hz1OZsHEG(SLbV!nR^U%Sg!xFSgsHe0DB!~@Ix zeFO8#nkJz+%;eVN z;w55k+?p|I3Por109=Ye3nw1^){Pss|MCmqqFfIE{y3|`pOlDR$q?Yh!cf6Pg)=(IK|`O5n~GphVV}3LtwH2$n%Z_ zszt9CbybA3)VTo<0UI}LUcnoBd8qIRwezt>it(Cx0duX{Qb-Uw=_JlZ}TMFkG*{PGOm$eHh$T(Y0;uZv}93f z#kUd2hM}D}N0kGZcYqh7wkX|4hK;J9tKK4hqCLXl>U5ap&3^Cgs}Ywm!LHY+@t!?< zysTC_Im=jZ|If;AdJEn&3>LEu_+Zan>_&8W;SkgV6o6%%h`3!gum)^mAr8F`MM!$P z3})qJ(d(zpm_`g_+l^}#vMMYp|8R`OdKn-{a)-Q;o-9eQrV%w-R<}#3aEpb4N=Aqf z+JO=Q$4=+Hd-twfjifdx;q19z?B2U)(v+$A7$IF^kPYl51A=8T)<>-sO-^8d)C3^} zHN=G$hTqtStX=S}gb+nfbYU|q%%y(Fqk%O=%&1Z>YMV>_`VBaD{CG-wTH@V1o98c7 zF-nxOGDy6m=r5c)as0?3zz%&7Y=h!qU=e@kPUMwvG-nF~0j4~~b*yOVQd4RfCpeZE z7kib7Tn}EtWT_E3k$vc8=0KU^xLm*fv^f3Y1A$ZkzCc00h21jYDi!VO+o#vAo!dnr z7x5{=RJJJMCofT&2E|9ub<=oefAX>2p3MM7YY$X`?61UPdNRa^ns+f?$T?RcR#4kQ z4tN20E94zBm~*XSU8OV#KuzWv1{@xnU23db^9{$qQp*d)%bpFpKPbv~xqDsZG7NhmU z4?hM<$cNR0qLz*0ZS3*bt5^DwbYMInK^}mG9t(a`MLTnYJ!X^HrpQq=U*L>SaDhCh zV{bsbIF#>W(TCuO?0B~g}gA*$+E+PyFh?r^K zw*ArYD_)e+$zB#to)SYDlw3ReCm=9!GD7OOJW)=jh|lG+8a$s(`LuYc(8`r7LYwe5 zWh~@ypxaZEiYtd-a(^~uQt1*!t5vQn`YN{&Y}s~;iD_7;(ZYQJEi=`RDl4_YKtf2E zt!VdpLyxd-BC8Mz3aA;eFnYjIl}l7n>W|vjH*eYe$fffL<}62+`0ZlJJ}=fLBn+uh zdm<7={>$q|zvd9xLzN<@9r%f1EQ9MCE(6O51|jEorEpd*3MCW=Lq$9B4R|CDO!dqBC9k9(IFIqOOc)*e^U)SbHGj8rG>6u zz3S?f%lGcbeq*)_d1d(d3zyPUGhXQ2{>0$}>{Da$&U0}8p^D{84;nCVD8@vL|sTC&aB^y(<3B-Rd=KckkCaLx;ZX<(22V?|x|CsqNm~ zyONV0_UhfQOqJ?*@8h7?3*+f@o;!Q`_~Aq8_Uz7`yH=`N1)Or)GB2IK@axVUsNS91 zwLfy~=-?5r`uh5^hx!i-mtMaWRkd1;!a+fRP>c3$j~qFc{_v5v)$8EVBNHcmYPUOA ztzHumevw^Z2EFt$+v@=P=+PTiuV7H$rcIk$x9`+$*ramh$`9i19Y1yi(~j!ZYPRp# zDbU|<{fd>>A}-yFz5T7#tN+Wx*KXcivwGEvRVrJI=9^L1H*fkmHa50OrAmGJ_4oG6 zhc@@cjA`Avh3(k6=i#H2-o1O5Enfl2fwdnXjwVmx=-_v6N3B}7KIL&zr!HMeX#qw>)@|P^wg^euH1~@wTpA zv+DYlhzBuIUzjXIMvP%<;njRI&Q{9yoBQQiZbp`}g0pW$T$UXPP!?*}82TL`2N3=>6NbjTk;W?%v%k z+qSlE*Kx!8_4)h*yZ7oHP^ge#pg^I3oyaI;smF78|L)Ax6qD8K$iV|uDwXfkyWh4Q zyN(Y^q0Lj z@!N%HSJ+KMk`tk-4Xc(eu3Vw~rAwEiZ$`Ci-Hf%n-+i|{_Wr%@-MiPWQwMQ>KQ8Xn zk;9iRoU2)*YP$~Yjb4FUwrz7|r}yjgA}?+K{sXt~J{U4+;EMzL{V;#t)X!%fKXvBh z@snd-A1hKO7ontF6zKW!x7GkUz%zCz_E_Lk&BD-E*u3%d@nc&zuSfB`aQNWlS4XV; z={tWf!&ehNeC5Tkr9Ujl<7@qR#IRr1tTcM;Dar9&+qF4=>O|2Zg}#_GbLG+{=u!d# zGESF8DJ%c=MZ&#%Mz?+W?3u#`3{FjWm=tq!P}>&C@o@-AAq*9{eE(kTn8AY_S((=C zwBeoFJ$~?j-WVBvX;inc$SaXP7VD^?{g%&}k&&Fl{sV7~8TI{_vkCb3P-h?1X~}H6wQ%tfIiJTeb4sxeGWi zym;);faZ% zF$;46JP0Ka_PkJ$*+xEy7B*4r2<5z}mS+klc+50ik zi>6Jt>UH;HZY`Vf`M6g{j=Dsy3}xG4_v{(1YE0@`o zV^r7+cWzvlS3h^orgiH|6)E!lmvg`T{4;cglEsV9efOQ~m%@|p$GtarNb#aYczKQL z)H-+K7<&8dTTyJ8El8~mzo;}8sWCfGX@tR%59<1pZ%&>ce@zWMa~e0mf!5onZ{MQ_ z4u+N}^5)prap*zMrTNjH%?$ayeU~i$-ej^ozI}7w)@=u)Zv=SzW+o(^IC8jqpI)n0 zEcXxe|8da|KuK`^fpG zFoxUoI2eUuMT(xfd>J#0%!IpZmoGnh_z3phyMEo)uv4dR7A|6_Td7FUryqVm;KvV> z&hFl^cH7o+RjWd+wy#&u%qZUF0J$7ViTf^I^3I=^t>Q24-oEYNq0kZ~+qZ4mqeDAe z+RRfYPm`DZ`}f03^z72EbMuy<6CG{()G1yji_>O@z@Gp8yvd(T7|^fRkwg1Rlq&t! zTW`pk9!Out%zDGub3bp?y5(0tE}<;FyLax=xY@ovyPGv?OvfEIc<`XnV{IN?gOZ`A zj~s2@ts79txEsAs(dW9!pH67mqM6=h3+vf?@6KOwp8{|QFDr)(xfS~azkzNFLoJpe z!cXgTr;newdhF2pUwD&m&_Xio`e)zQP1borq-!Z>xEV=dcF{b!Tg|dHW&pLAA1Px8qhA8wxNvSqR_0oVFHEJ+p+B96lG}^b>YH)q|#T<5b zYq)pUsbfcblrA%Q^5lGoKzp{ul{s(WcjKqds#(2y=OII;PX4rb(PC|T_io<2IWbo# zQ#v{7CIn8OK7D%k>ZhtmxDy!%7fgySe z>W7$=7%jjK#shGjSZR1;yatFF95q_Ber)mv&`Z{9e)oPXw5bDwG&p6CD=j5)%CxDw zwr%lcJ}@i$VM44~XWzbQ)2LU+!WXc4d7Wl0NP>=e{P<~hE-qcQT9(_&MlwxWw8C=H zX7sWdtflMM^UoJ-_n0ED++r(;7aBLy;fgJg_ggWI>Q8H90PSK7Xp0or@3`$SEe~$VSY)m{O%m$s{CA#9*x5sFBlR zK_f3ysb)fQmOdaj-Ds{{uGp_T_vzhex;`k;l1HVum#*C`)od(Op)$hY&i#7<1qwkB zL&7N28j1^HVrHA^T8)}vKEzyfsp_>VrKB7E0-?9dlr6t&`>)93sPku{4nEwp3P~yX zif{a=Q>PA;qCx#;(B4dS)@#vX@9~q&7E|GNwulo16=q*7*0C%ym})j`YBS*>-Bh}I z{W^^sV~>Pvt5m5HQ}Nzz`~LlmJf=R3y6K-M!1*YJ@gJj&citM;qG7$O$B*p~kC4qM zJf4Duiqf3olTv~U`-^@r6D_h3OKL@lB4O=wF^e}26{LL&C-nUIOKLzgj%VrsqbV;s z3;h!f1$qF?053ognSqk6W7vF_5TewW09{mZH-=?q+>C%W6)IFHQ@#4}^A}HqUpjF9 z+>Q$u=r&A0p^LEE95A9qV;U&B5TMHxlPE_XGn)gDuAqccz1&V5nZ`ul+Oc!jxr^t6 zLraNqhF*_zk4MP~Y>c3DW#5dvP8Lyoy}iA48JVne0p$?0S0ch?j#H^Wc>>wJOlAbj zlq*Zb`%fG@d@+3AnX`K?hS#iF%g-<1!GlE1?ualsDIqgHp8P+4oczwnkz+@XK6mrx zv`Lf6BvGMq^39~2On4EIpsHxOKK}l&+tiZLG2~GDjsNA>{%h7`n^k>w7om4uq0y1)q^=i;u^# z40*)+6)DKz-sOIC+#4lJmOOPO{Fe)XDS~&BI5&cP67DAFhvYSNRI&oa0rHx0icds3gtLKuO}WJ;X%?6`kDa> zak42A^2@C&TdC@xQKOEWKY#SX`6HJuv8K0R{`_Q&&6ZeKgGJ4a$m`gLK+0ipWI+@t zpj7KiB|6fiMARwg7N*fTCvxVL)qcv!$&Wu6@#<@%UJZ|m#;-cjQUvT#uSH(rVOG7K zJ$sgy!o>#=cY1e$;2?fTR%OCi%FZEXh0oEMO8L8wXAZc52*83SSU7*s?W_3KNTDCY zZh%;rmC-q6K(Cr8)=#WjVP7vRJr|oTvjqfc+oA2vi{YnFowfLQ-?()%JUpCdu|Gli zqBn_=Ae5slQ3J_9L)tl;sS4gGd<5SmExJs6d|bZ3fP#7RuHUpFDJEKW>SC+%j_rS5 zw=(j=$p^QtFZlXPr912D)Tu+q|9;-QtVc;(*RG1XeBSBE7LyVZps%9i8w|DU*3||0 zFJHA1TZQDrkG^XSa=n>J=+c-GD^WFgN$ zBW5!|8zykjIF!-M3M96_bQu=S{nDP4ynFq|$m1v5wr+*cYTUT#?%jLtL|%%$d1c+| zB_s^&3Y8!nh!ZmvGKl#}y@vHLFZpiXm-f_$Th=a%x)yp6edgWz*VqXAd69iceTJ?^{aOxJg5By=bvw5#g7e*-+e0+|L#(S&Zy0_;$XC zL*6Ww;vAwnighy(icMr_nKCK2Vz9iY0ieUmOW{P8)4;GJD}>MG0hDnuJQO8UqzE)Q zH^DgilEP~;>qN>f`0A@$k&)u&g?52l<3XTXq!5ZLiDG=j2t$m!B_i~3kG@N{ZaaS6 z5qsyZ#moEHiQ`Ob=Lzty(x}0Y-~W(!_ujb^C)cfAO_8yONl!|cJ$14!GedGisxq@G zVqT@Wc1tegB>J z-g!GQ;hvXScm2k-pwcDH7JY1N)ZYF3$Tn5QH_bwmI)686Jx?FS@)mNf$sPWcYZ-^hL{-H>_5xcmF}xB5$ApR;f{|WzXIpj(DY6jVjq$89Hxo zQZHVrRJ%U?`o7q!dHu#kN|r^H#K#~qr&H(7XHT83SGmgF_yikj7}n)F{ixSpZ`88& z=pjR!RI9Og+43(}uMR3$h?oI7mL1XYsU`=NN!ek0l_E)9nXPaL#s>+>O{&&-f9zYm zhYc%VxeD7N*J;wMZj%yU4rZ(6?P)W9{CZx~ zs?}GlSXHk>N4&Lgi$p<5vn%ewWKFIoQe z$~6Ux6vtM+O~(%BFND`EU+HdK!q{=|e)F%d>sG18>M9^o77B?`vdLv&8Ig!Tr`^SV z(0TJ0&=n|z;z^&7{n5ZcG2vwAPgGgfxrp?|mXIO~RbXOys!0}AVL=WotKpR7Q+8_h zptLTTJsVFoGMWLpWn_U7pFwXC0tXRKi{Yh9%rr2GR;sjJ=gvh-m+jZBN8O557A#(p ziCbZV@r}3NIeGjn#@!L5fv>R+PZJQf!%txY}h!YOgV@?mUm6~bW%vk(4PGU^y)vPK#?LdXU=4u z@X531aNWbZ*tBWm$M3(d-m}02S2(0ZGO~Jp{GBx*TEL&3{y%ETI{*#;Y(-$vz1$9* zl=|h#iz%CI9pGTcU9r+d-2#PBoat+9v(CA!9CQ%IOFAGcP4HKpkrtf4Fqk4EVX>TL z@uZl-F}u&88rl=ClLZghu`$9$vlLC6ng+tr+Jqv-7fCk8VZQl->AM2_7>Rq_m`Tv* z^ZNU4SpMVZpH4b?J0AF8mdS3*OwG<@-mGQia>K`r8#Lx!wt%3w(vRd>3*rC&R+jdl}TEp99!BaMSDq!l1vP{m!9$e>P3*C&& z>;OOULu>aqs1Tfj!~17u*x1)CIDa4*XUDKkmea8CTfmfbT{=vgMd%3!Bou>=HT)=e zc8@DPHO(JsVKLhoAHc5Lupl<3K1vSo^}&TV^~^$RV2cdU0vXaHpFi zooxrMO(@_}VnCYF2A2M~PqfQwDY8ZmDK$PG*^UG0wP(-1<;$0&*fM@WZey_{e3Dqg z$RvaaImNUvbI68WMyTMy068HOYBum%lC%$qU^i| zl~~;U`@LGVS-)#f;bKJ@dEh(K2ZK~4v{2-ER)!sFeo}CQCGQ4ZtnJB?chO$t`k#D| zlV^O%`?EJWNy`F}n9&3G3`V~Ie|igGWOq7DRtv*ox|x+$%;s5`7Sjk>uS~0l61B2% z0T-m3kM<)~DKqD829-?$APo#8Zo&w#>02V7gFmo1vCo(J-&5V(;>t^Q_+}>&%r96WL>H5~cxQFUk`gfjAVoDKbKj@sG#i%s`_wnq(UzKT|$$E76Mc0SxMK z&&9w={FcjDiqc^{tz)^V&@xD6hC%Axk$J{eq#~*>r4h1X_3{<#Hf-$Kudlor1z1!m z%V>oPlW9!GQc^|a%|^yd7S@YO=49Z->tbXEGs40b)8Ln1My3kPi`XYPum@=&tO;wm zkmd|@RD+@5l!tiKvKoWV8i!N=Q>OKO zZA?h}2sD%2Y~FhdNi4h1@-W~pjmR}@VT zD_1BoYd{IW`=u;^4005(1lHt1h1~+MpdO%t5`-jIs=E}FJdiL$Hk@HKZ{0@fSi@EANpx;rv*es8 zMC7UwhQyV%OK z;+z*Fs48ujpS%G|hj_+smvHqIm}bmNAQ5@XQV;=T>RRHEiv8x>c|60IO#+EVfZ)^w zwiFNbf*PZp%N^~={i}Zc?nh35=YRf$8hGY_n}$x~qMeQz``YPEzvz6u>2vRXI;mT~ zL0sRq@#kq1KbB#&$@u1kk9zg%hqZ`wE!sAnn9eD_>*P0$S{gdtN<3=NN97|e-Ni4t(Oe#o zPR}34`z~D8ef{;?BFhuAVu#bLE z(>Zzk=<6>H(q*MHH#_*@E0ibj|>k4VW~XrMXRiW+Crho2tWoO zlnBE~%0NhYKr#c1BwkeH7U$%Fc7ueFhC!-Y4hbjul2BxkKe^xP^!eAHUjtgD|M}_s zV^W|=(o29B9Fy+3e)GiD>u-JV{;bIpv8Kq%OrP}8yKlZR_Tsh3cRm{bugR13YRj>l z?u3Sd-K)nX9#F+YDehM}r)LShB28V{|I#x+8M+thIK6@XNxz~KSUjFDX3pq2Xh3MG z66=>Ozj^)Iu851vWQvK4TQ7#svbj9^BmEW)6yFte zNijZy#I&=@k^D%BfC;*e37auYAwUjSLP8uq?pV9bWU08KlJ!AgIuOYZ>1t3dGL+B} z3ETr-Q=GW1!nv&|ztSi~X+!HY;YJmx0G3FFZmgt{e1nj-thZhp(S7KEleccpT)bfE z+?kP)5mE|)5-|=3<4uxfn?>c58+3m7R7wC@fRaxNWLBot)Eq#Uu{LZA`{IpL!Y8Qsfdqe3{Y@Kbbj z@m8vpPqG4%a!42kNR&e26mZgFQ@NGeqe6K<>LcMLGlJa;EO}fqC^e=~S6+|cI`5M# zQwsT+lQASaxs&SR6*4j)hmt(0@(WuU8jwgMK0K~op4dL2wYx>huBUT-GHEjG0xDRmR;|96@Nrxmi!y?AsVT)n3c74r!R(i)$8D1r z&Xx#Hf2L<1JG2+B!(_A`I&`RFl?tpY+xhFRW5w9hv%kKJVH`(lsHTQ(1WbDRv~J$gpj z%ze(E_htJIZT9Tgnf^F+;J|@KZ?83L)}VLw>(e(+iPGqEk>M8)?%flY5Z|~-^VaRU zNPh!_G&9M@B|=?$DuL zy?Q?_S#kI7{jM)|V`Ca7WaHuC+)%9I&CazswQ1+W+_qaNx!cWy_yk{(bA zh>G@Y+s!Ch;=rE0 zd*b)p+icpue=pSq%60A0vqsfQAAImWM~CjcEHa*89)fIZ(FycOtnwu#VDKSmri73n zouJxpsG>s27Y}QX_%F_X5G{U>cP36Cd=gCuAgsQG`ujQt$WrnLljpbf^x_CLy=05lQ z@84bnIlba<&&OYu0aRtWJRC98O`tZK8vdkw3Kc0?tfZfh|HyaWefy2E^S_++;X7}< z^YJHkqpXgjEyEx(!asN9(5EAZEnB+C-_QExdvD{fW9g!W`TTq*jCg6&>ScZg{lxd* z{pO2V70ZX-zI8qMQ6eBoUlpIOdi~X_my6e_>CJBICUexyo8`(?FvBgo45iD4-iy7Z zP(oB2{P5}R8LQ{a9M-qLJ|iRI*iL*w{O18wiNeS`HHy|l-VL}5XPLylWuN6$I zeads%Eo?oXCs@O&2A5iRKuST90Gz;IE+pmr6r6hUtYEdGdgT$Mz?1M=gcO-8B!oet zRH*!7R3rltB*6!q<)fXekm}-7$o?Cq^Q`)aC=xMKid0hCtJ&6Nd+uzs>uO{eqq z`RS*hSeEwi&aDHRw;YVT;p^j{^6=itV@F@?Gk`6I8`Q2bWI*pBFAa7XEaHHho`5Fp z)J%XfcDgUkV;unay)MFdJZ25R(!fS6nqjGu>Ve z7~gC>H>+t$RC|wI9Dg@#Sa%wJEj*t3ySB0Nkc~0%Ki*=`))Pk$weR#o+U;BI$Bf0V z;mbo`QWJDS9ng~j6w%t51NHW8)qmidqhB95c;JO|r|b_ODhIX#>JS>pL>7Het8-?~ zDqFo;aDn`DKm2IvnzfZ1H!V`A@QkTbjh}xOeeGJ|!iDGg{6%y6LJbmZskU23tbz82G`cqdp<&h^MLGJu6^S=eneM6&p8c-@N$;AANZDZY&mu^kK}U*^ZLO7V1G!@Gj z=I6w?&0wrmx1IqLMWeB1t$K|cw_@54qq&Mzs>UZKdj}QhIcDs%cg77F*yr+vGsFh? zz@IXw2+e{x!-~1t8Qa|+Hx*yf(A zTRHK`Bqmgo$`9x86;C4JI3RKB#_ z%5`;zn{r=;(F!YJ)JINK#<)(jd^k}LNMm zYfqg@TEEZvQ#YTV{twr{|I5qxA5QHb5FMz9M$24W%+1I?UAw$8Xu#;#MuE=EFz!9D zp9cKen9<(;0ps3z2d(zpspG78r0KJ>2}{Gx7B7?8$`l(7o(0QBlNUAzxH(1?*c^6@ z!CSX&clg}-?%ldge&fykyZ18oM~#%8rq`D$S^VDZJKzqcAVFoy+>E@&)S}7bxE>W< z7El9L6w1TiFzL@TaLE`Jh}j$Ca|8)Q6)uUxe{=_!1t z%to77kCs+or{F^#(g5YDi#b=Q4@(h;Wh&KNXvV<&s?aLuPUx(*9~$`a{gxt*7ho_YXb)^;g%xf8AvN>RkMV>9KkZ zQ-`=)H-`4^-L6&3UOmDRV{gaZi3U&MMs(?q4(!?K%uL_Ab31UHziD4WMXc(8aK zm0LAl&Xux)UWs}sJfpA#9`~MYzp_+v*sx*Nyg@Nh(HNY`@)J6<+f%k=Xw;2JAj@ho z_v+bW`Hw$5jJf^zPRx(XSNC9kPjolct6-@;tbxHo1crs4Mi;~qgP4p~+0F|(Ss!`% z+QqZy>esE)zjv?L%jfmZ?9CfiA3u1&X}5#zWDp+OZO=+cxR)A#7dzyX2ML^OBLHLx zeNx;HslC(q0e0)HMu*I4id_n@jU8}mQu4lCyF)@k#=bEI0Tv%~`|9Zvs}}!2Zm0?@ zV3?3Z@LaigDU;nC9>#w@b6V-zwQ4q~i!BK8*gRiG|v- zSn$x{eRpD`!+LaY(yT?q)oTn#GE-7?i3y>_OK1?p8bcWzX+V>UN;Fyd$VKsCqJ1cJ zF4k}(=aeu|&@y!Zjwwi$Z)t2QH|h%0adIK&P;nDN0%_@J=eaS;RWZxS(VYpUGN(mB z9<&efOMcXclbl2NaS5OuoNEEJZ+>c5h?x%u*U7h(QVz*K4NOflU}~c7N>;SHl2)JK)G7Q!Yr_U5EUhIwW6F#2sA-f-xtXX^9q)9bv*Lp}V{eRdy5Adq0 zZQt*{cX}nEca)yc1*D2}R1icE#X=Q96i`70Q9!Do^xg@f_uhN&5FqqGAS9%>-QRD_ z9Xy_M-@E6$_rBxtyJUTvu=d(>%`(=S|1suh6K7eG(}hffb`EI#!cHhk~hu3W^;no@umJILHPd5Z0IP~(31!3S5*o^DmIHfo2rdiQGg z##_qY3b7v#T#2<4yknLj-g+Q}IZqBJNfoOjfd0YlyF+__jMcP2(SqZr{$|$eW=@?t z&YJH8tvOo5=!P}cy5}cRmtA{ZV&VlEe1Y!%O zh^H{_a%jMFv$BSL^Nl7cUgPWY{%3t7Dpwlu<4*+LA*~}DvfO|*fT?q4O*_3I8Uj_T zUw6*@IUd}=kSj4hFd&7P{Dr*%$_xcnzLNFDa=;*u9zV8{tWR!j>FRYyj+qFF*_X!@ z79L(8Op>>#kW}@8g3ba>0$%d86aeMPp3(;1IE+&~>eLfaMN~-vFn)OsZ|W=$rGCGz zI<3g;xsShpyhXI$SV*Y}u=}GSRF(cid+Ix7fOUn=fUIiwABw22)P}MJ8b}A}wE|+# zjei>iKk-E!o?@JOI%B>N$U2ho@BDUs_bZJ}1WWA#)>#*tVs0 z8Hc*?clYLh`72N$&q?6l+Lgalt`gjb5hHlT#cZcuUzdUKXAnsQ9VQ;KU~Y11U_cN- zgalKNi9)$!OCAZ4XBAq5M8u+jik=C=-1hv65id70mFQ}d+1E2CP?ezL0IvYXVP6dj z@%JA3?f0b8CNo8PS|&zLZ;Od)5kEbiSWG9}0WlS-mXKb>>lVjqI!Cv`CdnKP>v1@^ zJv2BFNAi?}2Q6#Y+IRD&-r_4kHZ&|jAWF8Pb)|a&_9a$Bwz-mlU|tlsmgy@xaYiBo zM_2|KJ@-l@&S%Mquo;y5)Y+;@qY0BHmakTmyc?)qBxqC|i?z=(GK>VA`+BkE7@>XQ z{*A^)krCmZ$WIRiAR9rYqGTffwBQsJ6zH+$I?*!u`+AX$RBO#{*ZQ@71NygY{|>1? zR6sn-9@UWedMiLgeJ_;&<)W&GBf#hKFpT~XlPy@If~16Th*P_0TUhs?-51UzunIHa z8YQU{daBzgr8u$h1}LaZl*DcFXwajHE45TBHIQCJ&Sei})-^&&va5<9Pt`grb@3l{ z%3sJT#$wJDN_qfY&J{jNvmSU=U*+9X&q7H57dkx1t=Oa}4WIX!6S~cFB>Uw(o95Eh zgvc`CB=Thxf5Ji2FLW;!!K7q*c%qgO9v6i@78dk2Kfd0 z@YK}sMVEjLmi}{?SiXqEr?4@x$-=i7jOskW^jw!HM9%>qp_9-LXHg>ZMPDR8;DE$4 zmQsT6JS(BCEZRIgo}fyliJ^ni0rXukS1Bju=>S2kQk1^SH5!0gDqh0?Uair`~ z=yLAcwsqC=rRY;c-V%k2h@Zfj)t2=v@hKrQ;u}K6a-gtR=aO^E1&;)fO)mTXUEAXB z#1bqDa(Zx(4zMh1-=*8UsAz}LV#kdKk}1$jh~UiJ;wKLB5HYC` zhjFwJCvL2dz`Y31x+jn0H_n`Hkn~F;jVZxU?!nQ7djUR)EkxIhibFI93RkEVVbJ1D zej`q}MSzi+O~WoS1sd&E*3E*^B(zfVTw^CYxT=jtU@Y|0fy#wGz*z(aT%>A4rUk_e zM$AY+IWBnIk3R|5IbnE=NMSXMh;O!>lA2bP+iv`w8LuN*xVa0oShFjN^y) z(-fvokA_`_BXq$@T(IZr#S4e_>=6wMNGJ3|<4Q}m$rHzp8}~C+Q3k4sI#}sa=n-%a zNHRtbubTGjZ#%Z`klf_-(UvQ|h7yp8Tx`&pETAH-QMrd2Fix8d`cSLvD25V==Od$o zZp7$=ydn!rSGeSGkcU7Ru&_3YQK#TD0=j$k@@0?i-XYZ&%$ROKQqWf30hNLZKEPdM zu;M4)4VD-*1lyz`O0u*7x>@D8@%@+NHE&Py^@(m z$cW1*R9p1!9}Y!4ZkNMz{tpH6zvffmzo&q_^DoqaPa7%@rWDYG0FC4Xaw*6{J_ebQ zj9FqMh1r4=C=-gvG@uiOOiE7W;=2!D8OU|5Ls)Jh{It+ zbTql`q>E(>dB96hKq4CjQ-LB1g9rjNZg4@g0Elc#76c8Q-jX|2_F7?z(~OA!T5|W{ z7|1R(R-|81PN0XC6*~KqiR&RdS}E8QyB;t}MmPYG(O~v8>YNX6UmeutEuABmrclwM zHzC^)iM&lEz=|yBM0uqvT#0!aB9KYG60-dsP{idD0`wk$U6u$*C$V;Mg)E|=R zNx`VBdt|pwp)pTN_$kD8nl)}TG6s7tUNrDDqtnK{lKe=xjYDNzfj*E^mLoi6s4B`8 zXD-{=^B^qZ!R}y1!?37UD5;hUByE9+us`VFa@aI3tC1bZ7#lV7hA=nC_lqA6#NjEE ze@Vk8mUu=A(DNGLVd-NwqNi`){!P{#dvP%#0 zb-`;tfA-9nix*xlS0?uQ)!EZ$#>K^zE?ZW%0t7ppjztUSu3WL~ zcFeWPl`5LerXvRqUO9g@J}z#-f;ms(A6BbUY4e8l%a$x=JDSp^OM`|mnDmrq%N8wI z6%}>u%9SF83;X#8A{ee(y4aqRxog)BGN>Fse6&pIlD_`_;Mcv_8>?0jI6BErAn~_*2XofqLwZ_arBs%#e(ZL5VHZ?u{jX#R?a6s5x*BLTCj3O)a@JBs#L6qPd!6UczAE&{JEPptRpGPzMVT7 zHfmT3mw7#D*43&C4k~t9`~ADO*F`N4@ba2BclOmw7unjM46sew`2erKL7rZ-YU}m{pJ5b-(ENt&!k1*wdZ6- z{W9UbcRI#iyAoisOd0acnk9?OF6)Kkhdyc9+>yi9Eyh9by>s)*8D!hZ6UMGyv9dy` zQVXX}nlgGMpetrI_P5cxRxX_V>)_98R@mZ?wYb^o+}fl2d(lf5mMc-@z~)UO2M1@d{d#;f z?)%|k!68rX-5$`f-Tv+CeYDvq6DZBP038J)YLD*SE?+!TBrNcoem#%u*^WvY2twzA zasZ6x?LkJR@TKz?szpSi!p7ZY`J@Sb-hBtT`1FY*J-T%2*{!p~k$L>&f%b1UGiWl% zHn=Kk$>mFDA?)g93ww9#eD?TZwoDk*rNhCU8`=24(*XPxxm_GO(7RZzQIjWr+~sYg z_;bfk_I|H(udeU#1Wz41+NoV@qr-}j+w+6>qZchISF-57t?RxW{5dcd`P!i2I>na`+U6-%encdDu_ioJ}{nK}Y2KpEcOBc-R^YI5?eLf)A z&u8wenG=5c!K85{#>Wll{s98D!7SF8YV`7s ztXwsj zy%#N*ueX>UKY4U)>$a0IF-D8|(c}1&CysV~r{nU)3+&k$2Tq+rlm5l0pVh5fclnYf z-+%ixDjj6u=ymJSb(oyar7M>A=|3PNBW=&hl{1$v0UboF)M`tWD%G`X*A0u83>f&i zmyiFs<3}#;*}Zt%jw&oQbk47b44OJ=!uIXk-|g0QWmI&ZzMqmmMC-Ke-@Vh@;>8wg z*Ds#jyK`I97O$%X139ei29q}nDHa*loSck%_sRLEte(70&d>Vv?%cai>$Y#a_j;4} z1`X={#g}*4JFVd>akp<2FI!QJwgO&k{;P?oR&5H2Vl=u(aj_>3A8yp*P1H|H#2UAg}~z(1wH3kTeY%d(&ovy|?&W-V|8CLX0)tvbtR&A<+M>D>8?hmLGpxmF0LbtTuW z+joHOz`p(C$B#?9b;CQ@FZ)?Cawtk$ToJzM^XZ_UeyUyfl^k(!!$QcOvuyZh!DPs{ z3YDu{wYpLjDkJ7a)~tocZ#fc=WMvTCax>-{nt-;iHc%oM4QVc3xysTnqGlbf*$cIF z=?Ya6k~56H{v=vrr`@gFcYwV&Z{BR&xFI`$GXem^vUi}7XqEr{5o7I2{Sw!zG@r(CjZ?lQS%T=n_i6?Sv>ojP( zBYHimZ>tE2;+xFc9v#KvxpNmTM4^$PbJ?rctU(MU+phqvN=g(rJTIsFVSGH)3NKhN zo3#uOsC8GZQ=1%HCCa_T*sIsAYtkmuqH?yVg<&Wpf+ifE=AO;Wz*|QRjQ2Uyk^}7xb2lJSsJbU<444G-okqP zMuSGOhDW5<>MmZq5Od?kt4*3}NX)A>7AaZQr%-Wmrl(OF0x46n)Z_TXm&FYn#hdU3%qFaK#H`CBfLrCM8RQK_nZp(3a+-Ht z%KXzQn_nQG0=%y;AQ6~P2y+ZPg8*x{18;$tm}~3`#?F3Kszz3=T&2&a{lt3?RM6{F zpCy0WtJkneV>`5OcVy4bfj#>Q=rg}`y0_l#Htn}*jheL#E?$vNB50N|soC2)a2@p*>PGx$s zL5WL=M9uj4`J#aZfW-krNdMQcc0ILQo?y>Kr%s| za*i)2o73CGBZRS(D_0HXJ`-GBM&j_@k12!;`=H4 zc!^Fz*tdYIbeQ%SIBFkSkr@FU#3@v^mt=>H`E%!yL1p`n?L~?dY0HK*Y z_U>xazCHUN1D?1Wv->U~he09Xbl}$Yt6AA;)n9o9w3zYB__T+2Zk#&0b@j?OJG{-t z4xzz84DIfX7*+<)5}sHJgcK`M^hV6hRcqEMj~Gb~A_{)o<6tVe2&oAAX5S*kNPp;w zS7OM>$>ijc;X`azBe=i|kb&2@0OFkv0NZe7;fmU>g%J?(C>v(7o&C;D>ksVR1G|FI zfE_kUasERFmkcK$IaX4UN^>1{y?FnSlhF}TvzErsf92{`cyXj8rSIOg-=;A%YuP6D za`udCduQ#8wWLfl#sj=tJ=^!bcmf;3Uj{jweJN!b!Gp!mTkT}3{Ad`yq#2+Wuma)h@+9+*dSgA5iI=|Ds zUHg^|8W%2625}G71fm~m2b<3H>x}t+K?S6r{Z_3C6)p;c0sQC= zuFwoTtWc4WjYt=e&;slV2Z2;*N z!5Vx<+o8IoN9a~>qMq#)tS;bEgj`~vq~}WBgq@)-XTd^L&To^)dE821bKo`5#!eK3 z0@)F!`56z=gxY$>|&b%2$XfQJ!q*4n6fYd6WhswYq)q@%hVF zYE`JTXvNCnRjQ&ziCVep(t!g*zWcsHWVJWCfAYl_gSh+jsT1bHMf^fS$XGjiz~_7S z?uFgLo!~XBb)W%idzg-iQL42F?LjU20Cv+XZ%4j@`=29LehEc^=g}trPI53OGc}sS zv1wP5?IDFjO~mgpBmW>>2#oSwB(VSI!*jE$=y@GX19C?^d6FJr(v~mbXK~_Q&2Km_ zyPTP+4XadKv~_FA(q&Lh`1l71o+3pO<)P7}XJnxV_V)J&ddy5J7w%_SBvXNoj>EM^Lo$%i z@PeQnLN@gibcZi9iTXt1W7*CvC|JM}38R~; z2ql14fSb0kXA4<`#%j=ioj-XKOK~^#L_GOb=pamq8IYf=>``G?5Gz!xcy;$c_L>-dCDKzW|Db2ZvGyb*i5H7;pdsLGzRm zZ8-H)2{H7rA5W;g^ckIt(JQ=AQRICI@0L5TZi%>!SPGuS-Muq;!uXe7uF$YSV^AHq zP?(1zK&BfQ$EK7aAwg8d7QjgHXovg@6w>+lU`Hk8A--`AESDO6Kwu!%qL{Wjoc;kp zO3;RLAS$w*u!J?QA3Sh-u`B@AG&)+%6<;0u;1(cp_EoHzLZhWNo#y?u@}+T zpqDHK;HgW)4#lu=9uEWz2@T7|nMQBH?uvny=6I^2r#BcF%ZioJVc|uZw{0hHg_Z;? zzCwMcOVW_T5F`{61ryb&Jn0ewsv{1QA}E7kAv_LxBnY2>dW}!$o+v!OYAdA`S`bR!mG7;*!cu;RHXCr#r_E-xcqKe}vTgI)$cV}m$Vg7L zr6y`}vrJ|$WKpn5dWASjeIiH8WX?>_#DT<%Kp>a>(Y@Ol&youG`;*-RNyn^lB|naL ze5fq{CnNpvqtT>@%&YaLJH?j}Bo4+;(t zF%a1pcWHPA;g>X*^2~w{w}F5}Ff}tHDZxu`U{lIlH)2YaC`K$yN>YL&JKb%y`Vij2 z4si67u5*iG5wFxoNz=rLUIZyL*BXEO7V8u*FE;Jen9SPj^ki#Bx-}zBW5-i5&BreQ zu%%A9J2(4I%(a}XoWQ`K;E=G0>b2oipgc%t&Ca+Vd)3d|%fRNr_-3&7LEu+OX0*24 z?2HT+3bbY+%qk@%J1fItwya;dyjamcI%!>Q~od3w2I{+Z?R@+w{*5-R z-+r}0$A%4CG;MO}!bNeD%gt?Bt47nv%FS!lXkM-Af;n>-6hqs4{){6x=YtL%y1)0{ zz<~oj6(Y^CbimuX_a`5=u3NKn(?%WMXqT4qOjaIbQv!g*;OA`hTGO4|wz0qd>JBjb1L>t;1W5D!jaD!y5EkpdC$$x?ZQ*v|&S=7A*yl zUD~rJPc*MmnPAtHXNlv74352d9U2ZF{(Y)b_U!D~8 z{4jEO+gddb?%ylMR3m9kk+Pll?%in_QFXw8PpSFp#fy_hjL6N(L~GXKwWiIhN4AKJ zs8_qz#PMT!j&b+zwy5_Ce(3U+!9@R(p#2}3srk+RcNBQxfE$S#&9g`fdVSx~llI+w zyzkc21uHj@nIbnU@uwjJhxG4r{&LKqA4ZKEIozHN$dW%$Yu4Cfubx{pZuF8ZTTWiS zRJ%#zQ6s)ZvrRREwM<$zlXPLW%w02YhQ(>a8Bk-_`nug47R=Mwa(qY+qH#5B()8%n zSRcP&Sv%`AUre63<=VCHzW6k1%A{-OFCxB*Hdr1O!n|(#+SNW9Ysa=NQ&OM3-R+}8 z58{v9x>YeEa@owuW=FOOW1ic!WY7Ksv2pwE#P{kym?b?5`T>3V1cU_dy>owm?7fkb zrn34+cn6~hA6gT&m`}U8CB|BfjqE%FM(*h*lkq4Q_&39N)k~ zjpkue()c;^j>g{GcJ0QKxY%857I~AtL*uU9w*8UVd%N$Z?z;KJU^ZEt_M3-ykMGy> zmlaX_FI?Mk^G@ON73@a6UFSq*cW37g{^a9thJSw|=63&4W5y2X>&!?5w($bNDaFK! z#F=YxXB(W^iH~CYc7Lbq=Y2cB+chY((4hy9BkR>i=4U7SCS5uoymocY>ZlVNqt~xk zOa>A(7HBUZwdjMuU-7>u*^4Ii{>@*9f2}pSh@KT@05fwyPeXE2N~ibVKk(r3p*!~@ zs=Tyx_9QE?ZW2Wd%+BI=dfRd}*7UXWXBb>IFI-3LZi`m8e!(n_Bg4mT^K-iEz24;T z^=k&dU?+Qxx^3T1{CUgyi=Pi35e94?`HWDc=`ex=0+Kr6#qJ;|L+YnwLNEFHjENntT)TNu&Z!F)i zVWo&jJdQtj|HIhhC(=_>xJ@*w2wIr+bjBKW>!01X@80d(3e8=qDamW+&x@#28)&51 zgZp=9O`A%vj)jJtPCv6HJSeD5>lWDk5)%_yfr{uWYH)DDId06)!-fsR2d;dDN}apD zhmjCpkwy()J$>#xBft*N1FxeMt{ynX#Ubn3Wu`!-R=0+GNJyQ7KLLUib>SFetmGkfv!6$OhFcG?}s z4>Gkp@PQc=j~m%V1d_OdyNz@Svk4fBRdtNzEDO~F<0s|ZQHgZsuQZgkk$hoi;qd~V-b4-1eq~?YNJ-HJsQH*f_%L(@zI7No#x2#BbuztPMy23 zDCpFwGb^97C(m*Vlt#ycAyFi4;5j}0qwXIxYT35`z%Td>X?O0_Uz46Zf+iHep5gCp zsavaNu}T$@YQ=gh^$VFP$y!Q!j-0D;e?9Emk9+nkU$Fwblf+S;;X<}bRVuUY;r+la z-Kb&1^Jg!hv7sSu#5l=co?5k6>z~-a|G~Zc)?7Q3N9tcWe?iq(>SE`lmH0ci=gpeQ z;}?~T+wE;K77Ph^vvn&vlJGQ%EByxy*uHW_T-*az0x#Zdl(#DX_&+V_*fXMPkAeXHr5$Lk8lKJ+u6cXKu z_Z&7ijVmrTwoHZc{SAVKz*>c87QuKmZBeQJex|s^RBkWga_n zq+Z>IG-J@4ub;m-ZT#3PCy$z)_L__T0n=%}$1U3zc&V9m{5I&Mr> z!ef4dK_)%CcYNQju_L~VC*ZzAdu}A`MeLBG*_RaIjGfM`iC+FVA?f43qzr?7k)~0c zCp;5_A~>ZmNEQJH_Z@t#aWf$p?+UyKXH79+y30pKRC%@D%4JKuv@X3pcm0}`B_pdv z)~>@(_(Ue&j$J%@6ck1o!rG18IQuuQ{ic84Ds>yZTE7WeChQGGDpiQNepNW2^eF$x zr@%j_zzYZ5%uOZ~vSIU9Eh8hUA3C_dW8((XCr@G-&E#s_xG4o@Pnlx&_Nr8ge1SqA z@B)KjV!~5#6+o4&H+XsZa5XbC6X6{X29k?Pd>}?Cr~Bg`pRAuhH#aQ>u$w<`PMgeyk|NhXPeWgm4k*aV%knlNtShRQV-}5O}%*)qD5Kv-R z#0zin;>Aa{Ztc;xKah%}^xOUV1%-qzSg-&c{khX8*||OONrF2&d*{xbwQJX2vT)IW zPy6C-MzzR&4!aFor=M>CoB$UM|A0VEZY~H!?F>-{g+rmuzA0*DNSTs#Ua1AUqBo+p zrmqrwD7}Id6DLj_G3@*JTd~2x!FES3)bU&w?E;zyG1rQEn%)vwKXl08ppXJhTeOA+ zRH;_I^&4;8fB2}^`(4Sqf>tOcjz(c{q5rA zQD6NuJZi~Y9in5dwQJ|jy+0dp_}c9^-s$o|o7T@BJwONoCV&&L!=9U|b6A;$+1Z)Q z=-|)-Tt`bva5VNm5^JDsY0#{hcd??|*R9^ab&EaM{(AE^)F88yS*hGRaF7r~ltwpf z)aXMO&h9&T_0W;Cb7#(y9MG8EnB;EV{kTV^%9a8_(3OiWRqHyiW836!zni{lUBTid zN%)Tw$-u9_A2i~pO*?nhXz|*TMRO6W36K4Uyeo^#0=8ks0aFzWHV_GA1t%3k9pv9e`*^smSh2 z)~q{qJ$A{K9WzFcKDuw4$(2n9p%31HxWT|8#OoP$;B-v<`KL1{kIY%T%w_R*8VxmH zt>1IdkWtfSMMp>7JbvtU>{awKVDjLh-+k1t|L`#rs@AQyZPP}PI(a~Dou9W~R(b}* zmmqt+Aw4}cAixiAc*JBb3b)Ye{lW_N8$5W#mtV$Qz9d4vNc|9*Trpt3ff2NRv*Z4w zXAfV!RiJ3;&-)M5yBsE?F4r#JgTf6}gy*+Ihm?D%^ut?scW&DB^kLl1YgaE{xx@+! zDxr^JvE#DY_wCp*Y3Q&it5=7WE$6Wx(f}pso3K8ev0&*0D2>6KYY~a1E(%-J9Yll>0@)IPeJ+z*9#Rb^5D@!P@YEfk9-RJ zD-?L)fLrM~1&dK}*lcWx<*HEeW!uw)lxNQl?Arsb_5QT4m%r~90|zl@&z(Ju4iuML zhs$2HMB(dKE|@h=qsD$W_9mbc92Sn;CAf^10SpvL5Zu!D>htNmIkVS9ul%@2AHCUI zWLg&ZASPlQksUSyt-*1mHu2F84KLE5QR4&q_H#2@8H6-uIyo_J-@Xf6+HJWAh=Kly*SX&E_v8G$m}%9a*)a$B&{&(c(rQ_HoC@*&P}h3I--5B)FZV zW|HxSmoI<+`gIG;!0Egbd#l{b6$Iu0FrWzZ0rgy(f&~jLTDIcw#?6x_Od$GPOst|4 zFwj*(QxG*7gvn%xs9YtWSn;!`P9r$eVb&!e6jcE`WWF}K9g{}?xP9ZQ^{ZEghlk?? zjmr(rb_@&!N12i((A(U(bH_h0040sl#D}CJuE7xLTZ{DH+qC2)m zTtpA=+40pU-KQ;?U;mXlREQNqSc}k%fs3((=cYHBd-Uqb>SyVa#l1fH*vLLUP+Y24 z=#7JlIKyBLmxTcO6YK|=0S+BLzb7R@-?N=b6;`BpZgLW;Y8?AowQY6f z;NFX;jyN+?=FObi41*yq+r)xELdoOr3EpNOFw>iz-br~7clN-cw>ozvF&^NG2QhGg z`!UWVi+->Bhu|8&gYXDmWlNXNc@USKnS(WN|IY10dv+i3 z)9~^o3nxCflNcY5F8J}?TSxb7H(0Z-o;k^G2_Jm$0UCtn&70d3pRQOq&yk+GZ2lY~ zZHkvI7k~He{yn>Kwr$(K-O7b?Qy$z(e)NERuy1$nf)W<}y!cmuQX<5%ymytWP<_;_ z`KxD4TfclM`d@-fP@!P{B-awDP?F>B+&q7J`jlVN?mu|zt#=a=GbfB0WhbmuU=G#= zTe40XGjh$cc~gHG52YVJe2|%(NDA_UJGLg?yOsSkVfwVGUg6Hh|y!HDZFI^t> zGJ^R3>_1}EbZRGe- z<%$(({vI!nuql8Wi!U*jS}Mb3M&F@Rm&CjG&L2P4;hoM1;(DFr%#!Dc;|e-` zlCZ^V>W7m!bOVp88PC!A4l0g9Z}$x~;3{_@L; zH6kOc)$H-rSKT{x#DAoy$Qy&PM`KqOG^(7D@dTB z+w4uwRS|DVlO$$NlSCgHOjCaxGj-&c!jYBx3?AIMTURiN+A!JiJPH&D(HAIi_Uzd@ z^%|{Nzm7Ube)p}&c(h%*!l6TlQd3jE1gr(fT(7ok_hzR~BwGqER_y!9Q-1pLi?Jg| z8N&+Cm^aVp9LbXz8s^ zp9}Dsy@Ah}a~8B|-Lhw|-ZSUUlLZU0Hs~l^+9TU{wcWncT(H1fIHr8~2}+4g8`t|- zd|2!tkJ}}wm4v^dcELFY>jKFXrB1#1@}+ZQKOb1LY89vl*++c+jX<@=;9Io#&XX7D z1}-V$PD5NM`YF8;HMh~nxBaL6vNO^H!ix|r%1cIb^cSXwip*p(5t!=MnJ14PH+jr2 z#i~YrHsqUkx^#nsrz9uFUAlr6o@!<3mVe|^;9sDC(hdF#<^F$){MB-mFc#LlnC#LL zlYG4`?53vE;_u8VmNa%&#GbU&jNrf!@wP_KtJkKbWn#@k|6tZRvU4)6cBhYTAo>x2 z6d(b(nZzXULley{b}dnbXg|rnZD0*8q8Y|6aks#YjLm;h&;roXK6y59YjbFEC*cJ3 zxKcb&7*No-wy#+^Z}yz%tvi(i2?D!dR^IxX^|e-?OIK1;0)zYsCW7n02@nQxfS?aa z&r+}ol8gfd2%9KoXWM*z0%U=&ab={X8qF4q1s@Dp32F&DO9uJ{bfRR(eCN>dKQbia zCg+MMfS>|xgInZV!5I!6JnqWWhQG(ipMDN2;D>UFN^A}isz?$%wq&HMz!v<5(bHFH zOQT7AnvCuxAT)sSVuD51$Mp`0G&A)Xo{l(>2nC#Yhi7{E5SU765Wec}l*F{4uuuX? z+2WWmbFqJ-$DleTee)k}DcX?LRuoV;Vxw|`D*(7285RYKJaJq;+^#lFnv5SizGAJq zc;}21^fVZs<#y3&sDsBN1y!RgDOBC^gSykB8>cY} zOBdz3=RPrpN^kmHelk(F@1onIs^z!cT2*!9wDNz$K;<5i`tWQ4l-*LJDv$T23S~+`R(>@d>~p zW@Ea02Uuj5;{Ty0CbAmA9CpbG>XF^31L0FoHfl5^pdzzV@ZP>K=jkn*HmHtwjdhK!4@^G3K^8os zN#jIZig z1h_HibBQk@`r6G_J|b;H8M1b;+?SOAJ4NuG0*d7EGZHoKC{QrCb*omu9o)-gGTNa7 z%MLO6i6NF~B9NP%=~25AyB`u34ts#H$oB3I(bQ83{xUv+p^|IYWXQ7G%sRc7AGy6m zRt28vTVQw?=#I)*P+RqcRgEV(2_sUEf$kHOBg}!Ju(*L;K`I%q8nsLK(^tpZ`!c)$Z-U70-Wg2@0>s?$UpKa@PDAd z3ny^{$5^hILM)b|WaLK?JWxsII* zOa_A-4`O4YSFalQ{SPQj_zkhq1lTaKrKEfnEZ{^K1alL~8nmDykr6QoO4<+(jB-p{ zRfa?s^n~~qu;xkpqtz=G<>cfH`)MroyGaGb77b)-rH2UEAROHVz`$rw2GkI|7EVAP zsh816unFJA85}8G@W(DVh_@f7l#$Lk?I0(Zqb!oL1OUb)C81bQ)jp6OA8T~CRE~Qb zIiX0FD7_Wt1d6$>Y{V`(?i5}NB?aqYCm?vP-6qL7=qO4Ckc}M}kn16aSZGC|B4V#Z zlM1I1YQtHeo-O_1y}Pid(C{##=_Sn|2@{2z2({!9Kv5wX3kUUG8jw6+uvPVp@^XGL zc%CE8GCJWTZsC!F`3hfUDNJ+hzZQh6w4t~S62AUCZ z0k{}A-J`Q?l8G0GC{O5*(`g||sc;dc91#~1SdS0^*_Lt8rQbOdH%Pb=gy$!SBQ%zt z!z74x1nOvxjptu|rQW-5wr^LvX7~5HVrHaO49f!l!j`zh!$AcZIn;v!F8Tc7m8K$f z2IrZQ?Cbi`IT0sVIHw6$q#acFL@2B z{we+;T~HT!7oe#J@u`8{7)(Yk>dgiy%L)$?<*{&SKmW+5z<)=9KfLAtU9tZ`zlq0Rfl+BAbAr?WHU2}VwfiB@x6!V&U`uOo0e@m*oZb1&8>h1?Qs}T-Z*oCRC$Uh#}PEev(7HPRnjw+ z;?VZ>`GvzHL74q=xr$7J!|i_?S8)A z;YACH)2_>TXxFZVb7!A9b*5<1BEg{{Wc8adZK}VI_sSJfcAKqOk>d0P+=rKdzw1}6 zSRJ+ew7s=fFJCNOx)J;xn&Q%}w`VV%Hi?YYGGR5EV|#ZT-L=K&wx2k-_osfn zRxO$DYcWjxW>EL8??f$H80zcyU5}n?R!2)9kI3EH-k*HT2DgBCzjr#`yc&b+--_8& zhxO}OFf=s&!TonzwPee1w=FyBmkGVQce@pH$!5)F!LB3IoeLS+{^`f>Px$WJAU~hS z4{m+lsl(naTfIpWC?XgM_wjWRxDv%QBC6ZHaQf7*-s_y5mW<#(YCzAiBffIka$>K? ztX{ep$r>!mOv_j?aYA-VlHQerO_2oJ$eY}K>gb`dBZrUpW(cW{4(!<4xotDOEyJ1l zZ29=nw%jz*5}I80h0}k#arKIstXmqbx0lKYh2RXT$mTG>jBDM}gPs z{Wgv3ty>jcs#vj82X?+)Co<{o9cyaRxF5bXlM~Tl{c-5vF+U75X>xV;^a(>gGud-T zelvLPuahdhR5~H6loS1!-j!Ma(^HL*6jz1SJu7^;+33S%}Eu0j-QctSp*T z0cjQzJ(qb|j!~g3lb|~*5LQo8Sh~lDRye2TJSke5L7rFfn7=QrxKEi#DJvkS0A2kr z=P*|>@GA8jYaLmCh=fmL`A0ql{y7DHSM~k#qW?2Wh=`=L)fm4%`k-sSe*GM_jA~_G zI(h7H+qQ3@;U--L7DI`I#U@Yt5EC$3&6Z^_fgPfwma z{_eZ)P?(uHeAv*YO`AfcR_)$6uz!EqQpKl@{Nd**zqV=LnfX~Gvhu~Vrz%&iLc5(h zcKG_6VeACM3LHNRblYc-pSZMV*Ya&!Dpe?lQ25o?Ls<~)-nsLWk3L$nXnvnQy@LuC zvH?TL+d5rVM*8fZM-Be|`>s7c0)ZMhE`X=l$;8h_pwKvTxr$+}}~$ z17GawpfjO6K^jFh<@|ckmjyyXTeN5)_zFW+QZhc*vJJbiG8rgHimZxWif!g zJkqWFF2xYw*$ETgqm0jzJPO?Bx_Amof!}YJb7AnXNC9Z-RpPYA%&Vr1dW$F~CDmVG zUi~i@s8Ea=o*!sIPV!t$0D37wb7B~fYWYYjcos>naQQX=$fv+RrGUJzFVrD|7>iV< zna&j1;I#~$x6>3-x=z!$2dSuSAqD;#Z2#?YBjgf8VaPGMvkclQbsM_PfeyFj<(DHW z)N5$=^UE~oOIE4y@cu0eIEinB+fd{6Rw*W5G|lzv*FST7fBfAWOwgg9_pe%_NR86P z*rVn2nUmsPr`6SM+$7sz%+;H%SV^5`qstg~9wL+0Xwod#;G3=Ss!_KQqFwymM{0jb9N6qQJtJZ=6n3Jt zyGRP{%Rpo-I)l|@eyLFt?~sByI$!UgB8AIGoIP>IfF}crdM7|HGCpd5E2?sn!KyWq zMVcUf!AqeGodIo}R=uuX^A=WvC0B2{em%Bv-g4ckk>af5cYz=``{7xVNK{0SI zh0qcJ<8-srDJ4DVD$+M%u&AeyveD_w$f%9WYMAn$SR5oZqj=Bi{{S4=||MA$>b;|QWE)TT1)=lk^OQ?oc}IpXf$4=P_?q;F-S~5s{&!JBg-|+0UdUV+AK# zcTQ@mcTf=X<>JMQwd>a{UZM=M=hwOOs@1AP15#m;>V^C(~RRC+v3ngnJrp`eUv>3aSzIreTnRSF%ROHXnsL~ zUOwLO@$p~;H6+A8khMEd&lbQgHYaeIQ54_=;&x=9U=f!RH=B{i$HYXA9HBK@5Tw(d zJ@fVTVJ9ue(*y*0QLPB_iJP+n)X+pqylfofAq%#7at0T(V`VN*AkikEqC z_imwLB~XSFe}C}sq3rDJ{-1sEio5EmeVSf~jAMLdVM@UTQi#syT~ z0wFJrsWfCnT?w_QM%6=4rB!hbyTbn=f^^3VUR6!1LVe`^zet;$700K4TSmc-b&7vtC<0eM8;8QhIuYs&6r znNJf=?%jLy)Cr_JBic=bDdl{PWG0zJBr__0Y2<4{?lTbof))V>m07<`bR<99wsq_E zQ)e1AX=e5bc(wJL(b22ewu*4)EyO^o{1u=RaLn|EQy>Gf^Dat(n)T`-G0psCTyk>RB&Ry9(9I1#o=5*vBJcFO&72(zvEnK8T#qtZL|9b!W<%hAc^JdL16;X{$ zNhJPurl#)Oy(={#e#yMK%yGB^8Ab{gDRle#WqW#(J2%6qu|2+cZu|PEoRmj1ejTsz z52#e5c42IrLBVTQE_G+8E}Am~)CBOr#n6zDySHv^icbw1J;6ZlE#Y^G!P5I3hq!9m=5nk1STOkj1%G4BU(!TJ@5rQiE}ux z!@%T~peprl2uvfGOO^uAS*x)kdKxs=JP5Gc6)XS%KmbWZK~w~?3ss7fjX$m z$;fS7z2?XtM^%2cp^`jUag)Yzq}fIuX)3!O`9~^d-w>k?QHeSyHD&4jUwqYPZ6L1h~Th-lb1zp z*tF?&+?G4M8BwEV|AB)DamiuUA^jQ*BL8B*5|EII_uKRtIk~n@EnBy1)Hpdk>z7}r z5g8v=xJZYOdwtQf*XyIt`YBB})~kT(w-? zhCV^z=oaQKUev5k?OJu~RjFR<-7fE4yB;(4i-Gs=-(!FbQ@reT`ltm*^j)2WkCx_)VKf5pWoZ!H(?~I7wTLFOLHt7WL+U@ zFnGNT5bBwXBr8_6Jtv*|B7gqaNpAOa_<3)CFO`4tZ=?XP-9LNm|KF|urE--Z;Ue%D z#S<8vf^2pI49MhD6IGU(PFHGLG8+Ye3moiH$SPPgry7mTEy;=4OP3gK-gaJVuopa$ za76S7r1sC0UE57&1az)A*{6ymu1}sm!x7M6B0dF~4;a>y-AHB&cY)ucjZkPTJ_R;5 zz9R9r+2rMLo1_v}>S(O(c0%iHVlq`;4|cm3Ql-r@PglEJiAYi_PJJJ82Z zL~8W=vYl@g-{GltJ?AgxP)^tqMYo^^mJc;f0M-vmkkNxr&35H z!vqbeN}z$I4)#;HR$5vb#!O!?w&x^MfFUzG%gfIXP9_mX03u5qC?#Pj=mkg#&XWLr zMbhUvC|_{erKBiWI2gE{ZQIF|0>n6o3GyWUkmn~sF8~WBTfr;AJE#rRaLy@z`8^MD zc*3G*kYehf0hlXVB~emIM+7kQFi+kMSU2b{C{;6k?T;LH(_6&t0AiOrIQ`=7CLf>1skm*>D1i z{+xp$*mRmCrzRur!IzDS2mfQMf9vf({_;QN`wOSzMm5b-B?TNXKnw;-JXwCqCWH<) zDr14HH|c!?!9VoKci$2$Y*06|)ue0&pmTc#hR_9KK^X{&cAL$K8eMN;O9bj>5sW^A&<<@bIaZ8jlg@-L zg2k`REm;7)g9>nqT@(%i030R@SZV_H7zjX*0l*U)ZXtPy0s}Rt1$~@gs1i9veuR^; zS)#=PW`p=hy1+CR0~YI$0!1|TMzL**XbRWDdxZ|WeS(7ENs`?KpE!-q+uMhHE-*IY zk{}V9ZXkn{go64wTY%CK0=lVL$KwEzsZkyU7)keK3k#jyL4q$M8EA>X>oqpsQV!6Tia_H3 z-GM{_q1552BkwONm*0zfc!uXm;_S42V`C=I)Ac4BV`FY?+qP}nwr$(q*ya`6$;M7L zw(a}+{h!b9yqY&NJ>AvaQ#I9f&Ve>g_$neb7P0vDFf2LQneeuIc!=5cu>06|Cbqq0`bkwdEX*0Cp6LPSaWhKb7mx=H!BU`+GF;SkOeE8U7OmlTK%bnH z%2=s-)$RUyR-Er=Q1^5Ev>zhgjtOc)?x1)AB@iBLuN$%;z|)Ud2=`A0iC?7)Q6Ga$4AQ^u%02h$%c5BT>(2)>={W-q+h0|jF)(Tf8csfjtoZHFFbWqHS{6>oXZvJx5oALJASP0yiR32`D< z#$q5r8f2P|ckmpbL;ct%AYpfM%6)`>f4e4i%nxVinP`U>f;p-8@SK*LUc}E!<;_Fw zN6v)=&{aS-B~A#P#Z+<*7EweDv8BbRf@n2ogH6YTx!LU_yM2)Mh%xt?`+Q~C3uGPS zOR3l>FpPNcPwQ`B`!!)EjRfnJWU4{?WrPxN>Lf8=phmHXrZL6MCN~wh9rzS zIX=eWen!F~O5|=Y=KCw?v2l+uT#)JV`21UB zi9|jPA#{nq<;eIanI2U2Qs(If`c#W~v9vhFA2CwhVhU*y%UoqT*zX%0mQm5l)TM+n zQB#;h)?{}yQ$qTVDOk_lKie7j;xd^m{Y5|ST&9#$kLeGWKEOty=DlVXaVPnfp3Ccl6+iN+L^TD)nXPUbj+67B6)wh%M_6h1L)kbt?m z-zNUNRW+nW8I=*UW4t(Mi~=_oxfjAiX1y&>_92*pDs~k})me!+)&YAFtu$(Y#rfw= zt~Jbq-7jh)8!(eK^j6Lk@weYQ-2@+Wo2g%=Q%9t2h7+HBO*NwKp`4yV?C>8to+|@1 zeGf9ltOy!q_J*kxGxO2`dX9M)QK+OuG?a_k16mA{9?`SJw~4xIfrNqB5j|_f`rsAf z1|3DskZ?!33KheTH;K-AE;$Pk0+2v#U(kODA=YlSnvAETyLt(WhN&ZeT4=lwOD%II znuB}4%*LY*L7&{-;+x}OiIsRXb!>q7)16MU)WGRghLcJUf~z5H$GfjwE?29$x1$zo z@)%4z5c6C$B%H{SV(FM5Ic45%46mS9AhK=)+T5rfv}pjhpQaovxbXNld-X9=oBfK~ zDx1kDNaR-}dW5@(9{`h@M6S4G%h8@u6Zo`U=$J+{7xT2la@aGe!di$FrQ^+l5br!H}Z#S`QRg ze9_pMtP)%bk^;JXIGzc)?A0INUn($N9vxB{t_HDk-k<9&HQ6in8l738JiF`Zbb1p}AJT_;Jb+$@tIZlCBHRJMp zrOo1bWJen}H2fDH9Lz6D z9R{DvM$y#C^lof#^Wi(R`q;hUsBtkc83O2#xq8_5sU0#5I0Vf$FV%+$G^~mWRVvle zIc9MLJq~v=zN`Ku(P(UTyFIs6_L|5zEx^lSr13Vo^zOiL=Jj#hm3P_$NteT%CbKy( zD_1QtR~RTrSdQ8E3}i{82eLE2MD8ERLBqg2ShRtlNB4U?^yNJq6c>PoqR(WzpRFP_ z#=Ss7l9KS~arn!bh@b_}O%3I7dwbRA+gms=K>U}lrqORU*@&~H5S&P-YoX!-Y);av zM#sc>JJ`j(XFQLAV;J)#_;Us#$ld@;$Owh6#hH zi_vOuet#O+H3Wx6y%DL0x}h40sgD^sT}>QJpULxkJJ}uZY|1AkQ>oDKn;e0kOsDy0 ziB7L6)hG?nmB;7TWbuC*akWlLPNGt+hAI<8b9=ow4Rtod=KUJ;j~}efSQ*q}OlQ{;~#=;CQFg zYGQitLufi;Zk5Jb)9ZI>G$@fWyxnZ)lE>lJd|L${Ovy0KH`{Ko@qO*4Q1&iY>PmM^ zz#0e1MWINFC}b($lTbDpB;;f7tAvt4D>RVa>lU3~soi!oX><8GZ~qn~yhEOpm`=j-`KBb>IJ} zFKqvQx$1FCbrZi91*m|ap^Avsi=cuR8QGLGVoO&`enrx%UC^>2cZxM2{}vGy5J{<> z_r8YI@wDCfzR&i&b|fhOw*@>jW9E5sGdC~uF+b++{hYQpE8N{nav1+>9RGT-)rP$^ z+xeN+%XizBtxo%;=Qb$1X`I<5F_!kq(J)EZOT^B6UgW;r@p9oTQrNKhe%<|9v!ZXe z()hZMz|Ziu&HX9&(7mGnu~2@G%~~Cf*=sb+l<8*?iVt|MmDKmiyLA7$>r7N0^(Sei z6p8tWuVm{w4~4wOYnh^MJx{(rgXv@7x}{p66Hm;H*ZIjppZ@Yww2>(;j9NvC(Re>jHQ$uk~ZnFfetx z1+40&435v{Hy5iykG<}^casS5jhgd$Z_Kz3W+E1z>>#p_hKIhGb{Ds~JD}(L`lrjS z zr~Nno-e$YUR|dCz3lXW0Q#aS!6|6WM_4>$1nP~5CmI8xM7!mUt{{sJ@mT?~a|FA?o z=ap4?{%@461DL*_Exo__E?O98mnWy+&P6eNUY0e)3Z>a%k79ps`wz0h z9#3ud_+7?gq(2&JAQT^WlGwJCgsh^nKaI_rLiv)a8=Pf55`e5QfNSu)Mqtnv z9?ou3DBj|j8-8Q#a*co1Lfx5gV^OB?y1pOd z#i? zyYunm*_`o#rn&dWYMTe3)&Bd@d|$iG=Ifnb=k)EvKm$3cUvF+R8jk>|fc77f;bOgY zcfIO+Oa_Zh0gq`fX5;%y{opVB`LbLlM}u{E92`?K@RM-JzEhCj{vN0Oo7KwW>V}!w z**eU&0R#CIe@Tc1V0Ac=Q!2Z*3$Kb43`5a&)qCB;mdmwNoBVz9{rQ@?^6jj#(<7VB z+LJzZnwlynOv5BCrNE{0(CJJMxQj9%@$slqV=nbwdR|0VVVOV|tNT1To%4vK3VZbb zNy9Kcw9;a&Ci`Zvx#$Vpi{`+}M8^q?{9Im}la0FGl>Ih+&s}h;5<$6<&i!rt*G|}c zkAvOqQQYrMX89s%`?aG4?%Gc5LY3Tn8VH(v*bK$_>;_7)#Y<0*YwxQMNFtv$=zUG9 z&v9}5Xb5-NHICd#xeHF!L<-?e>O!!ysrCf4=XNy)$TLGT-}0`M!MfhbB$UR=MZo zg;=cR7M_OiDWfH`gvrSpm&-N!t(R7tHNt7FPD@xA*Xe!a&p1B7EMZNa|5L^4h^vG! zFpQgb{SXK0=`9XE3xCG(DKwfRs`iGh)g@GFNzzRo`h}4 zu-bx%U3dF%du#InA=tVj40u8ljaAYV2ja2h?r}Xn95qgav+>XmjMNNCUase!fkY8! zf5`J%KQ#kx2D>^!tJGbs*BZU6mY+8r?)NGdgfJT;yU0L)gD~J9+~YIzS%v&corxLF z)8GK#sy=_I?K%Ru>v{T~Gz9Z`?AwNcFt`(P!jT&Pn=K}XyI>aiP>yo95Q)I=l)>Uh zElD^I&urrNIc%w=&B9-AUQ2P@?|Cswnpfix$7Nu$KG-`c&xWp)2fiY=0T{3^HY$^(kk71HOkW>~tN%(uwz6h&p4WhO2Ggx?>}mv34co!0K|#q-=X zCec^)9wE)z?YA01*uEB8R=yd|H*SsfJ716Zwew_boUj+G#4qhN+T_Go7gbbMeSI!( z-0|Z^Om&6{6h0%gFQ_-^$>n{XX1MkJ6d%gW^E|ok+y3OEK;)m;X?(walli<|y_>`F zbDxMYGc~c>>CB$ttjYn)zq%9XEpjr`#-p?K@RRA`tboT0O73pY1viuW3SjXL+9u_{Mq|n24a|3w)4L!N~$B;0PHrR1Wlw zgf|32BEoItR=cFr1i@OJUkI-UPX|J+#W0}*JrDGP_&4s7PbmCjvSNZ_=e-fVv~;~UP0qTp-S0$Pa@tsU^^EO z*d+5OXy@>77hMsFFeiFmd<3Q9XSgr-X^`*bhO5O!_h|826Zy=C+u6mMve~pz`s`7S zw(Vv+$RZ}p>(zUQ3q%+95EcK_I2IB@GcnI}*py4_a}sN?gZi$@O=d`-@|7zHGQh+S^cw>BqLz>^u zZQ5GTTwRjcZfXR_<$}WKKD?H9FW=}j`p;9s2y=HSj_hx6CR8@Oi$c~hiT${Z@|`|n z^R@0*n~|j@zLV(XvU%TaH{i0S>qGJ@^!iLLCx5?2aXKA#g|H*ajTBp}M(DZ!7;u~j z^Hb7db0HFQ#vh-S<*n9xo#t33kM=z6CG6xVg{0U2u&@q1o>{u`N==Vx1=cyhh=EoNu`K2e_WIDBk-DI7J}FZ%Wq zp%za{V;6vy!pxw5_qz(e?on)a9o$E_S1It0$Aiaa9kWvS&Slr5TW@?CKE@MA`(e9i zv&i%Pmu{B#aVW`i2i<;+#djV9H8F7@?UF0T^R+}B-|t^Z**A0bQ*9cpivq^Yofmz9KlxyL`;Vet33=7iSd~r-tgf(|LWw@@ferT z*E?z~fd$Fj?dfPT6OWRr|0HAS5Z|Rmp)`$`n%G^OnbZb&Xc{U?3L4Hqm*w2;lH=ED zv}~T=%VvK)o;{ww#DKVwVshI_zD&}S)o8akomiGV;&V9k2Z0B1QyDQd)SypmUjqNX z>+2b|?~1iC5|h2aK>RQhA^^e{r>13G6~dm-#YYJs26k^QiHHnI(P4PLCp{(7=!FZ0 z8G=|Wiq}4Eo`2LevKcV|;MG}wa~%g8e^xL0J~RYl%w$QoGBf|=r^(?r-o{B&!Dl#m zc^>U)FxkFbVdWlfnP~pgey*bdjVBp04jT{^;NF3EuPmF!3S$ni;y4{9xS76)4>NmTIa|i*x_gU^E6DoXu`G-|PMCJj-X^ zHzyt#05`$iV>lRgsnz*(;VdS!-=DG3VLpo@hvV`{it&8yMX~Nc&MkgGip1mLKIc#M zlV{t@a;Q@`x^x;C8tvyW%i;g6QlCtt70rqjEtAV*c>n@awA%E#-KMA6cd9Lfh;|tb z!AwoU2*jj=btDAW1z6S#1iO>p@EpG^$?@ka4^IbYP_1bkU69LJiyqI|eg3_se{o;s z8hbwisZ^7_UCzCxmsX4q!XmaCosUQHJpQfXZH@9g{o@A)Zo`heUy8log9F3T@%(pV zK%ym|TW}ldUrckEf|XWl6@$hITmCX_GxU77C;V4EgXu^F{_Hz0`>X7F_HWh~o?85(&O_ZFxU+eUFplbzCm9f8AsMzJkMdolC{g z-)ypd@BKKW2k&$vT7SIUVDHuI{>tT!`Tj?&v{Fbeg;GtKI`O+h8E!if9fVpR7xL$w?m!nAWw4LEg!)R|^_-`tr?|`2$L$Mn zd2=;eHkp~{ds*en<>q-FWg^}iN1#=!>VwbuKOQa`{*2>nR;5l;>DmqZ9b$|mPiQe& ztKFNIo6c^2L{q7@7p{u+dQf*}&-v#X)t@n4g;oMGnOGlWkx-xVti}PTw71gxbQ5gT zK*HM+7U?bpI9{?6i#-g(!2bDqbjj^*ILdDG5~0QF zVyg)fU=w0_0X&b%&e}D`t{$5ff%i&D+Ps|;_Bf(3i4|?PdI5YdGS;_{Qii*u|2kQ zvt0|KXlitB$TXObObBv8-)# zIf?(W*#fkP#bWc}dueC!-b#18#0ZQFG;=}w4kQsbQo!&?w-42r_;4wL`Sf%~Dzte9 zA<<%0DTixbD>N!_44e@hP*_b$Li9<=%&fqdDOG(FNynxl*XMS- zATf!>=B$j437^*_{Q)LrBXjjkc^EUIUpdma?R8$mXgpGXfgwtqEg~9w4k=ys*mRul ze|LM|j-gM0XV#^JgUZT(i#jX=s;Z--GzG)lVs(0hnvW#+R8&w)net%A2LbN1}t zuEUVRs#;H$`tlr(;QRB|pa-nAtC_~BPaWZ(#P5_dg$3Au&lxRVhj$B*2%fLW^4V;* zF=jx9I^8v_wHSydD+2rolIi{CWaRiJof_5o8dqO_&}n(JEAk0ngEk33R-I}g)_$Kt z)%+h%aZwvI-4fL*F?cP?i}7sfDx)-ZlhUK)ds{eBuQVFHrweBJ8qE%?v&D+GZ7=If zws4m#Dl(%8=}y6j-ikulLr5%?01qp(H)m#MqDGy_02%$zT*=t1`c?)E;$jxGmZ-3y zVtseqpZE_i5`)TcL1|dxO~um`@x1FK)w0|s$WdmtNogk$h?jvYf3&6844nS7&j$Gu zn&MuJiU*NuYV%8fIYxQ5$Z6sQO}mbLHnhfNGH7aIl}oW;C4w-Cg+2vl%%b(9zjWQR0}H!6(xjiV@m(J68w){!0;=LwMFMM0?+GIB1&GCz-jntje0$7`sn#)Z zV8lZ8jD2NTAd~ev3WR)txO81qbl>)+r+1;y0YaLeAf~e@XCm877V|4vd?cfg7`fTQ zq=l-MtL?S{@Lc_Sg@qE5NHLKoWNc7IO*ANXLQFE^+1tONOvuA7tFb#8uZ!v#Xig0) z5G)g@7;0456_>^o<0fGQIkM8|2t12i9fuWP$Drl=e`8@;%sQqR-_cuGQ@O|k~#_{|w<_ybnJoXHEk7jdYvl1|{E5IuNrjyIF zpYN~sN51^Pi#*@s;zfruUB^Mvm+eN^>qu419R;-Z#E|s#Bt5@pjV{@juK%T~_|u8k zev7(c{1oKtdW*Fj$6dtq`AS0sZcWr*2c%L|t&usRJamBQgR=ewtJP8-+x2_vGLBli~4Ckm9(^k{7!43}uH8c0y>D9+%* z6qhwA!Kd`igCa}Zmq5bDTQMZ@r*h(XJzaF|4G6=<7*$lIJ%)PWk!Sc@z)X?*m!6+o zD-~rN<~D&bcieZRph49TjEgS0rNW7BcCTok1VSzXNuVUBvd87m6IREQ%z-%33gb<= ziS#x;I<=W8=qUg+GxI@tx1ONl`fSuy=xq~6IW)klQ#sXxYF^aFh@980NVmfT1ZBSV zbwQPelhOSv$5}kCuK>*%b~$SP6?spf=Lu|J`$u#oQ~UEL!ac(ELX~B@+|+rnE+dHY z9;6nc=o%u(h(;P@Q6dC9HXJq+Cbj_`{%woh+F6QW8UNkCpA<%j4T@n!#m3A!avmg< zj>ZS%7xYB%%DW5@3YXBHXdQ~D)tYsx+|wd}-CNTFotz6Ofyu>C2{4qffoSwGMMlN0 znA;PnPr}xDMnpYF&>D3-na!lp4wbySoA`YdJ}&YktS@Jo5rG^n(W=4_j6EdeANaU> zkVoZ>v=tU*pD60y--mRJh#h7h-S$=1jfKR7#V~b~!ztt=z}=ys&+5`JMJygIhC$ao zY*)K!p7Z`_q7lTKqGCrv6~5$tn3}*svgXTibv7dzNY+*TC%LTHDOAjpm_yyv^q8B_ zp!6qou#+Z_(!eFTA08pZWAX{9ohto0ei8QGku>COwrG=tq?k3D6c=Y)<@|H9tK@2X zlt95c5_`c!CpMMmS%dg&$Hg7s@%Nt&kG*m7Sxi2hfrt9N6sW&M7xhN?vRI(BiKG`M zC8zvb9t)xv`t$XIysYh1B(UFi3kOxN>F9h$LlNh8?OzG4m#F*m7l*Xp`}p54<5hJv z8m%d2LQNvjlXtZ~JlCq@^B9Pv!p@ld9wq%n?Ztvtr3;su_pU(~7Txo|REavZT?<*I zm{DHUSYJdiEvxu7oPexUn zp_!D*Pi!1W-U0u(is@eXP*mqeCZ#KqahK!|l3!AjUtH7e52 zN`%M3MwqB2{)hyDz%7wWF>(r0PRDT%F7j0wDTX963nC0-t^`rg>MbsMyG`^^=^na1 zaSjc5`0L`~)0HT1;%m!P_mc#+w&`KtB32%pSm824Th=|^l;aCax%YIf+kSP}M(tq+ zi4DWh3~T_Yc-)J)(T~MCP3TB5A73n;^e6+<_7&8JJL)844INdiQ|Y`4I=Tvf(vbNm z0tA-(rm(*c;VmrUd~v#NqX*aeZLHD>OmxqzN~jP~8s-iuZB)B#X~T=8|1CPMPkgIlA18x$hJrjDz^Oa{WTdnUb= z}{qwY4GF?+rN89EfL&3_4LkI){oQ(YDTVwzgkKBQK z;b4+$P81vsFFY0x$|cxRYU}*O-?K2nz=hk!@?!%U9ph4`qf#_mI3+EUt=7??6H8@9 zrgH?3{)?7=B#?+GS`E9@^qXPZaz<(ik-hkIq5O4x(E;z@r#VsBIe2LWh*HpfDuJb8JRY!Q)jPEJLN7CQrX?Yo=D7*9ZRWg zgBN*Y1>Di9L!+!}VP4#50f~%PS{%qL}rHNUnH_^3Oh z3eb_PkcLF5Q;PF&F$HDJry`kdIxCk4YbaJ0O}MySkQ<#ceknJaKx$nAV2J9Em-){@ zBPD_09fx4-n(xo4mNE-~7T4y^5W1%(7jyP2n`aOy>(N>(oAM{h$wfqoXJ`+S zlP%UAxt+a;s!Dvo{hd(H_fqO@d!vLl(TPGVcT*!(%)H#IU26_&5jkfWOSeR`U^Qfq z$b16lpO`ZHV_-r|Hr$U4P`v*o+(E&YS}!ZDC|^&P4R-`JCrmE1+{{9Q^g@nYJ5?x+*QKsCK56p9x9~F=OM)z%ABk+x;`=~X?uV5A8K#tY zHe3!|fxJ|>WFAVqTtsp)Ir+G`w6_yo;E7`@y`(e)Q_)!#3JxC=Eeac$4TT;7|BteT z*fKeHI^6o93h3uiBvM0l!>3w$T^3(y{wm`U{U0jB1)Kax-+1yBjR8M01^3ceD{YQ` z!V)-z%T)f|EZIRt7;n95b!9oqU;=yc=ul3JhLOS4C9X5xg1GpJwbAJLbBab~Qb%Ol z>nYRp6b>`P7EFUZ!olbFWLSOFPmABxPX$jf4n{KDznBLzme@Xw8p-XM8*a>q35~f5 zmnD;2-%Qa2G(jcGeFp)LK&?D6!?>s5wLy)oae(u?BO(~T?8!T{Q5QTYM4(laKnt;5 zjMLf|&%fDmi%p~HB++dLXg59Wl{)&>=6)yPeq8Og8qL!bR6E_Ky)SONShQ;;E=dui zRxI~mZnir8hSlLFhtVh3s$lR9ri(G+qHE1o2foA`-f=q4Oh)eJZiCVFCDM&nfN1Cj zyWMJwXEwfZl7dJ7W{2&1vmo0jQ8rqs3FU$>61yrVQ*7#5cNJHy zI5cmC+jKF3Uhg~E%okM!)(n$hh=FI~n|BT2FXyLA$2WDQGKO^APPYkqzHw(u$!kgM zx_++-e7-%Ie7;XHnfh{>2kOopXvIt>{+BH)d7E|DD=6eWcNfmV?+;!VbKz~aYj+dE z2<531Hmwi|Snz5OODp=EJ@VJ)czy=M1FxfxSFBcx6f8iVtG1fYuW(>M)xgmbag0n! z?!7ZZtzKf`y6$6tjXamZ>A;iI^|xP( zS?&sap;%nKcPD+H!6I(kmfzj@U{{X~1h#1);s#}uNS@CZI0U|8umNTmz1?Q}^Ey6r z0B?H`(`9I?Y%X%$akV1G=T#D{X)BzSxY)6&>dGomsMBk{|Fw;;J2GDq$KRoEy1mVqeI6xZ zU>{jWv>+#`Rj4*1*Vn4mZ0UUTim}Q3=9k9*tJ5y>FqUCu*oQ6|H86dA>VHD+Oi(;txAgDmKXoE|EiHb{#dUe&?@& z;r(sTV+4PIp8Y<|J~LPRVt4PHrV-zv^T>O@fX!2gdTJ9n-XoF=ECPrh zB>*{E_&=hFG~>xIVxcDEV>r6bV_t}mEh{;4qSi|9 z^ODlr6eI;Kwao)U<%C~c=Jv<}T)Z|6}ocgUqx?~UVF`FlPX*7An z!tcZ%f8Dq<^LcCm~*j1|b?XbzJ=M@V4T@rI!tn?GTKh9P%*h@^kJn@NrI_lJ_u zEss)gC9LFo`5%Jbd!8dyleN>$RM#9`7t8K zJOvw$n=%gH%6*uNkMb=~jCgE&ChX=mDA{t$D1sr4yEi;~(K@Z|)RfWJ7sP*C$>Klp zJFrqiodzL7K1bFhp2wVFdp8v!1NUci%>lliD>R?InTq0Gfpc=kVuW zoJm}ens)^s+9hys1xg{XJ9v^T|Cr+n$mk_qOpN+92n6;nf5!8f(aD%Iwbpqhp&juGwL|kc`!z^^83Rht+hC` zoAp33c>dfb68c`fyo(j`Bb@O%t}!8&#j`(mA?rgmhzk0m0kW~DQBl^Ix5co_?NJcijafKh*)60_ll%!?bMEWu|O zzY-b@V{V#WXsD2Bv?vS52sovnp^BC^`URzLeVL}Fq!d4r$_}cU*Iv>=ttsFu7d5Y95s(hcWrjrI z7GT2LTb>FcD6-nVY<{?`rh-fEE_;guozJtf*U6a%30esdVI}pY{S3zT`WI0g<_M~l zv`4Fr?{TwJmhb2F3}QQ%EU;0krxYDf%Tz(^8W!{$M!a)cywazva4sp)Kl83Huo_z( z1WLto?+AB(ua&{IVkPWZ9Ig?UJ{vSzFC?AD#?)(AMmq=3f)d=}16ID*?MTB8iUL~B zJxK_Rp`CmL`&f-*uYQY**OmZ%JPMq73z=Kv0M=x5lx~uiA8^kwnlDJOJuVeISBQ*8 z?piqYy~-@kXJFh+=-Z^y$y9b2ERafW=jGuUj!Bz19t)aew@>tIOo-4UVmcEFicc{c zTb&ZIV8G#QFe0jO22)mzZG&i#lkPo;scEUmyhSYvPJKmYh+9k5XB`P%LR}XbLD6O0 ziccwAY*TJPX`4Jj55T!i_&OX;<$8J$gh6K~on%K^D|I-RFCM9t*sJm5wrHfB^>Wot zzzydGNu1;=yB94kz)CKmy|$?RqbMuG&e|k`pi`EcnkRUPwx5dLQFv4fF(FHMB<<8~ zes8IVnW-LJw}?*^Pas$-;!LeSwWdo+zi?6MRS7vqD_Jy)S8mxZuyNQ*BSPD|vfwBI z8?3Oh^2G#gkamQ?LVUio*eb)L;#|?JcaNCWOb47$;V|-dj)ym@I;@F=-J9uvjfgXQ zUQRlZ%`_JBYP$-7LM7GS_^5ArJN{15lTxB&vKdaGNN#ynU=0e4MdC+?RDyT&YnmO?xG4 z&u*!c%+A7pi!68Pj%pIkc*6L9ysifg``GdO0k6!| zkEK}GZt12dqUGeC__}f*1nIhexJ$ZAtIIBu(QcbeO)WNkA;Lbj$ZX$Lg^IZnZ-0q+ zNxY=u&!m#zlqC#e0aeOhS|OxOTzl(CHxb<3WCEm`ODRFJD#Ii+5=Y{fM#=rJncjer_UL?f?IZL+HE-uAS`35E16%$k~sh@#x`?o(Px&5p2`ry zfkJ=;3t$r|2qHhz3tp{^ulJIGql^$3ATmG|{{#j0NnL^IRDrk@4tQUH;6HQ3auMX` z*@MMRHYiLmIM80qK$j&u>mO8dsB!V)pGKRq0>In)fX}azgQAIVdXpl=lUICgqW=$}s+0veLS984!Y)gK9PsG)>B zH9cLE#};_ls}_m-pq0m!)}{+@9Yn5%)7fiuM?q%44i+Q`3@RMCO7HVT{#6;v7*CS8$gk)lIY#^WU%5Wb}W?ly>l4FFr!Mvc6H`vsaYT~=wJ6kdMh3@p9XLkkF44NjK-K# zD^j=}iG@1m%J**xX#yFk%GT4E>k|)IDob8;01}rhH{+8bZLisCP&WgSJR~yR;YClk zyKPnFN084pXB)L&x8f}DTIfHCMzFw=plL|_(2^lDI%!SffrV~r#}i~0FqUkRLLOA zYP((QizcGITPap*H;VDBDjVw>GZj=?f=eT)>q;s6&+A*dbs z11F))f9UTit=qMo0Dt_NU}yxVUpuC3Ddr@xL#NLY!_h$yA8tvE7FB>{<3u>iLwX!% zDJ3Y=QQZ)cW<2lA`XXSFk6to`o@KP*6Hh)sz~@Kyb3cIsHb{zfD^yT@I8-o)A$qY8 zqg{eM+k^cZx%@aiuWkoQo55rsvBUz@t7WNL7}?91Jcu>1W@Mq1YN*ergJqm$M1UXc zgW}Q$9^^WSFXxVKHR*U~|0K~&7c@j57-$!#T~OSdnJ^}1A{mEPOZw0-9=+PSZc1;Z z2zDNU>~BZe1awDWlY-?wfm@gtP?H_9o2|?YFI@`cgaVQ zi2{W!mD~X|__HIb7+?pu6sl@lxz-4z8WrlI+faX8_u)YKBm3;mBzjaR9Re`poG^_8@P z86bs3SoA?-kUz2n_oE&YApJW`NEh!#yH!N2#ac*%N~VvF*qspX)^wtt)(J&q&lnb@ zaZL)~`$aL2K8-PJGR7jjj#|b7>PTtXc2n$11bDrrDsTJq@bhSsfB*`Lm!zCnazTMEJHS&VDLpzT@w4@Tokf#fB6@O zXmsnVa~65Fcyb5KxK!FxhV%bL1M9>F5!{zmtEhj0-baER49>AkJ2va?KX-2#A3%83 zTxX>GpO+`tLSz`qwPUrKF#a!E-~|n9U+9UF1^Lfyy9oCox&xNVDdT}2G2sacb5HJ_ ziVd6v==SE}L4-F?)rv}a{&UFxKVEjBx5;)HZ3W}e2bH21{xvH7yQAoh`6Tdp~ zk!*->iAf(UfJIzyZcVd4ChKAu{==unA7GMWy+^%Z?9R7f-^xV6=w?dl5rK?sDoHsi8b8kdMe6=jOR>wCc7NLf(b$c6;jpO38gec zLvx*=&N)j=^Z%v}41^n<`I4S=Og&R1`Fb?Zsr?{dd%Jt+KA3hn(>#yU{OLl1}zGD8NOmHdBM%PBi$xWicdSY_sv!UEw5!Bgw}LBpChDZ3Ai#SmN(-TI;2;cX0a?e-BfDl<~=y#KhK(_ zj+bLKlPG*csai^H?y<6bz^5b+MSv^!usU5Tn_3U^ai+jfx=~ltgpHp5I)XUPV!l|? zvPLu-oi+^qx7Y(-Jb9+XTGU{x@VZegGWs^ z`Q`0MiJ}^xOQRYONzwFN&CcQvgjOxcZO&!Cj`P`(hb7<=GhVhdrefO@EwGAJsoF}y zJt)Fm#7wJ&>t3vwDNFf5#YS_#Tzv{iH>KYexxm_)Yh?%kyh0yZq>@68yA`^}0`E6Ss1r?zE8Ji%GB#HKBH#XHIw& zR%Bg*1)xX_L%M+m>Y>GZ&TFP!ijCAl6ximj1DxcCz*CtV>Dreab>^Ozn?W;R*;@Ne zqtP2wD*WA#!6cB{2Tx5?VqwuC25ZVIHOUba0@j?;=l1khEgm2QcL^F?LI@fl5S#>e3l?00yGwxFg*@ML-}l^m?*H58%uIK6 zO?7oowN=-2POx~6MIF*l=>0R(ETI4`3 z>2EVGsnluP0D@UNb@Fe+O|14|kd{AEeU}5%kWIvA{*^I5UxZSm5LwmC<5+|ShLhEm zh407wq@0T|?+jC!W#FDzZ0gw{WnvUj|wHW0L zimxM!#V0sCyMCRnFSd&IQ;hQoZFK4W1-aHDC9RXNqKLV3zB$l64pHhQy9Yxn@di6E zW38Fi(vTW^il$=e*qPr7jBT+Nrzo7LX>m8p8$&uF`blnoBPE$8Bo!MY(%MjO)GR1% zE3Z&Z<26C4ATgpw1iYQq$~R^D5$mR@R&y~yurlB5qwHqJ2C_tUKYpR1yMrizabqDd za{h^CFe(x$Kj|m8WFTb@!3s)^+%Lkoo!Sy|F!-0}({DjWBQ$8}BHod5@=ZFQuuDKl|7HYlQ)lQixNF&j5l4_OQi5%~EYX{w zX3fcPEH`;2?wx?&lUHmTvz-d&&ZWrVPzlswA@A|YMklacas`Yy+AuVv$wlZL&X-IU z+|(vT?Ge;_A-L_E<_p2=o83x~&<9V7fqq=xF-r1sa}5$)Dbl}8His}(yirUQDGYsW zr*H3JVEA^W&e85BZ!KKMv*Ru0aPWCjh2dEGiA1iQ2!5<(y~Te!#yj|&aWJ4T>58hP6^nw<9K8bLq31F~-ZGGJe!Wh$$P z4IA5Ti_KcT!qWt$;UX(&)ct`T-d&_tcX+*bC97T0m4ALBe81`Wp zdRxEU{8l`*GLbSTGjb2YPr-O5Qk2{EFav}jd*Kb40gUsg_V*d^?H0~`WyW>l3{7A^ z6t_ag^Y)xOU6<9SWn5l56b7Pz+G{c1;ATLOM9jO((=u7sL5X65%b7VqtuKr5r3C__ zedS!9Cc?zz{q$>j#jq!doZFQ7TYD{x>yG+JAr)|Q!4}tr8~G??OW<*F_$>^8$bh=d zi679j#v1^a6G^J;l#h&l?;Pl)o_tCtf2l1T6o=j@cSFLA=0%~H%M?snc zW}`!*+Fv3<^`9u7Idua24?d&Y+lR`u?J>NHR^S1CD=>R`MaN8SszsB(5c{&WIpMZ9 zq;7YaJ;8pz4n`uabcXxfKD~GCh{+g9{RohIl8vY(vS!Xr7n#WYY|g)tyjKYKn7T~~ zKP`ULCA=^@neGlV&UPvVSlpXrfZ`M}Di9^q>EAuV2GC}fZAz`kTCg;0Nszs17KvLWZ$xh09SNY#M-e}5Tn#B zAvR6a?o%mgecCI=DXRAC{5u>Xnrq^ILPV3HF(JIG-dsASyiEy%R77dQ4GlL>BUw2O zm87uVFNlY-<|fa)o44%bLvizIduD=v<5T0ZYLyMS<=;}PG|Nm)sr01Oa|K{N`y>gk zh9&Wp&D-s|l1V8eF~K-qfDbD=E4=G*fY2qEeKniO6748L*5TrWA5|8cq_sO#fixhw zWe1xiYv-9GnSznKQ&h=p|D91OVfWV4*nlL&sEUSzZUg1_)Y2W%z!Db!iJ~8hNU_qG z&sid^tXdcw{G{2%5_&Jn=TP(+#7BGprM-HWhz9iB)C$F<;rJ z_~2XhMgafeT;)f8rynJzZT==_$7u-to`^5d%n77m@%7Bb=KBlSrQCfn1D9|9s^G2|pg@1YL^z<4`An zWyZ>FD0R*|=l{fzeTA&;K+E5II4#|D5dDF4y-hXp^1tg=ZU=0=$k#&1nnA_F1-Ax5qo>FzK8oRhb>Db%DeY|aWn89Z8Zdam#28TRVvjvsoH?J zFf3vH-5%@PR#!`mr}NGpXK?R<{2@=y57EFFoG!`=fBcI6!TlUBf@CE>c;&0}%`i0DXu zCeaujlE+x)AfV_CHVi1T%N3QdUiexJDt&lAXG+9o!e`U_$r#IamqstzMH7ErzT0L- z#;R?*QGCJmlsAvjVCOTUZj{JJ4H1)_OT* zNCZj?IE!Bs_S~Y~!Q(lTpT{uPDY@ZD=S0~tejmYi1(v-#Id zJufsr+chL7&*j<MZPjk~|e#}3b^rBFn6E>m-0C!eHlu*2co6*#Pzn|SKucDaV-xNj7W z^nmm^+76fVrWc&uM%P#*C#4jsECE+Ma!N1(QI1lTA-F<{3I6qsnM(3bEG6IN499Ju zfAKJMP|5{0L0gck+RfBgHq1D-28Jb5V=B@EyCzw5BZcPOPb=bl8Cy5rSJLet!$=aFBge_FJ@RI6Cb=*KaJ|fBwOb%nQrhsl<0TW>&W78v(V=Q3Lq=a;ss zWu<=0`(q9AW)PN)kd`tBvTRqpI{A(BdXipw(VWD}4dwfjg}z<24I3PJz_X1ekjyvT zDNB;`pe7kc_s?!th-yW?ECt(ojJUQ7d8H6*iP}GYhmK)_tW|mvvIqr&g8Ia}>P7CQ zDTWE7jkTp8URM}@^q>qFlkPPDxuz4Y z@eAn+Db-;;>6A>Xp_1;n=ei!R?O6GI)cP9r+v>?Uvm3t1XRT;K6Kv#V6Y(#si?P#0 zc(EZtSrSy5qFo5kP!NI%y@<-5s^tm8DOB2cc&ixyrv;L+6_NtEd3An)nz_g((7y-C zrGaahDqa#Xx-(1ih{b&zrSKPa(GN=`n*0Ii&yxLZqFA5Zh8}3cg=#6^;oj74Z)Mwi za@3Sefvx(KETKxHx}(6~a;ED8_nYE5C=B4u(~$X?+#(ifMnkr$nK&*S`; z6I}k>9bHjNO*uGg1V9yDen0)=U!kz8lIN@d|Dj=b+Fro^;O+J4 zCJ_NaSA+0|N(NtxRe^FH|7eLf4Hqglp0{+_P)L13E+mB(YZpJAjtQBwcc}N?NRf{` z17flJYaNMZ1qq!HA0L0Fp`lJS1$^u?X;*2`PtV+l3U=U7OIf&rs|AV?;k{A?VzOF# zb#9s>tzntOt$L0)bLq-=T{Rj>Cc6XsKORcuw~ZE*K1U5r2l#*q)L*r>etT6tiQq0`@j7Mc*13?oNq+4Vi_|DkP zd|^#^ZNGQ25iFt~QgImkL$>vFgGTk5P+VA$r&m(7IUubZM zKKbMTVq|BpNO+f#l@Wv-6yG}$zY*%bn-I0jTkJHjqgAGaJ7@BF@^GNHZ>U`(!+V>O zer0ZJ%I{T+Wq4w^ex~Bti&u;I#v*GeGhF@SOnUylYE8X9n)#aFbGpg}Hucka`xNI5 zr+)?xN|ZNC_B62%mdq>t5%{dFte!H^uKm7#n>jH?`ouBwCDkMf@l)U=uiEu^SYhzT zv{u}=N(tGJm4srGQG$>nbp9FwG|S%snw7Wj!4Gs>S)^U z=0z%{B#Hd|Mzx8(^8)kG_cc4etouvoJ>q+0-o$#}w#rWoIgLlV(Q}^24;~BpGq6;O z6|K^#+oUE)TYY;6BzcMVHh2igh{N+9I*1v8q%E9IXisE*cJ-PL5J}DKl&)b5W~6D4 zd=&>9jqZ7Nt&GY#&T}fc;!&m&T-((*L9q>6Eh`^VmUcJ)o1aw`cTBDRvyf&gT}17l z(Xu*Cgb@U9@hGbIkNdG=@Px+q6vcs818ab@IKQ-ed(2$pIROQt(?toIn`YgYVX zJLC)YFX35GTw^`q2eR#YhC#4I6?~KV$N`iI!~_jgw9F1s=L)&5z?bi+Uz0|8`-yG5 zYTt+*L6hlhor_3Rurj{ku=<{AslOwA^8(NT*byAJ?k%$EqyjdO?pRW@YDrdiVY8di za0xK+D1Tds&FByK8Yjq&QuyTQDPhr*v;IulOcu$a=;7;@G<(Jk^UrZv05u-7fF8CP zmd@EJyJi4i=kj}8CRhBB2(d7d6nYD%tiN_I}$2n|BU7wq2qm^ zDcVLEx0G1|V$bmOV~N?d@#GA#J18cm4pN6HA>_T%U@d&@4_{Vde){ahgwz^mCgu$g zioC_O{OaPjc5`SvCgf$)5wnmbar$W)E69pe1O;SYLO_y{i&Irh9K3sk9hMNVqDJk) z*(zI^ZzUob6^u`S7VAf1rlHRfk0tS%Cqv1LE-s3yk~qg~e$^-&_Y=uv_!*+z97R{f zYDl7VxBAL!Mq{orm0K~@;D&6wb0&w@<0OKHzA)Am=HF{(0eVe3%Ih?A5yBPl4W}`l zC8Go`99qLqc<)n01E@tuan)OeYU_RB4rfJ2h`i=z#vytZ#uhSK04=8NFG4J94=all zz`~Z=#mNuGQN@)N?{D{TW1$5ZU?KuwnPXB?@-@ErCkMar9GMRZ4#K?;gpIz zzpk#PVt|o$QjsMmuvCA<1@B=6PrmQrk zmS*um;#+fUay}`uD*ZBV99LWMJKN3ak_Oj4!t^S}#^>P%otD=Cc}DW5q+o4WCk9%( zE2EAjM9W5qQpLDE5a^%;kTD>V)R{#kkTvZ$&Q8|SdWLdq{v@QJTez;#HLw;<4o&rY zKL;VR18-FIOtsn3enpoAT}QN6wL4DmCIMv7gt262E;}R)omOi zocFGR21;8IMz^=78(q_v`Y5xEr`(pj@BN!X!Qet$x?`NH2v5yl(B3 znaFRDw@qb9Y#1MVjS{Rf1qu8Pvn|r%jZx*BXWvL)nY!bUS1m5v1+3K0v*ZQj{YbwG z2}HFTDC&S|Bt9RsI2z-NBSx{)En$e~a4|jYp^8^T#C5V}J~bP!kVY{(lY^$=cLcj) zG_Gs>vfqH8m!K^xkHN#mE&Hyn$Z5!WN1Z83I>Aifj(@wO@4)OsN0JzbK#J>gry=`r ziZVf=S46h-6u&ytHsuE+T;2pyMEHu8Y|mZkGB-9Ed~FXoFj9^xudDPbNjdT#Lmg)| z_!v-(ab{5J5kTesps1j@pocu#`22qwOFkis)~75xp?C@%gy{QuD5?7nmMkDTUNBiI z0k#Qd5F*)R>e&X4OObYd3d!t42@;emjV9f;vSiM7&Z~+czLvTK8MW<@F~2z=UuFlz zG|kG|amhmcXa6+e6v>ZezA=rf8S|Q09ooLE(*G|_nwMh5$PiwKu_9pU$@SxkVQdwz1kA_&q z)hOhhSOP%hnl|P$#~LuJJGG%G$P7EXl&SBAh?G+8PP6-$mq@cn)j)k(O$FGU<~ z@OpN>tGA>$ow`fW3!Mzy>K}P~VT{!t7!gLtZmj0sQg$Ql81BYpvHB_N43uw|sbF$g z;Y-iYBl3Wp&mW~jO%O*7-UGizM<|f0#whVb166PVFu8JTWW{vB?tk% z2Pc{F0Yjb!Q>RV=jpjcpj8k}5nj8ZH&dM=`lV^i75~a@ZNuunHX#6aWM2>7FZ z+uFonufYD+;wE`QPO1FlN9h1SrK3$|x$QPWp>?=Vu9y9SN2vwd0Et#M`4wusp-*?syGtIUW#Q@W?c_GK4*nPT1WWf|w*edMVN!n4J%VVX=-Jx;**6C=t zC8wq*(Z*39!+6(!r7snGs54MqE&WC2l2w|iEJg*y{fVIo&Hv(%gnd6yRk$^ zvZ+dg#+`Lf@$9Sg!vt~cN>CZvDSFKg^qYluuA>d9`psp*(qe=CwEfp;m_wZE%`^3C zMy6TGmRKLe9B9@q7KQ)R*uwiHJspKy*}#1{?{@L=56=^@pKd=l^Vv3x=6 zpoc1wO-rML=A5^Fo!;_ER-ybFvl*n3N{Ng}Za3s1Pe`2ajYW_9^mD2t z9XtuDYFxal1kr;Qyf$YJ4=JYremt2`2NpGSb~Zc{9IU_E7OHjh1!e9BV-#JxU#xgP zpD#yp9k2@xb`pF#Ae5*2j(k8;8#f}j*@RMNVfE$G5Vm*DOuZ5ZE__!0V4|pHi-?t0 zzoHzcXUCW)WQWu;rDJ-~M*ljQV%OWh4Y>jM%t8jJ@ma$@Hz{`a0|g82=Yj@xmL&@a z3(0LadTh4Obnu8Z`#f(M&9$D9!@}j(u00=7?oa6a=V8q0%w#A1)VuXhn1NQ{Ik%r| zAa|$u>&t;q<*Sn>?6%CN*FR#@rsv?thzpPGtz!x()#(+y@cK1~hfL`Lv#y*j#Wc;` zbDGk(=x+V$vrj6lHP7DGZOm_M^ud0>a`BP&bhFAxDlIqt`|CNm>Y>fR7EP)~pooiJ_=7F%n3sy8cK`SqvPFYh+zfJ-YB z!gaPkyf<`4+~e-!teZTxetC*KojaJybNq@$f%8S2_)q#Yp0NLD)iv9!ivH)5+L6|u z-D)509Z?|;i!WD!hukodTA3>UA8A!f4v~~Uoy?Zl7q}cuxlDyT;rg8id&Lsqv*)ZX zO4QU)h%DQMFUka^YSn%WWWr}4tx}E-*RGH0Y^fDUG;Hg#WvDB) zsQpop%M99hkPJAx9M7K1%yaEA2;ofh;?xkkTOh4+F?C1CHLgvOX;9mDPdVGp_{?AM%r-UKn#mC zMVcUEv%w1)b0DvUkcOFe6>++#Q^jNSLY7lt46Pu-^oL`bQx^?Sib(_R3eU(zGtu(IhPKxi^@miAXZn?`As z%!^?EUX>6oGa4(?5v%zNpp2r3Q6_ibYiq!x$Mac^Xis!@qLS0%^hV0}*Pj;0Rx6KR z>)rhmv}QdXGBm}?Y3yV}B{B)VCiR?|9!9`tM7(2hhT!S`M2B2XN${X%yXe%jty$JVfj@W}WRL45hCcoZ8w1;6Efm@7^CyL0Pbn=B6v%Pe)aX zUxs3Baf1(7a@sg6cJth^_ZKa?n;Q&uKrI#%B=P>Gw|m}7I{sJZ{mAaS6ZR9ME>fCe z6o5mcAFAERYMOgC1YZeHYQJ=rug zgeMI$rYhI0G=zOeJ|}Ljs^XB*2vIU~!=W0{lYJJItv{>2YdtFZe=Z8s1xYUY{Cwjv zq+hH2Ik*J9|J^_1Nj6>X``g{xPn*GQr9I>{wXAozd`w_@h>dmiI5Rw%WFY~-@-llE zooE`3aJ{oVc3r>Ld-JU-eW=(~na(#759zN^8`b~m7svw9in@0M3RQatBGA zNS7@ZDk!!AT7_sV<9CYZee-w(H;e0f#QU5X$aJV*RON{Yxv{la1iALRQT4I<@yf1S znW|5!%Y0J0v<8Kkk93qaRM}@ByaRw2^RBOrO7mX*NQxJU^FUJRZpD%FS{>26-(|H& zx*_ke+t4Hafb%4U|3ABeQkbps2HXG*UZ|v0xuHknx#UGCfBV_l2h*Hcty-qy2a`1R zhoNlhd!F^jrULW9j!IcO<1HzCAiVuBA%cBU(8Pk*)7<}0UOW1;JZl3+m|eel^5kg= zOhyl;0rsSFc6G9{b%0R8yqzIbFV)mIpP*Sopb%GEi~os2qk3?h0(@vZy!--uf_!`& zaK~fGW68n61%B|uodEpe!XG>jez<#t!5Mnw^e^b){($o6{)zwJ z;RiyG6yQ((zc?H`+#K9o+}zw8f}D>8|AoUtTn{|_llOn&aC}ZQ`0XL$A%%b0w1T?_ zqHy?smk%f8A)62I|H>8zj{w&{W%Llu!GU&|Hl6z9$+IL|Ej^f?@+%<=Cr5U$rIR^? z-OJJW$%7LS_i_eX*h64c<`8RJs3^l>QyT-7t)(b~F0V3&va=M###Y|P6{6{*qGjP@ zZy{*OATEX`;w1zp;0S?%sk|H=pl(85q7098h2Z`NFo=Qb5d~&1%Anu~wuT4^$#~gX zX*oGLP;qjwb8s-giMU!?328{n{zC#D5@oP~!JLIaAWu(Ec2910Cs%6_r=Xx9h=U8n z#l;54U~}_^!oXf^P&XPl1vIKh8qyFq3s+lbn5`3(>VYQM+{qm#%D@1Zz`xwVY_0w` zZK&HnMF1}h&_fJ}lbr(ur$$93@(>h~w1v5;L0mP#j?NB{2k_riG$AgJfxjUKTQLC< z&_il)-y_ar;2$KBN5cOnrN0S07RW0I)EZ_Z#>vUc^AGlag8yaa5vXfx2}k7-fT#G# z;eP=CMbwuvmvYr~H-BW?7Ha+fV4Ul}82>jv{|)|&^WXXTzohXO?LWQ%{JTDu1pLE3 jc>fg?c;H`AgFxZ(WNQdEED%LjZ)A%-M7-dm-`L8$kE)0UKtVE)OU2pmZh9L{etnKuz zul#R@Qc4twH_)Ox6q8uP58haBFi-}gDWfP*6px~vf6idau$4lVL=kX8wSEj}+vxom zAhgjdsslwIigqAEal{NIm_z5T!~Cfb55U`->4L5;m!wFB%9RM`1GDWQQ^baP$s24i z(}a8U0CrKcE4rO6qMfe2w5hc8zsnBinY41H@E*R|4ZS6I=>2p;6Xdg^)Hds=vsh{- z=C99BOhs_cp{An6Gx7hn83oVi`%UEs@IzW*lZ?WH3&t7Ya|-kc<5rlom>r693pFpT zPQlbLZiFi!oC%Hv8{Y|syGt(Yx&{v2veHh&?~Hvi(RqnzQ#xDzqo#GBYJ9@L!sR9w za%=x{b(B;j!%5~Rn>;>#1QYvjY1f&d^o{5iYRo$WXsg4xSEJAN6n}AKDaGfXm!jph zy*dUBlIh5+@O2rnT}`=K+C&ox%C?vL56iB3gIlENjp@*DkoMK`BWyn)N%YS*nv#lx zAjJk1aynSp)mQ$2iI};iU3FuJP*q3x+OiIIvEA>8cs=lC2ScWX@XnDZLRZmt8v}gq z!d@S~A&_99Xo*70Af&?A1+{cmw_?7kPKCmffrFSS;8=+!YGk5pq+zO&a|xc2A9M=7jk!=LnV0MXRGw}9pGNSdMFE8_{ zBo>`}hA74)HKmBt4unP(j!hN0Eh)1GhdJmO17VyEOKc|ZWjI?~Rc%3ws`4nkEc?%GTP1rs`>79j=1SdXV1dQx_%0V|i>}zIhx|Dhueiq* zY~85eLyD$Dwe?GY3c{M20xeVzXu*mBs`8^en~&%+-!=CWV7>ndGc-4#?xA|VvbA#h zDE3*Cf?2hT9NIG3Eml60hS8{YJHv2zNA})ysAf%n-49whKxV$WEJgm;9kg-wQ^lAF z!dFLrbqoK6s|skodnBmA(|coQ9n>N7DKVTV*P@hE0OOC?vK4OnH$|EHX}<;n z7Aei#FIJ>&i1y-f1&1@`R``#&q&-l@VV^$9jl!(Ux?`JkK}{3+-7xxm)QaFkxvcbZ;f5q_~CDa<_xgy-dzPe;{++P)9~RuBkA?w zisKpc6`bVtIkXu9bKpP6e;$4yf@??cas&)(k ziCR%#IU?#Hby>5rt-#|vG}S+)f9IiWD7iD*Qgx*43$yb#^19|{XX|FKf2CK@Ea((| zDf*MJKv?;tb7A$!MMo6D6H}a9nftqtv|zokvvBU%V99F4C#gu}vIsIJS0$=63w)3xK z5>pCn8f;3Ats4%V^*&0sphu5K<42em)ff9my4!X_g8uYg<4eL#_9jlnPd;eh5dI?Q zh1i9lgpi}Jpg*GjK_4QfCB`FMARyy8Gcl1Dq)ka1(;HbE>KODB8MVXpO>~;CQ;8{H zjqx$%TA^P8TB$oUEFS%s@ik3$PIZpGve~uRZKrLCOc*gsSfeCn z$*%nm5BD{c1rCwI{>n! zTaT=bi67t^3f6KsTUly%{_FhU+^tBf7@MKT`JmFF``mFqha4T6hTM*0Ko?G3CEt;? z2QCEK$Sr`fll5|*X=%936=W9^oz)JM_;YL^jEeRksv??_-$3G2A|Cgb`E2X#xzS?% z3BlI$|h1YXxb0l;S(f6qrm1Lo>jUU2V3>BP~Y>pk(k5`1s4d$57= zSErauXA9AT^l>_EXl>hiA1SeXS(ThrswzB@so3!i$O|nUkEP|LvV-jThLDSa&xTlJ z7qyIwOY%cSzIK+XrFB{Z(e?T|0Y?DOnaLJf13fflqOK_jqRpm~Av5~RD_UiIT z{T=@CM^F^&xN~~F-Q@1hVfv8uF%D+-{0p-R4_at>I$9?8?*jWJ-x|H z)zA5#{NDrFoLRKhu1_)&9d+a;WbuQMK^w6psDS^La{)5+o=73eb_3Qz0yX&BVrEuiD&9r>C%;RiJ9eM)xAF5 zY3J#wXU4g~@>d>uBA+|?)wwIvqd1lr))?J8jysKKxB%7X+o7pr5yMsd#%lZSsnY`X41?|VeGSN6FYi3gzx$btu^GECiJT?F3a>8^z<+z~sM(@9X&BSG;>~ir5_39N z<++Zv=HX0^*Zs}qd2xfax~l(9;=<4ie7WVp+x>Jb z@z9U?xhU=^j*(nfs48&hbx+yR%W;TY?d`~$(O+Om^KZ#UPU93w_RF>Ar6K-f$?cs` z%4c$)+UKIHv8?LBZX4on#4UFM7k{=guA2uPJv(o_JsRKMtlN%pB+Y8AtBHSPyTGR%9tH936%4u$T zdgwR%at0L1m}U-Jl6a_P$i!AW1QGFX*!@&#&21 zjc+;4_8^|l57_L;V5}dv%3-;ti@c(vGqkbiR=b``GCWrEB^W6i#B8&Fm+do?x!O*_9 z6yxt5d*I~XTYgakfw1pBL<A#P6nHi;QJA?DBze;A+%Nn&LH;J!%o&P6tokoB4ZoQPYRa ze20sCC^;MwDnXuj%O|ay510rsTba_Mw+ADeCaJR!r~j^>b%SGU1Vz~;;ZTY(%-2mZ zU_L%Hp@l&R(hFLF?AjnrWE^wh=%Nzg;M%Pz#AQsKvWbeq-z8eX0;ACZ6v$0{wGgw* zAEq-Xjk7u#k6g;~{}foye6iOZ(7s z2VRT3^WpkbxoQcTPyU!+Kx+My$emZ=Jzo%Zfl0IZBKGE@pVrO^Anj?5lZC`E1T_`V zh>>Z;(J2HUM9d)?(;tNvN}o8;+|J(0a;w=mo+^V)kWTP{wjs_|-f|5X&1?R2sv5nb zvj*#?j29!5VTXYv>v<9ul^uy?tg4X)??lBH?|LUWe9(SCXx`_@4|Ui+O^qy(x$FMZ+pnnjUyhph*=DI>8N$Gl z6lvJ581|G3n9bJL9$HKw z^cK1`gv%EjLIrr&%^t7xD^6dMB)f~GCYD{6bO>Pl^Rym(gGXqGp zuji!gFN2%YHl-|XN8oTA__nuO&#R71#g@Y`Hu z2{%i{b%De&DCLzfQ>piDc=nWreG`FxW{~-ZCN=8Jdn7t;K2Tp2AO>59FmW`gkUN7J zI92I{U1*zLtwIYwXQmRhohr+ogiiRO0)hvq?$uk6ttaipYqst?1t@h3SsQf8$g%R4 z@V)(c05!7$e)d}NCos{4na~)bw3UT=$~!ThWlSLy4M|yrt9+$rgJj1#M^P(#eEM%3 z3Hm5cF#*HuNY1!D8-)h|eQH)sO}G%|+;sJ*`ebOBlBkPvjPE*@=4V^QUhmwq`pcoa zZtLKMX`*Di&=$N>}^ezg9ZUaxLVzv8BqI<(am5Sz~yYQGsU4uB)n)Zou78MVicVlHetUpj#pawfLP>41#X{Ik0J_k@|O5uvD&(d;ySSajAv z$P6*pdcc9$MVIcH*KZijphQaKQ4qt>M7DgbIx{8Od6+f{0diI7kk)M2$cd9dIGTdj zHg%F@phlDkqM-3R+1r*eXg`M%8If`28cM)G7zxBd-Q`VTU`+4V;bf^9yOWyDEaqeOh^7BL(M_4eHu z9*4Uz9|<-xonUJ3T#ajjnNF9rg+6&5UHz#KGf)jj3s`-UOPI}U9AL{|cF;#vcRT=o z8a@0=g$Jh%M6H5WV6s7Vl1%92Mrc*JcY{7?vE zj?F|NW;WPWdzW^MhT{O$!#a+70P+CJ@-uy$%Rom!R|o5n6-} z$@a}a+!-qk*BV|ohB)JtwAraPPise9`<=f#9Z*godRo#v;E#T)@L#( zjc0SQ+wHFKLR_m_ZDm`#xaUO5Zl^xFbXY;O!DAijkAJ6z*4>Z3p(En86!YLA7#)oRTdd{ma z^*0JE53GJ*-prLPFr4A<%7*zxUcB%7ezaq}=3ah@GOFHcA*+n~9F_Wb@^3qq(&WNE zk zjiF}f{NJdze~dA?Le!=@AJ^rlCQMuNi&XQ!v9 z^?o#|qq;)c7U1_?*IhXBk#d{#*8!@1pDX6$k@WOs;o1#r(zqqV3s?ZG>?JT0*oc| zT_Y|MV4l-DwSQLwiwyPx@|_aLzK|LtO&uQdB2VfH-7*B!=jlqgVj-DQd>#A%QsdXX z>q{&={0f%djs#ykZ+As58-Ti|6RQpUrag2qE<$`AQxeozTRg8b_d4ySZmK(N4;{~z z*;1ajb#ro5fs^gdeWo}YRCjZZzZt^VJlJ^urJ;@n>{gx?}# zUSq+%t$ib`pF_ByU9W06Oly4zUua`Zl8r~?^%9q)ljyr4(Kb-rH*w*bp_ed1O(R^KF%ch}TR;-bLnM5a--o^3A{Bw%)sztsoKnu3X7(GQ%S3H&0^-6$ z_;!+^i=mZLK=ThX3kTT9x#~|9&z@ZbnBN|SJF&vc{b}8!D>)AsPxG9pa`bVXoTsCm ze44MRB6dS<4p^|E466TnDx~n|kFE!5yBgqgkceP3LE8&0u|0Q1HZ@Yabf0gz>7CBS z;FV+d#+pFE^v%P_#Rad4eKB~3K8cSn0hO(VReg3$Hc6ga$hS{~t&<}!^+I)$d#}8R zN_ZxNiU0E~LnmXGv2O!|oe8Z;J8NjdS7gJ=2@57|R!E{x!=!+Db6ie`|3#M&9yUy% zaIWr99?EN3xPw-3R^xgdC?df|_XkZTukr6&ngyHG3G1Y372{fJ8G~X0(*^m-qDz+4 zjMpnS+hSo|1_$0}Q(f-Ri?K@pvBM^K@hFM9AK{7%WZSCzc2?NzB?sRRNK-NBQKRPX z0;;j>;-wM4^dli>t;^i5*=v1PDy=cEIHuxTA`vqCe9eV=9mDqaq@RpoP~Yw+%%H)k zw^~;Br-muC0)j3$;RHoHWJ^_6mKU;U(kd$~Qm*c8B72z`cWo3gb6!^N?x)g%?z3v~ z!uD?Z{V0|DyXl)MK}Ss9yyUlsImfH(^wZk*O2Ou7uLu9QDLaSZc~4}ev|@5^J8;N$~bI!<7n)`J{ID}xj<|F^x6+PM};8B-4Bp$ zGMa2-o(>^DGYJ5I$>BZNVbi2ZlVD$|UhQpU6L-yAhYhNYcC4@aGk3_gg)QF-@%A5e zJ0CNOPL#xxVf>}X+MhtPd|8S@r^i>0ovleT7oxzyyvLTE#dWxh?AZCRQNg>J_UK^y zXTro(^+;zNR|_d_Hm1fDLiWIM7qjx>xQb<4Yr3ZBIh?QSI9=ddY;Is3P*k;DA?M(4 z_7X7kyZQI@IVdHx`Y&8Hj8q*@7AY{^dGXzROimc&$9MWcmTxrvfI&RvO^MtReTW3< zSZSrqFl8LCyX$Uwxzhaxb$ynY_m_N;UQVKK)3`vu;zaCv9(gnHM0`As5G@QZoL^Kt zfO;+afdU>AqItrBUHFi;u4~MnU!#e~u0WeYGMX@1fbIxy3O`f` zg;0JUFb#)nSzeCTn-dgpopPA%=RDgXo0nNlWi75)oH)^R4cfVk%#v-@lPu<>O$xoZ zNMzc662d|W%eqqHCvNJmq-3R}lIT(lrj*f6(a{2#NVrDA38^-dhAHA|$9YVgwv;<|tq0^J6a#@}nH_hR zVj&1GCkKcnUrwT0&!Sz{Q8~-sUNTOltGgF4W7qO9S?3GGRpqMQE%|DoK?l2qy>4qs zro(*mMvc|QdZHTVqUPVA;dhScg4N}DeC)rleCSbVTv5;Tf2I?ed?7NpvhuzC_$3W! zkMRB35GWfeWANbVlTEEa>r@co%YdA-zpvIsZ4T516CfWlfGM|E#_qL8c6aIUTtBxb z>+%vT!P?^56kltn4Z7T<+^rjD{2@w@+HOKA(I-H;D9BacrL z*$tNG!^iI89B;O}=bH@U{ruullt-hs*eGOK9fv96a<@&)1+T`Lus7abvhjT1!blFw z8c`u*)GGow#^C`?N`Y`r8saC6lX20Sy=Hl3yke44OyX85PN6e?yU(dz=s(=a7&umr zkGMHH9C?nNPSgjy%~!{5w0%OxOnJZgP~imQ&4FmG6Vj3$2!v5Cp%F^>&=+VDKz3Pr zn5PQ!y3XTQX8n@k&=y&b9dXe*wnW+&VK)MA)*nJfk^%f)N8PL5UWtMkr2Z!udG=Bi zMAP|MtxtIQ1DVy&2()8tSvj6Pnc{@J`}IFJb^5ORE^)J&Y8cB|YK#Pf5$7>s%~9L_ z%O`8CRBhZ92^RF?y!seaEy0bfLNDnSx_v}`Nqh|WW9Dnpaj+a_pcX{QdCR& zpYiC3cY^M3+aa^f>S@6_a~s}F2_qW!J#4)0v%hKwY-x*B%hHFcw;Wdu`e(PfwI=iE z-ShJv3srN>_`_ZA7hdjGe8#EfAdZFsn@^R?2Yz!nI3-Tj-|o!y-#0J>7&AI?){sKc zMT0$&Ow5=w0(3U5ILpvP(7J7ETswr zK>aY`9z;)=c+v?hfh5oykc>laUNvr1FM3Vwr^K5{Z@jV!x<1O>+d@5mTMTxn1o_{| zUkxZ-bZ9SL9up?EI@D{VJPMJ|msy;9?X?p27BLEg1x7TnyFN$+M+_j67jM(cI%QW6 z)|j-0%@MVV2N>Aes_r8w4c91Z)wc3)YbW{@H;fhcdn>{V`gA72(pJj{RMV_#O~og1 zeJ^B2eux*(1CN<`H++%Til-T^+}G;d+r5?ssms&6Wwu8f9F}9y`k2JOBC7Z5CZr&zKBPaD3A z%tbT;K{asPQ{VM`4io->uF8GSMK!{Lb6|bBX^N6fghjkh8myP|__R{c9kY~)hTWW$ zX%hnX(wq?;+v{&Zbjj_=JK$!CZ<$|S5nON8}gPQ&=r7rFI}Q0B&+(RYg$Vc1A?l6z2ZJL>Q% zzDR3xT!>R)m)8hsU-cf1S49S&POlGhgFo()|s^chHmPc;% zYcmjqFgp<4l6AP$>Cjwp*1h}YbwkVzy`{sn$(fd4X(IB;Z*JBlpc+K)Ly0yCFVbYbYldoNU z&B95EgF>Ib3G(yL*(2Yo9hl(cizXtH5*F!OFsDYEaWc{lM&$ zJg05z+on+MP!oA|@~Efq^;F=LjWr9`7p9*!#i`Z1Ub3PvIn;*0>fb%#m7x`gE#Q=X zPHH=m%u%s~UpaoJD;MPc`EOMd)lsIeoLSv5Q8Rw_H6fZErMFd^=P&r~%Z}HLp|Jgz;NH@bMF( z{RVf(La#B{j+(sUM?!V+phfFL>R@<(;505sD z^I#^p=3wovvkcPD9k0zJ&#gs12B_YZnf|+ZmhE_lc8`;ZN@)70V-fz3t*BM|jLLGf zg2pEuZ~su7O)a6u zm_#ufI{5(tMCq2)z7U#iUpA0Ft@RI{yXwh*Q%Fn?2&x3VwB%)C8`IO}mUjw>x66Ke zKJC$GAcP=W?#xZ2Z@;dB4i!d)veKz_xoLSbUzf=ZU+(*-I=ISfBVEb)C$kfOD5S|s z)SAatO2@7T${j$8DSoO=@rH1YXq#f*pI^N4RJv~j>owKtooL&WY0+P^WV1X`TLuKg z8NTkv-pRQO#j2L#o-$5!M%^er&u&k}`c`_sp8gb<$dQfJLjd7tUT8ZIYGux>L2gfU zXtWD(k*ofK4p_A=Et=bkSwV*Sm&+zcDpB078 zx>+705M^ORieiJC|A&1>Lt#3Ho>BppGy_5{r0Hb)#J;fy>v`jHd+|$Nqp>;UYO^>+ z1xw2O=(qHI@H{Gzu%RN5Q!sl7N8?W*F)ZJvnb$rp_^~EE^9e-=X(aI$Lvgc#_E;JC z!!JG+P0s6-nh*QcFs{bLOAL~h9_qOHT%Xx-Z&IW+TJ=kiut|fXfwdQ=HO@(z%8q5| z1s|J<`4MAO4RW}t^ymrlAWKG5gg_tX&)s_|K!iSX67uHF&eOEN44Y-$9-**R?mVsC z67vFy7(uo+hwHe=IoE)pUK)zy%Xp*YEt*ihRK?~k6_=NBthx=E9=|{fD z9U*!}-1_})*$WQl-}uADyNtO9F7tm+RNVjXaRexsE~{M;*!SNDc>Uc{xl*H~xTvv_ zI{}aSd(Mu^`*&*XBQdi|sWRgk`u=Wo(E-Mp)i0}-Np_Cf&tkkQN@~vzGBbx%pLIMk z4lO+$hu7uS5*B^@=Dsyfn(RPo@YZz~M+VA};!zDc2~oFKgNU zJpb(z{ksxG1E?9+#m2%>90`J~jP9G#y3~bRZ zxRXgF2k47yLcX|_d#t{fc6;FPq>SRH8VBvZu`W*kVabpZYgl{lMx!ruHN$Gu<&T4B z-R}j*ugJ{&&?Ug+%p~r2M+G!=|5;T00`)66&E*cA5?`c@e{@?ga-O6jj zM(TTK_dsx*qRd9LwddY_^G2F2+!33nIUmz^wI!!$>SqQ_Z2*6;aA_y4NhsFAK_v;_DqMy^Qp_eJ41%|;k&X+byJKbk8Z z(X0D@tSMAJ)e7iatvlY~(~2#ez*>ZG zM3Nm^%3%2cIo(g}?)IS_>g5&m`zxITq=4u$=pFCBNO022S+`|LJSSiw9kTZ6sGlYu zQmhu+YUPtn)tvDON~{2t9}}o5id2-@;c#9SkbV_uhy}aY&1o%7DTbkzYyX$w*9d&j z+E=57>xXNI3PsLNqRlP76#Ntl69zc>!gTvn*Icv#zeyq_tM0?k2U$FDwkRtgrp25? zhP>C1@w84>4;3-7P)kFup7f=Hz>LK{?F&7ybRWvzUWgP~ybZTi3>IWBMv3bD4FTT2 z_Lt;41sYq7pkLHds+yXb#A!oKI|ji9AAk0gk5)4sE`QIY6BG?eS0)Fv=LTW;mav%9 zASs&y%M$iGwHJ~^m~~G3>vZrl>c$yYG!vadB^j_EL0?VUU42neOAxY(sX9)27F+%=Qi>E@?eiVKLS(3hVn$o(r2*5 zhOTH&pQ@?ZF5TVO_+@pT{yNU0z9lGOr^624eC$lByDmVZO$Qi>_FF%anmSq)>ql-` z_`&QTE(%n1N=PVD9#J~>s>6Sqz!iXh9bnh&v5W#rU7KLDe7CUCfqvX%4Iz10_^{>1 z_{;0Rzu$(01B7pm(%efJWWpSch&xrX8C ziLt{x*WDMNF1k0X4ZA=8T?(l@_L7n(XI`Ax=}%-(!w|kVB8nh!8IGjO{??yp-f}d? z*6sJOpJM9zou%n$xxNJ3_$>Y-!}HFW`a?Xvf!DU-+oPsldLG$lA2dP^vn=<8W{i5r*IKxdal8Cz!xZ4AZDfD<$Fp_$-C=)DJF~anr zl$)5eU$|gt2G?c<0+hkyVvHn7<}PL{1{xwbOJ~dM$|q{$?%9)wE4D8Lu4pCn{;W6x z{09sK{hPS4cx%l08pDlS$e>doM}cq3VtuuH65MYkhJ*j5U|cWp z^z@WpJiiq3_`7-Y^>MK~VCwVT^dQL1DGuQ=brzdzk5gYNd)5p^VyRHQJs}Rbl=SsolCuGWsBT~TcNW1 z5v5{yB1?VkuMT3tXdJs7SD{i22YgB@&@;onY1!wrlE2b*7DNY-$Yk)O#4@`xZ_)N`Fcrpuh3o$ZUK40S=Px8jiMhFL#icWHA$ zvl?bVa*mWH=}Uaplt3sd=5a3x)5fi&q*Ezi5 z+vfTx!W4NyMCfe|)-^H~GYb~+b5b&JRun_cId9L0Xe!f%6eSkcc;fUSNClioX+V5{ zKpjZeg$PKGmM85wf@=HeyRK%3PH^LLz2!~pNOiy;CJ0rBwac=33Tfc;RYiH2s%8Je zUPxO7jYwldI9HhV(H4O~w5p8)zzi5DWCbbNMNGQxd$nbeW}q*?;w4U*W^~j3*5sGW zQ4g+ZJ1dpOPI?cuGk*-ivSKM0Uy@IaXe2oMFg&BdI}mKG7a<;f1TKuXD8!l;$#wJ^YD_?&>1_0+vmc8+APFk9{%$fq^_TIO%-f-Vz#C z`2l8EZO!C;rSXhmT^ke)6$xmCVhR<(NBN~GEStJCMVoT!L_?P(n$+RJ4w>EM=5?w z*`xVKX(WoR)K$M_cfY+p-GtIfRChg|&NFPJ=Ya@md!f< zESP?*$ctWCYjvel%=~6{f4QxopwK|>dsYh#i*VNUbgo64og?7Rh|N5`?$nPyGJMHQ zkMl{iI8MYXOj!SRnVT&m z7$a+fXBAD^vEynmnvipP)26)gQug%qCg;rzMc7p_!cYN&GxX-7>xng^4pW$N(O^mkZ`Mn){g|EXy*v zm?@aafMxD)T3IjG(#)=(!pcqhu)1m0hPyt&Q#=#a2)-)vl?|Lr3Kl2q=#UaE$pn-B zR>}SstHEkolpep5_1;cJ4fTZzCme^QUtt||790P+n7m>HE?GN@8euThHEJ3cnz0SX z#Drg*Zw1L32S*h*T{cF?n9-$JhMU3jQP3KA1rTGvQ7K4fh(2;bhdkUI+9t+zKi`&! z_&;9_U8SwT)u&EQSGAu%hLAr)2(2+VQpGRoXf4fgMNoQVqv z@iX##9s3J#rJeppA+zC&9F<2o+uMXHi4PP3Mpc&2zF#HY~0b~pgju~;c+1pT4CIEK_(@# z%QSq(@4km;$lybz`ucdH**!|8izVk6laoDeLbMc4AhURtO8W2Y6xPc#j@!;o4vIGa>xC#lzHhQ!7@Qe*O@!^N)vb04#8kXJGd zjaUmZw{nAEMq*2{eNZ$|CRat?^hPP!Z`}O zyyjJ8K3H(VD=1Qeb+?WBSen=R@UT?PB%k_TRX;|v$2TtR8b^>0_Tp*u9(?;ibzkIF z6_$Re16;ezwO*{!ZV!;LbOy>0Sy54ciecIQfYd`aKa`wFr(cOhY=T*p8p;^23WJa%v>V5xHwVufXEN`yqz~)qCk9@r_gfPB z^%1h=cVNWT?qnKgM2XY=4-DNwiod)6{4Y-Uk)G8UTtwK{uLCg|i^^=D$)EifB8au& z{aXP~D=^}2rJft!mR&oN{r~Mn7?mM0*W)68xDUfO7+x8$#_i(?^BxFr`G(*U3&3)n zHAB_YpvVAa za9w>8_X@gXCJ6}AuOXDEB2UXxz<7hK5#1kmcYS`0=MTi zd_teVKp`eN`qdp24_=-j#c8ur-EI9Vr;tL#FCL)rH|$N%aEBd(0b813FK<{)(mVjk z1x4_@g{oee_WOXPv(=5nS75g74)%h}S0Y>aUK}E!#UCkRP3s=I=nO&P6ZEZU5c7HZ z8yT0!;}T3#0bFu|modZFKI!`pZz5j)b0}JJn?>bih zDV5X^z>G0ViQ{&Tv9@F7T^TNxX+wr%Qe(m?dBY0eT%u(sv935TLa;7Cbff#9^i8D1 zh2=qYPB9n-ONkhH!jASecvt*!U!533mk6lV&Zem-w6=PYRH2U48YnLeouY#9=~E~9 zQC;X;3QIqZ$iSgW8jl6?tLw5+Y#kjGWRUa+FUPF z$4{7YBe7!T;vivHLK@E=Uh^Z!RWB|&e>g@U_52EMby{w5yCyw*N;Fi73!i=;rdaG( za+v)hMwcW+6Fw;MreIx8y@+i#=qEnE@lQim06;#={qxN1YxUbB8_#z*DmMl&z%>;1 zSp{H8uqc|eSjZ(27J<##@iPfAqtYJ*NFH6mz<6&7`;P$Dy!)8 zy_=AE%u6qfRa*TC3B;n*{Qu?tMkC1o=Z=WY0Zy?{_*gR#2`sIWz$Sf3c74B$~-}y_#>#WGgsQ+ zqZBpbx^a3$rTe(c`*BGWj7w1B;Gi^7s#FX(+Ybew>03Wr_w#i>rik91T464Oo_;(g z30x1YZLL5pE3bxRNIf-IDG1Iep@bEey`TUNP8%BqWhpN=gp_~TItD|IllqBkFA^%2 zm{$?S?;1~#D1Y#i)jx6!poSTi=?MzO_#o`UT;t3^%f&9PV7!Zq3&-h|oF9u)p@;&S z`cAR;5tCXT7uu$cuLn!eTuMPEh&28 z`oAz)L^HsFf+mG*39_Op*+PSVeO_xBxWaM*r1Y-{8>U*cu~X8`(DIL##a#MYBNLI1 z9*VqqFPNxXkik4vn`@YG<~aj7o#uw!Pi7&)zf|I02JM^yXm zGkI0fl%`MoafY!;xSAXlr5K1;sf1gpjn%b1+t;V$NkrICN^tR6O2UrmS>*@a z$m`AHQI`x7CO%}SH~{}P!V~5pgpm|ivSNG3!iP`GPy+0u->Kp-vjmwZQ>t*4_QcUE zDv4~NW0T@3KM!Jw=kx6dgdcHyLO<{0N8NM#IK&~FR}NOf!QUWru{H@ET+ns=>~W?= zl!-(U{7V5n?F;n?)|9AIAlz8`7$+h1|HIc?M@9KX{iA}!(A^;2LrR0hFm!{YG)PK^ zba!`mH%NnYmo$Q?beELUa39{a?)u&Pe)ad)xR%di&Uwx``|MBbJ=A=IJy-~m4&t`f z5sX0w-urTCv_3lxZ1u64OEW9m`L03e zDKeObm`LXKMTK9G(I*YF*3u=p>e*pr8l~hdCE6!uby~;#3Dy@(ilb)n%$G^=#L^2; zphDELncBLq!pAA&D>wSVx&;)b60~H55J)Cz1nwnVcyq{Wqv$^kB=YxjxsL<;cwE|% z+V(q~xFT>!WYy}GQKmA;YRa3xarF6n>2{S{VM8=2;7h<^*w|a=IQ(dn7N>fYH-Ti2 zOs#T*scG)!K_*b?MtIzLLk(PG!FdLYnv@~+OGd5&6I3J2hEuV0 zR&E|FS@JrDVaCSb_I{m{|EeBfr00cL|IQ_}EqXsxfxLj*fi;P>$e2>tt7(UO!92SO z3JEZUuau96cS#?Ia*{pWbw7PB`@-OyL^sFJQ_z5+WhmQ7hlrVQwC34qf&FyBgcwS9 zipVX@)nAySr6vaHkj-So-7kyOPT9nY`04sRIt=hAzw3P@f4Y{U@=~l~A0wgl+6ZOH z9u_XlZiPRJ>+J;#E8}b=5E!Ny>!p=|aITEy3pwxl#RZlysK zK2s#0eqMcu;Tc!O=fi++K(I|wHGxE}JqF;to;i$CThOJaea zfGT~3g`#A_I){#n)xN|76_@0F{nnc409y#lf3|N_cWPlIF8-7a%#6P4$ftnJe zv=k>#sh%&vf8E0%t|QICj{U7y7_FT%;62U2Meh9;z04}+y4=N1rrp;RnZvw(BAj}f zq8HLQ%4(PXT9X>~_AeDwv6p&zg_)v+(bC`JwbM{938LW_NJ__hqAa?_#N3xswru0^ zv>6n(bznoR{CF~mi+fFK_LSYs;nvrB1K&sg8aI^E4{n-MVD= z#y=rgjw-{_PM;}69vMWL6Tbe2*n&N~pZgv&0}>4f+|=h;jPkL}UvKl8QzWBZioJey zUTyF{dhH6Xs`x-1?^QU1Yk1_oIDd=9ZkJDWc=!a=&nlw$FnRYgNHREvYfH1^r5d`R zj)4k4vBPs8mTC_@M zn9NH8m9>0j;g?mdF?`CO81y7(j09}pKSeGlI1@@l?`ITaesXo$0x=2!)vqh2?L~v%_n_ zZ^0%o4V;5=47yB4mZ~(IBkE*1-V0X_>?1l}?^EpR4I|nd`G~_$7I>zO@Ojb$bQ>&~ zPgjg1S?m$)+$~IitV=RRMB2ihK5SwCJOw$pO_zPfr|nIUroce}sU|<8IR48_c~gn` z0?kIoGL?ob+-pO2^2=?v;1U!HH7y;s5#yjp*w-;)|8Fk<_WgO$G$&@S!_bhZaSEr@ zjwvkp04SqqltoIG>r3#5NLx)CxX?!C=dV<_JCVabX&{NKRH4lc?dDu(MS9{p>)bxL-m(+mtFK7=lv`|zb z72y_Ifx9fQ_o>V!9Elt1`og+Zj29t}n0BWqUuDCWc<+>SwCK{4)+=+TC{Ym-fDS`X zJ5(0&HeRkE*expYI~TyaqOpC%ti{fL1c(NT!`=dc;9hW-W*<`$b8SaLaO+B+Zjyn| z%{IWmA8#cpX|M(!y(nQbrSPJDlTRKXPAf(3&1K_t{k5fQ&@S|SO#5jyeo}lanP5s0 z_RtLz`zrm86)G|VbuTU6^1lA}#}vYC>!#aQwwqKy)JTeC`hXeu;%SJt%K_l@IUPpg zC^-YP6|D?iHwMCs5|#UnLhM5j&Wo9}QbZ>Dn7>E_Fp~E3A=fKa^H_1Yxx&FWPjHjR zJh6{0#w&_EIJZ%f@kc2fQHX`=@HCZ48>5j$kNF2%Pn?H-c@pI_FGBD=zHL~1QY9>) zB<2f&PYDyZIPWI5dc1-_f}Yk#Kn;5vLw(FTf06v$p7}i)-H^O^tr*)V`f$Acp*Yn- zXzPs1e}b@#RJ`NEC931_@Ww{pJI`1 z()l=2Cpes|Z1bzKB+KLi=FdPtaa#{1FB!LC)JyB5>!wDcJ~v>11~zU+e28e4EYua0 z5Sg6)f!^d5Ii_N8!Z8--N+UR1st~ASHPPo^R!(W@sx_(iU+rWN$6}lstZzw6PD@e5 zAOiUz9q2Vs;6?Yl=NJRx&6O_kVbr(t2~bHULp{^6sg;6DqQ=nCU{#usG67=X-J{bR zy#f!U8(UWL+U;T`FoXV2@2j6|pN}~8cEtkd#0l4FRqE**x#P#Vg_LDI0-W%5tHQqW zz3bhqJqiv`He!!CPFedrekwM? zP%e%|eLaKn7LwENcNFntEWJ$9r=bB@R`Fsdf7!~in4opz(wKwt$!on>MHdb1 z{N}tog4D3Lqf!i5b#67CLufD!AiW4r`z~&t1wQl7TKY&3o zg#+ay5fQhaFc7S4!rz{uD1M)gS87qvzTV)|fBiCA?nqVVjM#lDdqa7e9PAJpphc6ke~lTV@*)_bi;T^v zJkFQFh5N_4y}&uilATF`v93mPM*cXsl?+J^io#iTO+@BWNH%yNFXmsc&ov!|Zy~2x z;~%aQKh!6o#--^q#$kngi>fj&0i)K)A*U)@DJM;$*74mD;0yB`c5^(>U6HW0!(v9w zbfulPlff1&;1|&IJp@R#^q07Ht|HMu#-4bxFUi6->B*dt8yF4qh9LJ#WgL^}jGg-T z34CCGg%nA`pK+YnQ6t3MLk=O7%jfI@)S0R&KDv`=|2=KVDLqWSGa;C7IatCh_wwM@ z4~@fKOHgvfvu8NFNWRGLb0kiqdXWMLAD~ZzJ0XjR6ub|&V8y&+jw~Q=1mf3gqLrc= zHkB0s?aLBvj(M;{*YgnPh>BAf=NZZ1FIIsR9aA4%{NjN;^J46%`$MC6-T}Hib7C$K;js1T!xRyr=#wTZDcWL zkEDaAHcFp=)6}G}{1<~?C7l2JC*4D-ZW}7204>07fNaDf%wTmqPjrd5S0E$a6f~hq zeSmo>X{|zz1j8=IdCR42rpGUusbW^1uhJ%VBCe!2LdQ+ZREFhJH{l3(JfBgcgWf<| zX-zO+)l~Mt8T4L~5gdwj*~FSc)Q8?g)2Nc?4N%1fz7M~>7R|<4TDE>;+9Eml1eL10 zsikOlWc68tfiRGg7jsIK&;n-4*T7ILx-0m4xrd|&ma91i^en2>dHiCFEJlXcsgRd# zxE9vdRW;##!I)%b`AV6T#PgtZ$V($Gi9jm4s4*NaOXUq+?MMh$I&C1x@-Ha23j2S& z3RLQYH*l2^p6VZOUoy15Pl0GV;zMT|)tRwV(C79G$`N=NBSL9Q$IEEuwL}w7)Ap~a z107||no=%jplEF&jS;Yp6P&VQ(qhpC+Mg)zK=FqD;TBdJz$NXj5V)XH!t8CuwRwPQ zl&Ev4y7}Y+XkPtjywcKN9QQB)Ld9HRzhnjeYC0;1J)U-tlSz0Pwkf5AlFaJH&+$^$ zHH{FaUlo#(ZY}tWwkx3x0#GZou+W` z)st~MR=8?2#H=%4BRCB3t4B+$|9${+sZ=^-Es!?|_praPhVT_yFqM^v$yZ)$H^kX@ zN(}QBK&_k^t0byj%8{bdW4MX2#}Wzkn9WkyC6ACxIV!bPVjN@&%ALPpt``QzFXL@G zJL5MQv_-tuDp@Nv8zujmps-b>QuZm9c=c*M|0}>s8h)+PuAg9T=x7{k@dv~pyPJ@V ztoZmnKMhmdww|$)KZ6Gh(`JHu%5niQLiuJ%lx-E&<&tFxV*<9+=URr+=%A#k1s%-w zJsLT7IpJwcax;}Z&WI98jsS4#Qn1Qj9F_BfJh&wtK>!yCGtc@L` zXwO+tUe1`4tc7=Focq5iMZl+NC<6*YfoG*tmh)=_g-X3mWM0R@S#Co9Ti>9b1F0Uy zgoR(cdI*B5ib3~b$ZXc&mxVn4b)*0T;Q$(1X@~n};q4ML0&QYAR5sLTm`_Si$Np=i zBgX=oZQX>4)-sI`d;l?Gduh}_4cxTVn+-+&GU7E63c!=+`@-gTZoxh1E-HG#cWNd7 zKNuKj3kD39`|irX;);X5VCoxtO&|>lqt4$NTUFxuzXvRY37$l7>t*|yjP1Q*sn~%R z5->4~Z;EukOZ0)QCRnX*H@533YPa$=%B$lWEUI&f2kS!w!2WRo%0q%ttX<A$=1<|Qnnp=M2-JpkRE$de53x6FrX3Y=zX$wMBCr%;Fp;-KGUL zCi0g3p_8wi7pXhM4NYZ~JOAvgHS*u24>0}W0n9Vo@_?y-fR)5G$R9E$s=}*RkHEk{ z?7JDwd(!>nvvCTDinVsDT{N(`DQpqclFCCmTz3G(0hr$y?Puk#F$mYxSH)Ezc^vE_ zCpVs~sUO}{-Lm3Q7|m1QFWiKU&oCW51`d5a&rsJinn>SpsbgJ@_xteR0H~`YN|?%K z{aLQeMQFS+U#_MNG1JayE`iREk9!dsOMQ_w6O%ML;Ft0fgd*4<;ssJ~3tCjM|&Ko|iLHqvCqbGZLKjb>4D+ zDbh0hO;IBWdmULPa&qg0r0URkJo{gl->QhA&tGuZGVkiR+5J4#nNE~ zZ3T{iLuM5)%tVwSRRiXfu1SGIAiWJVK_O*XpMPo271r-xrV}2Rrl9536I%~fd#wkr zrfX&t)`mMv+qj@tkt^%8O*m&cpsw0anM@tXr5vUwc?LHMgGhS%!)a<}+XA7<+0Rc- zE^WNbh45~oV2D^7+fm$PD5{sv+9-?UjcLX!&m)clKz+wFI{txfbNOsONF*1F-V}QG z;gbX)FFXAH!n?Xf52%w}SBu?`P3BWMRT4L|sa*pcI3X5kcOWblL2fJCJt zgVlA935HS-FhQ}5jTCIjn^WZ|bG^x^Zi)(>O(HAFD{;a<1UBk+s(~U&c$BgnzeiWu z0zt|+EDp!GHpYhpLt~ePDWQuAz_TZs84mU*zC|Gs3n1n0I9RA$jP4Dj zkR{c6+J}O41OwGoa8@u>3KMqToBri<^|_$LEikiERF zA6oFOGjR5QF3$G7j|1Y9Zy*{^4|t3LbA4`_2zh_J4{%cr3sR7JiTchd@RIS_Q#gJz zcWG+dzc^}`!oImkPkVg0s|5mV2T8&HOnnko#dLB}vl(doI-KNM{N*?fy&_cDcp!{b zuifILvbIa$PQ!LvN-JZOy+Sxy zJlr?8&FD7O0S?lUI!q@O&-Gw}ovB=<%p!zT3wy@vWkr2g!k@qqzYGZH1m8-Ojhc7n z;^RZo`#c-ZNvR>Bam_(Bf4STZU1;m(8L^E&d{PeZyCxb1q_2-TH_-Y;eb0SMRQM(2 zO{7Vp0%^ic8)B=_>5hlWc0&4{CqPOYBQV7@7Q>m@QlGK&xyURc8Ih$})U_ft9MkN+ zT_7;@|TE~S-w5sMv+U*Q~-=W?2~znz7``k z#3THn0bN1z=3!arGPFTQ*^FFUer9d|MZfI++!3Pha67j=>Eh_<=x0lM!B%~q!fB99 z_Pe}5^w~Iwtrapd@@9C?OsRAYBWFNdcvixi8efN}?4*Q179wSEJ}NAK7^UWnokn6{WCY3af#JS%ijEIs&)A*jk=sDny*b2-)C5qR%;|k)nz$ zIp)7u00;t3An4v2zQrJI;{`FwU^M^wh_VHsvC$w`mlzD$;H;X28%nAe40!YB?SCw(nPWK zG3MGIzE2OIVlkq-H!jwK?^OpimY!fIQ&O=M%>ezNV0Z09PM* zm+m;CEodXBi-b ze+k5qLp4SI;6&Ud%In?zXzEy@1Rgnisz5jx8{t|f04c|DP!i-7-~B!uRwpP?yWGcD zFv8s=%Zgg5@TY?(+o`9zl@(Wh8B~L-j34)UlJBK(qlz%F91eJ7l7LNV51_VHS&!@b z(RD|SVt@D=3Aiff^Ct1vU{H@{Ep z?R~;#+4@^Mv#(sRa@karWg2w~FSQP^4d-^kLb2f@M$6r;lAICj^rzSA5V&{6l&Ssb zWi`W%g!EX)<0$eRemD$MYR0_C#I14|57B+W%#@uku|LbVlV)NsYNld#;*a<$j4^~x zKFv?8dUk#u8ptYa&mVRult`~6ncKF7Qf4x9V%#!^P@>#V9E_lc*S@DVYztiM?%7bi zW>G5fDPIaon*m+jL3iT*5VU@B;ts#aiwx|KOqp7u>6FI=?~(QB4%#tq9m559lzW&L zQuRF;F*5SxRRJR{#~yoXK(^vXtN$V%KcuQZz!oS#Tjx>+O_n27Ahj_9Joe;3I1WF# z7(!4#aTF(XY0CV4>p}r2H?}#7TqH-E(#lSWs;m-3&K(seY-sGo0QbeM-Jfl_4j;Qq zN_+^0n{Kw4t|9MKA31G97A1%M3b4V)tT_5@KPMA*853FjiQ%dx{v>=#om;`%S`v{~ zflEe~|Q0Sxk&=atw zn)gKl5B@$&GG5q~kQSql18u)hlDdZK3RAW9cfXYB2DIwT5n0pCZ3B*)3>$9Gng#5v zlNJC=%cEQa%srNp`j1Jd^4i_|vk~@vI z!Acqi<^1`A_zdJODmyM(!two%jWA^sxHgx6E7UYZqdCnD8i1P(`hn|)RfCg`K`yN8 zAXpoU9idiKs2d&9-pOqMsUK3A&+EUCj9Y{7b6*5qy1~vb!!EDt5++!jxk|$k+Mx2tg622jz zc5Mm4!dw6Wrf>MYa$J@U3{S5RL}GrgvbXHW0vFzqT-6hZV};6N7LH1xj6gobd}JQh zhb_bxpdHN#6`$%Vn4qP=ZS51s_u~XIn9i$}Of`B`mDb}vh%EAe0BbaC3B!`$1D>W6f#|@&$NV`}rdCOjMD zkD|)r*z}HVA#B(6qy|&l31K1Z0kD#8zFZl_;|$o}e!NfBBg_!?7I35yf(`I4rm?Vp z2#p=LK>j*iv?l~#s3YMTCnpARj?3$32yPK7`y?(FMu$ejIhbUja9kDwNt?@o7{8Uc zjhcOd1*tMhfsjzbkD{=3h*sBZ&F{TnvZ8v<+(%;bh6-7@oMUA&Qop_t$r^)UUiUQg`2Cyt$kPDo3mVbEwp0 zKK3j8Y_J84uRucT z3RGdOC2|04KI^cOeF)__>~(*M#i3VFNvG%U7dg`7hCYlm3*qH5N{y#})Mjef50cHM8Ve`x#(<;kdzBd+XI+m%Xl zoEAeo)2Y@?rW?Au`h`JDEbXQi@=g&JR)#D848YejJ;c9xzJCZLzAIoE-fRDFon2Z_ zg`#1(8qL7yz*Nw1M1OeuJ~k8e6X0C;J{u%(`WkqhMZz1ju}2Xr17ny!5o%@%JtC9{ zgL_X*pXEuDBr$h{ac}M?5Sb=Lp1fqP;bmTq8Zj!1 zqx!PTnFUcpCIVgvILhGY{iR;Jhe^S(9JsdLNsI$2aLWEHoemvY4+{x8_>^1Qwl+^j zS-~_0)=D>y|Id;M|3GhxXEHn{pMp87fb@v9uIsn=2KO5T>0#PjpQ6x*#R1o7!Dfsg zI=|Jx?!3GaF4?QaX?}Drj^MbuTvN1i9OoieAO<6y91%6=$gsgOp#3_E-kjktHr5(FZYD@<*WLYR_#BhQ>Y@v24hQcA6 z5e`TP@m(wW5(;L;;bJmdF8~F_nE{o9py3nCgGCU+?SXNe9y2dgP++v|uW=Ls4uYRI zXCw4>`o@9S4D`Su+B8NJ*~j3tgVf| z3ki~Hku{yIv}B(J$8T$I0kO4kS&sPg9*2^LqYnwUYp3#%A2q-52%!mokZxEqBO^n) z+>nUU`V&?3@<^_0RwXS>d`uN(sa(kR)2ENSXDfCl55wed6GoOOHo6qj3sJEiCixR& zOD2jPW0W(Z@Q4)S(~)USOyvX9v15>flTgc1hvU*ma5?XDJiE1Aue^U9{}^^X@ph~h z`P%ihsm4Wb(WCaac5YM6)b7R4h;OR89tC(#elG&p-*rT5y^$@Uk_ieK06Ce2ON`KZ zjfO(VANHiY=?bMdB*cY*T8%NwICeE@zPSih_4E4y8C5I_4kxWwJ96(V5@+C?)K*Oc zdnnjJsIdS0qnyOwCbso^urz77AV0Z1#8t=DF-59sj@ylN5~}c%k=jkoNRXC0!&elgn=UZ&q&H?Nbc1Mx*<%gwX?Y) z^5i|`46#~|yYD=vFLBCvY~GQH+Oe30(R?~_W=OfvfSjkB+^2lC0uh@mv9k z5xOE4%LF?L7xw7G-znIW$cX-*z572)`8o1=)*H^p_iU-8-PD6v@5zmyZ^F z>i^gtSYn23A%y5i(9mPYT^r8Hm55MW;eZJ$SW=u2tr>rP8Z3$)wv_yc#LV7^scx)7 zd?)H6hkAXQ84V_Om8%w?RA}k4Be1kMjVfIW1CwVlU3>e~)NRdt1vFCj7DJc+=S2K} z!$aLm*wPHl@Rw~St{m{8Ugj>lHl-(Ix316%53#~pooRS#xAF9f1L^e;At<$T>6*k( zA>Cb~*^Zt82vYoc6*I!mQBb@+8_q36sdGt8sdJ>DOgSdFf8Et)>mPeu?S=mv0^_Bc zDfI6Z9yq->9cmsN!N(b-SdVJCxAQ;|ME|bPF2C<7qHi4tk~5CNI)E!yM(|r&>X}2v zy56ySG=mMpTjmSz)H;9bzFDXY+5!Blw#YYuD+fI?Z$A$(tlRrzdy5^B(w28;EPj!U zNSM~L3jIs?c#bK5K(M9tnBj?k4)w3vpXoKlJAkMDJxK5e6f+(@EVRQxemfo7bs^Y& zVWs(5$0+iuWA_c4*E)&>VvTb#v-j{vuTVl54+-W7vW`6ow)XEoPv|o2uYeH&&fq(E z3J^G(J~&U!7>mxiKEUh>peBa97duZ}2ATCa=C1kZC3ZE^eeZ{T;qHLS^6Td*e(!$x z&SvkAqI;3CD6~m5)v7nLiDu{4zxsH*YW3T*`^s~Fv@_Ajqmo1%p6e$qG*t}?53GqjYpDMqF)zktGthXu>-)INthaL0a9caq>+i>QNey(Q6J|=bMiwZzR2z7v+1P&=iLvt)1y_4(_8dkb!4A8m z(9<75qR(;gI${scRqRr(xDx4;Kv+YQO$Dd->_w99uE3)E7i)d-{-#=Qn;SW8C2EA&TR6>ZU z7J(vF`VJyMq!!Q&Ou9BV=mDwOQ;FSeiSKQxuLoe|Wz2Ib75ur+sN^0ot>o%^vg>og zV&{&MVNVJb(2niN<@(EQ4jknpF<_0M8_?sZpJG3aw3rOfl24o@$@a6CG*77tgZWeB zm?e8S66djXsML`Gv^c_bd zGLDn@&CNeTQYq#V(Ar_H_GLrYC|Nk(Ab4vQSi=uUXmx|g%SfwYorzxVk7e6PI(&Zi z>S-W(b&w5p1UG}NO{STGMo=)j8S&{1Gk%^MwwnIvC_&$vJ{>zFF1ZoxmrD7qa3*v+3Td-$Y=eXXcji0sy5t;EncBJ6*j)bO?<_k&)9^BQk zirh7bc$U1*-p7h{8_;ZQ`GBWv-%LOjM?k#C_4s^1CQqOK*ggH}#qc==g5gv!t8@MZ zK-zCPjh5#uA-jVlJC1GaJm)f3ibh+!zqjZCqBNaSzT&5XQ9ufCSWWBrU2XTQdOlUJ z-d3;7T<83{$uUMY1<|-_J$&(fnhddg)IBmL5Cr(7T$3MbPv53|T*VN3LEZElsltT6 zJ!@7^(N^pHe_JYE09C%YxtaqV+YY00`i}XOaqFQbNz#zE7 zo!JDU|LaQ|;8?O;jn`Vu3MJcLJ&XBxExpD3b76AV(cI3p82#oPC~^;fzB4KlDmU^M zNl)CTJdu-Vh}5ccsXKu;27ar;wp9V8dOT-)QYFA8KlMYrA2hfRHtFu~rg9QsK5_&q zCl|ZtSxD_U1;!&ZYTX8zPrQG5UfhUbcuW_wd0RdjuB@Yf#QfH_2G7Q@UHg*BN&XIh?Vq$12C z4i`r6qOuaSI=Ck>xf{zD_GylBFo1YquY9q9en`^`ZrHb_+(>;--aPn)BHCGD;{PSS zk*@fr{QUNNgW8ldM3C~M?P}XXs`mG@OzdX9o<>qNt^DgSCV$kYU;?#;=gZ~VaSLd3 zX8?uydh-Mv#FSiOZ{UWiZsRrR5UTfm@mWs!h59X0PGS@*@wFAL$psq z*U!7jc3Sdf7ilO$_;Ph~{gNQ#@3LkZm$p}*!tz4x^~=4#jq8V`bX|FNYbIJ@nHFV~ zNqmraX>RSX_m2$x7e$@5UA!~T3F=W#)Si|PvNQKwRDGJBNaESAr^jCwz45?if9%Q- z!?{?9%_VT@yP_HbbpGqDPX6;AMBzGKIymZkrF5@9ViAj6s?u%FP0%gJWJ^Wr=iBRZWf0}x)*59MTRvX1428rS4X-QZo`j&-zjps;rHUM-U* zGBz)rLdMr9#sk_)6##+lQwqWZ+HzD{tFlZ&C_ckATj?7UW&i7569kd|iT6QV%nn^F zf5pOLbkB!AdWc(%Z{T2EnKoM3v@EO1%vP-v^xD0`3t{7F-fDaA-7T_aaohZ&1o>X(RWSYcu`kYOC8SoS5Xu8c^VV5^d*Y0yX#PmZ18PmxA{z z;go)qt<`mH)vBC{krlmQrzt#EU-ZUxNGY1VNH%Q;)tv8TDAJG165h^<_pJR!_sn|x zvOY>d=RTxxDpIpq6?pj+Y=cRKsa|O#oAh+0q4LB7?{68uKG#me@-DE_4uItNumhMZ zwO6aMKKvtmz5^XCzoS`E;CsB@1R^NLU3G*7c^w~pp&$#e_D_!jOrA5F12{^~#2>Fi zBTPg%etBOnpbjctzjd)tLh`BLqKsvv4W#Cy3R>TN`+7YZLj)>|nn*}DY7+bA&mfx* zM_&ui>XrAZR9h}s{ZJv%T6dE;EXycNNZ`ui4>xkjR;ooXr5mffD8{Gf^lzYJPGyVg z)64T8bVET*84e?CPH7eF4vyS1Fjyje5v(^;oB_gBiCgv>sy;v-*TYsfyKk=jz{k95 z6=0Yu5B+)phvSdqyoDDIuTwoHgRQomi*tvC0~^4}ToBa#Hkbt6W}EBsK%wH?{rn%t zL5Z@?ni6~1Q7rU!$M6XB^X%HC*7K);3Gj9z?#PsbE-qf@oJE-nUjRQgsS19zjwtn@ znTozOx!}VAt!bvi;t~v#=kZ!_|{T?07lGgwaXjTLCH`247l!>q=e70 zj0fLiR)D6ngkpAtE)N@G-6u@$UsU+w1SYmv$&`qgyAtZ@moKnqN653MVoN+K(+2yR0b(cd;$@dsV8G3{v*TnME2K1iFa%2?be>*r8gQj5!ssQz8lm< zw7dVEA2D|h*ag^J%H{Zl1mzOrxd_O^^GX9hhF&Ni3MoyMT<>Ch?}YJBcj+ZYBOC98 zK=f^hK@=Uq`5JD6N^#1nc|mCgLJCo@BZ3lBnQxhf*}SK7<&1G|4mHS1Eo@9ROm#OgHp4ie65({{qPj&NkwX}{AJWDA4M6w__^^yE~`u*ZsJ2lW-2t$pA#l;Pl}oK6>_y-v$s7!A$9X$|Urq-E*S zNUKcty-UC|B0j?+wkXMe;r{{|-F?@~243(j7ACVcv2+1#$o#Fv2rtwW%6zXMo=b5y zgNA13KK{-J$X7F(?`iU6d|qYEB@z~}QWP}F%J>-D+T>t3TAo>}UKTa$V2$hw#V5v6 zOW7@be_NAQ?f9tNWVs$`cUUBpy&MrMv_*+AqYNHhR9(G}UQH7UCMpP`e-~Zg5Rlzm zp+OMxFOY@{`*g_lxb4w7>wo}%SSd~V0B>`e?6jhg< z3{;Y9et=SU04{$2BIl}u+kKnaAiT#pDz$L5dxyJLrkk9%__)gaO=VUOfH!o`aE(pv zX`Fp-9dOhw$bm~bWAVsf3;VQ5E@S4mMEnwZ`1#v-Xrg?-W4%5Y%@p|st!V43 z;g)dqWQ4|`q15w48aj1}0H(3@A^H;8Lau_p{gt&S4bUHfYEk@S?XVhwQ#PTZZgjf%cn9vGE` zM!YCx+I;vBY-y4bsti2muE7B*Xzt@>X{=j6$%@}pSx`Tk!n9HF8VU1bWL+I-6h&_j zBHVcP2%^IoPY>YZPEsq7RW#=}8oLk^w){tk4+KJdC=Ysn!Yfyyno9K1Sq+cu9s+!% zHM{L9&|pbGbfW8M{u?y79H8@6?!LUatmwO4`-yS=KlNSNZ9o-S z&zB1S26ElYF7G?XUw{jz{fWpcTuyhR5TIbaPn^i@QgPSzyi-WEI{EwzN8Yh)p|F-! zO92S(#v?cQb=B5vfkT*ol2A#rbSBDMgKGaNwYY`awq3cW9iNCXj;;XrJg}ZW*jE-4+cqJ+>lqC=8&c;(VTx=`~LH)zSCY=6cOcWM8N9{?67WiLuNe? zMa(^bD(r2}P>k$FF$cQOI}g-r4-hqSftwA9>c=+gM+f0ro7+w_hsbw7sEAjGsk={6 zx=-^E--IbXeUgAogMY{RJd~zI>TXdLAU!^OVSC(Q!!MoA%Hi!giyk;=6&=G$yu(WC)Py7`R=m}FIM9r3eWr2t z+kP}*HLo9sgW(7RHPWdf%e>0MrjRMsw~&^^qOAUP&8KHD@L7 z9bcmQ{{86i)q_5e?qlrtRvJ85-J|HP!DhQ*Fg7Hor@eG4y_(M%XnyzmgVe|QR5?@9 zHjL{YxLp2HXLCst#*gBs*aq{z>FePcP z35X3R=!ssd+Rl{~Aik_sE2Hp#QUvao%3KfO{HoiEHJ0V<90$K1WtSAd{*Y!Eq1w+p(Bz&}t0O6411PE+?d>&dT)n!5FCIKpI277r!KC z<+#TlRxNA&H?@ch`}rn6I))z!g_IAQKw)W++QIw?kK3@4;RSjmDqgJUYX!Gp>}h=j zMz1_PNV3N0h{L(A;%LPizY+~RGh!-7>e|^vR38{ziLOzHW;u6Aeh^)X=KO|5&fs}P z3f^F{)_k*)iQBvW#`yi*bHf7eQla5<7Ew^GhD!y35xcrgs%n!uNj6?v{CBT+o!=Fw zPX9s4Qit^@N?r9kj>j*3`_8dcz7+eOLLiYmC4np`yM6yxRSW~8EHkN}t1esBbh<%@ z=!_;$1sCJ67=gUeCu8-=3%MwSK`OEV-0j2fM8<7NQ56vbvNB_u?*@9qPiJwy;E?hU zdo`#zgwOaMn=| zDtL;GUe^8j{}}a~;GP;l(DKW!5k#QQs?44hjQb@UYN^O?lYN7NGXf7w*f*h^fR{b1 zi3g#IV%#-Cyf!9c94@VcrePcJvth=GqGM8M+c=|LVCGJqZ-4Jqw&;T@aHc--g;9~2 z;*O7+GGC1p>9S-H3PV1?i^ODwU%RBAYFyRlh`P>6zAPYpij%Q=mhjZCiVvY-R4crm= z&EmpFr5;@C)nhXfaHmc(Sd)GH&rb*YTKEeZGfoo^~mhR$%gR%olhinfD`sT z6<&%>Jaei>WG0&YJ2&^NEtVp~xn8g&s7ZtlkNyLm0LK^oS?fYH;mrEiBi9w;z!L^8 z;=*LF{{GSs`~#n*yWTWZ^S{RfLb$GjXT41GXPi>#KX!@Onjs+WIkMCB6aDL3Y6XBN z&eeIh=GOoJgGHpfQ~iH?0sNo;;Un~~9l{*7RPtP&%i7^G@>jn9u`65}COPuln5$+v zkn($aLlQ;N%S-U&q^zPO%aEIsbd=xCxR5>n_v0!k)ZGhv1 z_y)1;AwdE;b!NLC{Wfv6!j=11`@5SBHXM|=VmLANgvd7anDER=h~^G9rLFJUd5V?gL#c99E-Fu2TYf4%KmT=CUPo`z}!0*Y&wc5u7C;0LOMBp z4kJ>_aLWRQDGQB1ZaEj357!2%7p}I6s7vgW%}saTn*_isV7K<^9UjY^t;cU!jG|9Qc3u~vX5>`l|^)e245mn zTD$(P@fXay`D%Qr!E7;6Y@|L|ZZ)%DR{Y})t`oNbG~s^`BrDbFVDuy_)6ek=u;^?% z#rsQ%T+;$=-07|T$ zixuts+61brQdtRn_-X6a;-p^6#nB3P04`uIqe}ziuF?$DaGTPDy6bc;CwUc4xbKXk zEpW}U1sAzYICsyw>Ln=K%NZxswzx3>dwyQi=F{VxR0KUrYz_4$$DB z3*1LIHo~Dfvxn$=uy4+Zqh_DxKw>L1ZEbbeNH-~A!Ot2)A5BXps+BmHL4!BeFxD|D zG0c_=6fWmJxqB}Zpy(qrm$h|_y(yAA?lLxC!RAsTCQH4-{^co7SK;RKH5IS z3x`gMi4F5+Sy0&ScTw*KjTu?I<_Fz_5#m|_T6#8K(J*d2EE2Mnd3?@MH3*B0$5!z` zm-6k#Dh+r1$XuOzbpfz$BELNZZ+>ndy5JL`MS$pV`^go|@2>R-v2?yz30n`PbBNK8IxF)RYV?~XAwot7- zcusXegV@^zX$OR(bG{$q(>#n*PXqg`Wk1+Kg(8G&E^^C-~C{vh(;mwDNb<@ z?rz21iUfDp;Lby#xVyW%lR&VdE$;3vMM|*(E#(XEbpCu=k>MU?AJ!5T$g5#G=#7m+alnYk8h%;m&i+MyPtF!&kV-1=6Y4# zI)gLLolRB;?0T`6-kiP{SdO6aYw@bozvfzmHh}%VQ9>EhM61z3iDR1kx^ZrQ@$s?T zV(v2Law%MV3eR(K+C)>jd7_hKpCsyqVm_<;2(m7^y$@@#_XM|SBG3YdcM$dvjn3njN#x z=zn+L|JxOYjldL#?`saNxr~n1yb*UBJRK;HWAY{8v=W11JEi|1I%D!v3)LU)8@!w} z3*RRGxwtnq4*`A!I_=%}@BHhwYT$fY;avhuNLlIp#_t8mV@ z-+ThFmwR|o17LEnrvyx?f;mqo-GhXU#flOd5T}fyVS3ACK|+Venb*G>`LMO)M<%Xn zjUnj7CG823>Je_Qqwl`q^%vEvR3-m+t?MnA$TgRyAaNmzJ9(H{A`5=6U)dxC{s3;2uBob4s3qmpidk2B0T?6?!^7J4?}rP zPo;^UVdfd-V1Tuk4u68Tng-#NwcZ`cjGsKKK%&Ydx!S&d;mj*g6xsSs$75Ygm}) z8RGg!An0ye<1eOlHWI%jYDZe=+Y>0VgUuBdAAux-Ph>IZSen8>UJL~#88hyRqn zA)j-bit(vRX!13?>CGO8Jwb2p6wZe$ivNK>+OLf;9q5H063! zy(|cD@V_ZS7vhzzq7PFysEBoIn4HkF@5YfIezzxAGToqX&qSzai(Ot3Ih@^3liRD1Ry$}cbIDyp>11_ zpcA_SkKSx_`!U@NyqkcAJo_;Yh{|zu{+d*F@ajL9kMZ%E>NflPYU{2dxsWw0P(GaO z=J*Fps@jHmxI46y;tg+bfmVhro}J8!?fwI-jQKMebA=;s$sm1d~+!V;C zt?1ub5q1|>OeVf3k9&^(3qcbs-}-sQ_rDd>p@cM%YVFjk2|{5gkt4#d^OZQ$yVAhz z+d?EZ3?X(j?}uowgusiNyTY|k^Fj_v}km zxA{>Q7nIfrvQ#qEm>oW^=22wCJg;{eujM~uMLY7FK(5Uh&WxBTOdWYXh&IQcyaq97@`zlOz{C84Z5WDQ}+rR$r3B)j54la~) z+v4+=g~mcg>qGgerRNY-v3_%C4YN<@{}3k&?ZKED@7GqL<_O#L^0=3`AGLTM^t{+TWb+;2%}N$~YZ_wcrc4lsSPTqNYrCIo3+vY~;q~Gcny&0wJ=D&h`O+;wdFGAMtoqNd2tB6ESvqNjXdTeEZ0yj3+ zNq@AizZr*xtO(;S8^+$dn_-`>pOO!{5k~{c>23sYI(m-SU0%H3^iyY{)9xDShM1PH z!l8!NxchJLQLy7l)M*2AQ(Y-+PzK@t7w78@r@UHS%P|$RL`U01U9CX#b`SqS z&|MkTf4hiOr(zfIowTo0rBtv()YoXS8#cSEXHqoe_8XRX=a=_9scNyxVRQoCu!6Zm z?ihpdM2UZX8j&>>%7i(1F2jdPhKJ&qZq9%MtbsS6>I*QX?P zLgAt*a#!Vl!y13X?QQ?bn2Vlx6NLm|$%N|s1cb=dT^_#A(turEG5XGkxMvqRN%Tsn zedWW>1Eiy7KdguUv*=3#zY-@E?y?&B`3L_IydyMnONhwVKiywheU`yNu=j$V*h+~= z&LMM+upOi=u>7%(&O2%&mGSfT-JJU$z77G9E1{@XBjr5D4yi7szzczRUFRH3nuVPV znhjUGfo2CVN4citH~B#`;nVf>c_tows_9TWAiFMqKneKFNCi7y zw&ZrBU7$W4WQ-y&1!VjzpNi&N{q$<{i z(L!5lzq}hLAZTb^xWm7T=a&tdej&?Zf|@%OQr%@=VKg7`k-Ye6l&FJjhx05WGU2DD z0)@@{x%|aQCv|w$0pJJ?pG_uDW^Pt8fF`dM7%-j*#RYii0c&Xu)@_x=vnmb)AN|Ig88}b)=qJ7*YE9EGu}7 z`~CNdl4{~Wtk+F!H)tun?7Rcy>arPa(ejbT`Et*E8$}!T(4Sbm~s6_+)F~GqQ%^PbeKS zKLEU#q)xT)MP0x`#R8<}GYV}7-oHDHeYLDDmEGx;7~~`QY5MZ@_4jMvSzcevd)o-# zta$JvLfjSg8iPMi*Fy#fk2J^#f8K?lLA20u*L4?662sNueaU(`Gd4TZ;=NbwGR=w$M)t8PisII z>ibQ6aE%x1V3k6QGlvw3By>Cwauy}idWM_Y&b>$|H5sm{BCqAm5220*O$zA?L0UvpJ<6R+KB6z(BPV1)=draTUTnb|S#B8+`nE8=xz{ro z0ii$rXC|Y9Zc=tM&3PQ$mRE?5*SF&(6rwy$`V*!O)!sfBdAD6I9r$(qwte^ln&iy+ z|2Ae41r4XxwX=i)myc`!H!k1*JhUdlvq%%yg37JZhC?s&Cs{qUSkWy|owZfMenTAX zo5BkhGmbYFDXbXEX2hkOqYR`b`S*pJn6}eHKc7y{V5&q7H{+o9m)mLLO$m*v{hn&h4=9Jpr?} zz!T>&myQ&fIxucZI6TDFMF&|F9t&N;PpJOKAaNg8rN)DpJ-5EN2EdgP?@vzA6G@Vh znECIJ_4`Kl(|LK}FPN1$4SIJ59oeg~y9-KB4IppUP(~h8@qU^R1E4H9>=zxn$+piv zbNSiF%}B6f766H+$R*b$)Hq#{ohF&X5mNbhI&|l^gMq+44U?jJf^#qjY3+YvEW}@% zTByZWS}N%`40!kI9}O!Y$X*5gi~IT*2S5)!{x!zt@-;E2;zij1`8K~uc$@*iSnk}dz{?ogsos&TM1f?wazgmsNd#BfbFWCWy->-4{ zB2Tpm-Z55ALsv1k6`qe9icfLy)BNi>AvI{C+8%+fQ>jma^C@&!q<#O;CORPLL97-N6;!W6y_ zOnWvc*7Bo@L1~A=^Emo$DPid_tWw;n*kESB$5&>(P2sbYCXaZ6Nte$1I>F2=y}emQ z?s5#bYw_q!anZOaz7l1ljgMjbD$e6B*po^NA$Va|Te0<@oCwflX24}txo(q`b3#8p z>R(8ct2k2>TWd+xRB`jo+EO~&P-->Owddql#@@EwVos6GIyaIH#}m(XBAE^i#5zeC z1xz}=dphHxuQa1P_M00Ddcgmn2OS4uuPuz!iw50lw{p9}eC*EsNo$oThKcGUC64>e zRaySe`{_=39YOtjp6l~w#QJkMJxG7E`TT@y?1P_O5X9#(l%pLo1fX?c@gV832z1jG zE4_>_h97LB5pKNx!bO+aQVd zP%a zZlB>otvd&BxTeH;Y_lqceh{qpjBvY!n~W6O7W9)UJfmj^rhfo{*D7j_ARAc2@#%7T z@yJt6;nV_`u0iAFD_xa}8Dz<8iEbG=KS%>g%+FgTg`*l~++@1P^;eeNMiVOYXZ~}m z%0o6~xA#(Z^n9Q4K08{n7?v-m?(T1b6m%A@o-FS*jiJ=lqEw~H@wA%&}mgvAAqoQDciE>jgaC+O9?04cz^UcV0Tiu@qf3c zqjT2>l{l3#lq#@uSCjgRm?(qp{;tgaT>c|WKLKt|?|eWA&mQ)oi!jr~9}grUqG7V7 z{q@xJ??+P)+gapErF(m~OYKcVem{=_d#=hDEA647MXtA)?^riig33z0A&|zZ zw%@G9J*v>1?^HEpPPRe;A$OCU%L*&@zc1YoK`FLJYb|?&eqV#RnD?H_9w;>TUIQ=D zWU+>0o@6BxCcDRZtP`}m2xbMe)CN?TSloIQhB62HN6>Nsqh%w~T$H#me{g_>X+uMV zA-?=PGIrgU45}XA-zOy-`CDa?e|2n9=`c~nakx@KY`=AN>_0ltqzyilT9?fQm~ z3J*p>TR&9ulX$N{hHtIv-If(nw8k?ejIu4C<|>lm@Ee=QzIPcSRWp&T1-{qhE;<*@ zm2Ni%s@|l%h0J_$g)C|Cw%ja?h}kqi^aA8$b%VCGQA9Ssxqg3be9a65>f(2cG`JBM z1=O=Wo-U3%q*CSEr)7$D9mSM`<5A>%jFQ;mKo6kX!7*Q$xTk;E++FhrN$F$Y(iFD* zMy9~}D!Af$yWWUcBr$N(_2028%X9vB6ET1b2=l7!!N~p{iTC^K7pP}eX}AXO7kazR zlT7tNb&Nv6f8$MmV=IIU!o}lu%S}7Iy%$72gbP{QZy2he8~I#{Ok|cC@;F0Xq77yz z88{LDG**ez`txKvKYnn=L*0N^Ri|5+!uQoBd1!p&x!% zd*oRte$_g6&&O@qxlce(*WV*+IwZg5kL|`Sh;4r&7?ctF?dL+s4TPT}Zi_(wF!%)X zr}#w@R%Tu{XM~UNzRlPHgM;G(`%>*=_jw&TorET3M4lqc&{hAozbj=f@c7#A`*xR0 zg=inzwWI1^4|3Os((~dftiLH2@!XfTvuJh_RFmg`+&Te^M^FpsL5RlFtua1Cktl(J|~S;(lA7RbNge#dqa>bVIG<{W)4Yqe+r3X2^wdpedY-384hPr4J2*i zJMp)Kov>jgg4Hok7I8lF^p|57xcA6t^6#p2Q4b%1?n987GG50^j1i5e3kp6tiA;A` zj`4A6-VPTXp?PoAT&Sj0IZ7BrUL4_!*#BGi_$G-{O%nkS%iT(KwR2h4%D6WrcNq+_F z;nLu`@w73)0jme{)oqbp* zN9HBLW_h;EQuZU_%=kqts$v-sVharN--5<>upExOX>3^GHu|qv#1gLztf7mTvfF~_#Z#FlZS#l*T zMu`ftLET9Ki)A0?jWrwcitVGxIN*(h0>xAdj#A?mrw5X{892Y|j3AMqhbjphf~IeA z6b?r&`!PG}Ndi3c(22(I(|SHpGb(M{5f>~m56l%r1It5Q4|8<$5QBfO^mtc&OW`Zu z`P<%0@9GkTi6z**HnpkGZzqBjPtoLsL;&))#;YvHsjB;^L}NG%qY&){IkSQ!c%WR7V|CEDU|JHGKyA5fxu{0GVP8qxTYjWaZA3PYUw(UotD;Ls8N_-W;lQP>4W))@wq%I_>#Ikk9|-%TdTIuYC`xP>mTEh9RztG#KE^a}H?1!gAKR(_F} z5EBW0)!D~YPaFe+$NDks{HnG|~dPScHH7D25pB zF)3TS{m)Wk4(rFoNa;MjpKS$sH}`wLSV5wzTZyV$6~P?Ti%i|&uBtn_H^tCPef?Q~VUEw!kKI!sML?C`x6Ws%t;*{wLBCI2Yk>!X zPTmG{xdqfE%l0;^?9cSyLhp^ycTB6O8NqEjqSl7b zZ$ajC{LxhB4RR^vzyg~IbRytbz5`ZA8+^_VDT0o{6U@zaxrwdG%_kgrWqd@15VBz1 ztpa`}D1Lf^5O=^Mb^FX`ec`%l%NAjoRI);(b>P9OSj@S~qd96TFG{ySpE0fcw{O!| zQ)}5C=sW7u3HS}Ve@BF39W~Fb-jOHS0oTSmDQNRs&6yPUO41;MGSg%N`d2ma>r75x zesbr^2wB{Y$K$Qx_O`x98M9m!wNjK5{qGHb*RVzPEog4LqfW|r7c^E6r z1X+VgxZFcEB)fE={Pna0{`i<6}jTmD(xXL`eQ%y&#q{Wsb`2c z#ml-Y@g`X^Gfyrx6lh@YqjmhcQnMZO4fB}CPa|C|M4Y_c*l@Z-4JLre%xM$?7eFl? zxgWP2)!n*(3|=6=g6=ZEp3NL}B{Y~sp#f9y}9sB4$w&hUf34g$2sAwK!j?3g43uAcqy$FexknG>++FY{F%$=+ib+H zLQ_1BncmN<7|0eT7jdXuph?4o^OCJMDk5*Edex!P1N94#z`HiaD?l_;=H#iq~p zeI7r8VNFCN^cduYjjL#>lzqtv6T5^R_73X#w|4OZVXWuOsKtziwpV zdHkjOZzeWCTF{a_Y}-mIabpjneXXQB3N@f`<|&kDtx(<*KR%gx3XxTkUQ_G2O`! zEk!oWk=ZPI%f5q~#me99oJJ>EGayAX<7r!GVPadfdw;lyocE|AuH@jJLlvB&btg^r zjl+E%$MRfC{9;wvk>(HRWYjbQZH}a>>P;jPCSfx0&%ndj#M>|SNS)T;D%p7X@bq#G zPWI`#1{N}DjIs=JYyfm4RF1Gn;N~d3jC-x(t0kDIh?24SC>Q+HqI-%)@~(r0<|}_n z)hy1ao+Dd!GpWpsekbm#vb(+q9al$SuoZUynzG_*LjS`zykTq`ur&QL zb$L4#VB*A@$Njrcut+5B*Q$G z9eFTwWr0vd_())r$TGY6G`cbKg!1*dNa&tor&JY@^{-dAcb&XYuan&BkC|~F*d-L) z2y6)`!zfxPZ0ihh6u;_?C&M7bts^;M4_ybx10Tukx!x2ruXZ*WxB9gbZ8G5%)%<^xdA29VdoQ346H6o_@n;C@v4dmKbj zFUltJ9%D*}!E#@<&WTbwAik(vShu`Cg3bDPoTrRFZ-T?Ez=aNZ@@}2#W&T%KQkg``Z|BATNm#E1j0+Kk>|0*RD{ zyTr*O7-89o6A*?)0wm{3Q!;c3cR+Je*TO4tvw++#P;?6tgMWvrX`Z8#^JV$jZ=7iJ z^x?|OER9=%gtqfONC$gZ?kfjRWCqiLSUmJw{}5B{g~VJ|aDB4M{g${1=!qM3tLEQm zs95(2a2UK@j)|xP^pux?c*4tSufESE*9nZ{NE*>D0wNFaxDq*h{2Uc|$Q3J*Lg|>5 zNm9Rmk4fgWQ*q=dVP~UN9u!=9sEO7+6n1dOSy^^+2q*7ST1Ilp!p`Q z+MWbb#SBj?->uXSn_HYk)L_KfDk&=8z#*Ap|Igr%ELf&^oYyn}&7QV;MHKh0I#n^! zIzRR%-M)gTpnMa(r$cWNp5_8($u6UlVAo=1-m>UW6dCUZIKH&|#olU-3f76UCgQ+G z2&QrIG+*|GW>^AuKpBlo`91htTkV5@iVdZjXzWS|uX5NFXjcEqA> z`mLh@V~o7Ly?SmJ26TTr8z;= zw^^ZpuB2%Q&6u^NhL^iy2gah(1Ld2Qj8@YFeXvaphZiGK^)bJ&pEx3s1nAnmpM`=y z@Qk@4k3XL^BEbXYsz=H(TRt0J_SYq(M9qjMs&=Hx!RGl+vd1%6r2)1f{Y`h~Xv(|{6hZ#UAd|zt~1)X2{gT=+4v@og0*|&5} zRM*-5?F*rI?zG@3*9zpiB&I+Jfl9FfvKmO9(cvaBm0g!i(Z*ptH(W95f(R3F69(!s zf(R5XeFiw2!@_J#k}({4nxFQ|X;~pky_rYz0+Jv7XeO!Pafz*X_CLkucB-9fhDxh< z<#jh{(%QP1x^%rh;hyjY=~!qDk_g*gJ5HeH=W?C+ur;6O4tni{t+Uw8-Lk|I_G^8H zkzV#3sW!^dpw;`OOmvaCNO_|%@x?yuy$HHCb;9U&Z%(edU-x>fwLMHzeDMDG0EwlW zW|rT|zh%qJArt(JW-Dl(sTD8TBDu+Cl|0ylI1d_YADnFRb&yoSgx4Kwa z|2Y+H29&HCnyU@age0@k#-i2EJ}yCq>sf|rTt*i2LyoVW6S36!A!yU+mnft9FP?A- z)G^J{!c&G<4F^}>Rl6-lw;1zLFN#j-Ri9wU2rnZ?0~-vdNvz493rg#L@hdFLo8~=- z#WREc$g$ErnQqB)S)W-?EN>dUu=S;WtA?df#PYereUh!->TsoUGndxqu#i7R&Jj!5 zFxe9|bEsoELIZjj= zC2}%kQ$he;rSc#*QP3>1#w~%;IC`hemlRBO$M-o-tor6(&NLBhf08nD6NgQ}qdF97 z+c-s=OYip#&Co1;;|qkB-o8?%#9zejo&3eF{_aQ!I*=U9-bdGK5&=Dwv{AMz;M!5d zb6O~4_oS>{?Jt3{`_?jQd@Ht-#`ny22>Q-T_9;~d>M^1%J088a$O3o9J>`fUyZKRQ zhTBqpVVT6jb@-AVBXjNo)^h2%;QDIqwEa&$qc^2)bijp{i?u>k8*n%kzZ>o-s*VuZp~ z3QsC4=?J=f0jvw)JBz5H1pT)tTg&Mz6_27oH-&zOFZSQhhRoIR1=q+A{Hly~AD`Z? zLT3K1$>5t@CM(laySCOAG4|PiT|Y)fL;eUIM>kIr=`{iots0NagT~&d72@SZZ@@J5 zwG7adNwZQvR_3Ih2WhjmKOOt(-5;bbF;%J=smg94Hw7#?>e;79Sh$FFt&ia@iQ(87 zsPld?bpFMlMxItL=NL1hI>r;4L!*pSL7&ErG@`*=Vt|%}>5`5JiJ@QKeb6&XS@`sy9)-MUM!cU420vyllQ!anFWu(;Ws;@%e8DF-} zjMa1*U4AiVDcZgy3*$4FV_^gEe&?SK|H-E*Alo94-KkgfN-d#k(Q!zeejV9VwojET zOkUhj)DH ztYN5cbDU!JkMwBjF|1|76)@LTX>*4qy4M?7+K*MB-DK-?U7ciLT{#&^`9l@LFI(QH z{NPRO?BraBn;@etor)54hlf%ZJp1I_B9nXJdZHrD6KjkVT>+G9S;h0SxMCEg8r@ni zWop$=aJ5~w=>;1Ny&U8tVoHUshfemgUf(~|4DuOe#&_^$CpIaC8(SjSZ@am zgrO6&*D1vOyO@cxt5~HJxaa#-+3S!T^n^hq09QmV?_Mf=Tu9t;EW)PAz5SF4Pb7K? zbgf#cM0IG2TRGamCE$EH;|PN|0Cro9^Fx7tem;#Et7>Hp5Y}6)R*!}LMddh#(SL4h z!b>@&#~5Fj!9#>@_S>SErg7Rk_7q~vG$Uz3B~oSGO-`){tHmD#*aeQblR0`<&Gkly zC0zmGGC(A&X1vDiTl%S7SNGQWVsCPfsfcyCl^EPh{JrtK6KaOyE00>0&@3T(6CquyLPbO2~n8sirgvK*7a zcUUN+`Hssm3Gg}A+!vN_G;CoSW=z?y&pNMV;X@>}RTu$@Q>Y~E|55gB07no6g$jGM zrx_60eVJ}9vIJB+1Q93;4<*hEm!jjd`>*Ofc(@e&u<167?FR#)Lx|+RrPohW@;0!0 zg;YCU50bLboCk-6egLKilsNfTYOLea(_-RAe2Un|CXjQO2&DBF$7JXul`EgIp9}P) zWWxmcy(Caao#MVF4dK}S%^q+<+6!x;Jk%7;rxwMlk~6H1M*~ z#HhnQv>$9=do;DVUD3~>Qxc|X&sfK?`Vp**Jv|go`G=77lmJv`7EZCd(OQQEpURhJ zJz_`wFkZl02UdA#=%ks2sJI#|#;$7(){^TDvstpFa)WV@R12xK>XP_>N6#0(MNczv z8&kbZs7F1=4(U~^Fml8`x?W!5#9|9Tc7@Q>TBwz{-P8qj^oHadKrb)VJZvc&~c12V%YDC5+LhGdMU;L6jxih&96N&SDzKF z*-dO9%2xlTW#TGuds38$9PLBDBzPCx<~rK<&&NBygi;zq@Dx^s$7sk%G(iQ#v*T)j zdyKtO%Tp8lHqtG3#~N!;l^+Nn(cJ$GOW4v**DY;Q_M2>jwax5D?Q^;Fo}P`CetoBM zy(+1wZ7m&r=Z|%cLF!3}%lAFyr1`W#E*Q$VU83yq{p_qjvuw(AFaeGVvba2OmA$)< z0*K&X&C0$vn;S~X)b?4>2oV?o(ZUggTs=v7h6SezZvEBciGN3o*GL?TCY%^Kj6;?f zT(ZLFhRCZV*Nq`3KhZ7D9D9*G$pBoMPxA;uYC(Ys@%lN0ThOio1Jp@RwR)>eWID8W zmRB}NoA;YUW4AOXN}%jDV^wmyYlC0GSckmn-WGHl4o!R_quGX(K*I>?40WN6Cd0?)(Vi-h+P zA4@T_Xbe53OG%3*yYI0t-@mLB7^9)XzeNfEmex3LEU^`M-niP5dm)wwJ zM{w)y)#-j;^pHluqr1 zR(~i#`+`=ZO{M)3`JoR00@qxAoIH`P7r@lAt$SHHyGIO>oLdGeKY^5b7 z(8&Dokj;A=-OkS9VcnSWf|k(bWR?-aJj93&TRd&K48xjjD2MY=&3tljI42a1*AUce zqr*yElOmTEGh{^e$;@2y{Pfi3=U@S8>Tg)nil$Yd8UkLQF%StKuUpLEKc$dBH1Jb1 zbLTIK1_FkGdUEQ@l64qIZmfVE`l5x$D3fY}P)yg9q3Hvj-%W5|X?0RQq==;9oY4?| z;+rs7S?(_}A*i)iMA;MA;}#l3{VzCuIMz`Fe3~-TXc(T3?s71z3LsxB^{!onw{A;P)I3;nyE`oC@+2R^LR{tw-lPs!WJFnCM!f#;rgSrI!}@~yq{>rPWu`UgP_i*y*>9pM3>@=TV(LnAq5V}$-Y zXGzv7Srr_8nArX^fu1`$_d2sY(n9^C`E4i*lG5ZC_e1mpPER5GMh>N$`rSoxs>|5q zIuA((P6h1>;`AymRigw!ZPPUgJ7`8de=}y7NQBB(qz)+jo{Od;jRuR0Lj#~ID|N9^PGHC07b_{YMDI9WrF*iq*iqsRYj@&fJ%=BbO@;1HdS#y_DIxODw=bx(@w#pN93~9Ewe} zJ><@z$j!PFpGWaIpbZ#nadGHHXRVzvng?PLZ%o8)y3}s8iEAdx`4?alzqD&FiIWLL zq+Lvy+>m(Fe#Ix@FrY!tC<^I>Z!$U7eq|M*m*R_3Kv{OExCPlg|c0DhF0S{1&_4r?F4`jo}1Vhv9w3OX`%hq z_z7un-PxGzc8ivkQm}-}_qnacEz>lU z67eL+=4$}*og8OC%2~VX&c`Tkr(BSCug6`D097^EG z;4sbo)LP0OTq|t_q>!xU$9brXU^8(4XU4-fTc)VGInFJUf@vi5NOJGC@if4CZW?`1 zj@8ap%8=OHK36d%YuXa&p2^a%$fAsNDCBVyd^uQ0G?9t1;O1+AHwwRwrg$h7r5IyO z2uwA}H3m9Ppxe{p8olRQz{ht%T3WVI>7ao078t%?6Vq?|BFr=A15=XF-mg(e$A~4G zeB{2KFBU$>jt!;@!<+VNJ`PeW<`DWEK&8x@=RMoNL9NVz#k+HFd0R^eVAU!ZtWhw- z*?DwY50_D3Bj@p$JaseyA*2iWmxeV`#PwvP!M$o!Kc7fD+Hryw#^uy@;82Eqv)=w0(4z`4uJG%%q*CgQkbVoJq~YDHfwS;U8qQ zZalE-)OZ($b#d`|Jl6Az(aM}!GwA?s<T<+hTsR{`E8U zj2(s)D$hx7T|aO^+>N953&cR)dq8=bYGI63`Z)f~CF4N|e&M-TT#1)Ah`=*r&IR~+ zZ~y44?%~lD5A50owikEImhO^66j=$Nx3;=w4EXu8BHKv|qIGy+h~`9~ z&k*J@=1T0LYcxL+1B776=+>zVoSQM7BX6#RcfKCqF6V+!C6=vaI%7?=z#ZCPc5E~X zw_tlh8B2l32BX@jQPA}eyaGPoM`_k}+0Tx1Dt=QP4Gpuxbfi}S*3m4#ZIB`WM&!!$ zS+$QT06Zf+lJ@*;-s?F9s>?G)cRQ8LXmk>UW4G1PTdaR?chqg!{ii>!M8;xu7^p#G zg%8m0VzRd-RoG8De$17`qzW|?RN^6MQ!f#w{;n4X$eH2StH}N!?oY;hK1j7>*s)NI z5p54dYvZQ6K%vuB??0Rbxz^}pY^d^GWg8LNYO?;l;i}JbAFUoa`D_XEt!xMw8ISO= z9C>RMK#oFJ_Fo>Sv9FG;yK4Lge^kn&`=3jO8`G9?zOff2L~SF%Nvn~eYE7kWa>2lu z28`D3`)a%p2%Gbdytf8Gb#$ae{pfOUslEVz7|p~+x^r>`!B(K~UJO`n2QRZ6)Ab9~ z$9;`%a3uD`)upol z;hI|2ru?kHgzy@o);nfBl@BG+r+squ;Q1EWrFr>Cb(mMbROS%`*7_$)mCvb>n&k-q zVsl|a+s65aCzDId`mpddI!{WXXgpm3R(f{bpVAJ;{9HCXbzJG!J?`6F)U#>L1T>x? z6G_obJMafipJ*~;;W%oNz^Hnnf+50ITLZEE(yG;AS zlv5m``*ahH6u__kH0#p4$z$LH%$Y8uZb@{8Pr!Dxo$%SX%srb5DhHv!sf9T2e=4r1 zr9EXVq7NnuAm5{x3+IQfFM={U?qWWjrhh7%p(W2&gKH3DCm61Q8hZv6fJyyq)+_eJ zhPwD?E7v^Db88rKJi-l`yWM0~RS(WNJ-fn|$77`$ZlSzT3*h-DM;k>uE>pvyloLI8|9;H$Fn?5iW#HXhJlL+)B zY0^tw3MA*?bo15da?gsx%2YqN;!q;>^(85GUwJ>jFeUk3q;qHzZff_cuz&aJ;n?Hr z%&m&9Yhw7J?6mM$BXI_>?w^Vy@UpeH2p{R=OM6K}e5~)zf4*7kiIvMQymr%m11g_< ztM4E514BDFoHxnDCyr%oMW$k9#gv6>#;wlND^|-6Pm!%gr*ef9g8 zJbD{B=Fb{N|NZJPhMbZ&;Y=DFiOZs7$81IOv;@-sSwoXaj~8xFPyc#`au##yNhU}D z3V+e%lQfm8lyVP3>}OL$q`}9P9ECsNtecI2b>^^yYqDgCeHwPCi*I(L1CTa_7WG|N637@jP3Plgug8tJJ;qR z%mm}>Q%a<5=`rA#4ezv42aFr$(CQ>|&Z3ubTW)~&e>9zCR9sCHt#P-(f(02|Gr*uh zg1a+#@L<6S1Pku&?ykW#xVr|2;O_2l=l$-z|Ib=yrcZbE?yg<+G-E1kMWy*xnkV&w z>oLsLD^2237YKZ~2Ch)!H5HBxacv0WG&F=A72ylS{o5J{9mA5GB(mUj*LF!IJZSf% z*+sC^m>R7DIpwR+E=xzqM5mu&2ZhO)eWTlDrYlEA#ejN&xdjsF;{ll2<*@hb`gvO` z>_dY}wVpXriUh%cl!S}*IOSOKu{6d>3j}jRnmwpR^0zhU)b{k$m}$9L350)104No; zH?$xu|9E-At=P3e{0+E%f^z!C4y|gA_+7W7p@#MgMuN2GG;SFy-n6i-HVvqyodr(W z&JFkR!khkxBlssi%PRbw@(4lzGb`}Z&?L&N3`D(kKv&gqn?SK#y@d%es5sGx5)+@1 z!m{+j*|cd~cuRO5>Jh41Z=orXh?!371x6XeRzwE2H`)={U4)n#F-4JkZc+92UB)5f zo^l3AQe&9H(!DnN<$8Qmt%1s^bc$Y=n}Ike@^=@EoWO19%EQ&N-K-P zJ*1xMNDUOlv$8L`Abdw2Ua6eFoWDZJIK0(}MJc>Jo*{Q5f+_mSV9_obE2ArrJvK{J z@{v#*@&r4XgUhqY5LH^mU&WV5F_M>&VXA7S$C*#&{9>f>$t2|QL3>Ubo`!RtQK*vX z8DHbmOUczV`EPHfVDV{xMggu!hH%m}75~Nw|Jaqp7;LF|Crrpir3)BeS+CO+iV3}N_A^u<&XuhXTskf{@?WW_%(ls zO=b%#wZQCg($QKKf*FkG3CnQipc!!r%qpgiHt@dHJ+D=`58WR`n1h0X`{5(5@-HTi z{P=X4yse+;XYnw*>ic9u$;V^2sSO3%@zNYY#k=w zIfP{#UdBl$vdbGdG1tu2(yLd)5!Fg4%5r6=)EbqT`AA?uSElZ1@$WC=sz^5AqJNZ))oqIW+@7zfpA0(MI zG}^avt>d3M{pCo`F^}|W+)=d(7c(Lg281e0phe~|2~YEeIhqi4`(9i|GKdT3a^UJ6 zGar4Bu4N&lSjK;}_BT!Uy<0B5Ln6vVVN;pYi_K;qhhG+jOfNQCkI$o20Gf#bi!5PM ztXzVr)Vx6*9cL;QZ&t)0SQY&!ODwB@V_+{eQ#ZLZCW&VlUHgEa?(^iF72*)-PXxLI z*7KX+v39wn0AuuSWi;of#Boyq7?*%9-WaS&$M0F6xMwLMdcH9~R1waK%QxP_)EJ;C zA*iy&+N|uaRk-Zpz734ZS^n$QCHao>igGUfIbMB-$OeoLiSHO*`#xv&vF94iVAbb& z1Z>t~LX?iBsk^|PYK$orEV4tRYvGQxy;F)-@y9!W30fT_4eUkL*e?R@UX`H?h)AbM z$j?rN9w^kei{5i|OPB7Z|LZdX3L;^c!FL%rJH&2AjfHHH!xZS$8_1~F!;TokG@McP zVLH0u#a<12bYqzai)47*24>|+jd&vf_zD8<{tYZ4>DRs!1%3`8oBE<^keQ!`ONZchOC8}kcT}8d2%gl*EdKI4Y(_x-ESIJ+~X2N9ymH>p-6HKIt23C zbgajk#4&w?%d{hlQtQ103`1{sGPR(q6BKduK!YBiv_d7UeTN3i)L)!R3lY;;=o8Rr$u|)j26JYuuvaN1T zBb|xQhprj%n%Xly6g%;k=@};>LR2}PER13XuP>iBmZa&6jHRwtnUxOTv85!@wI}7V z^!DY>92kukJM#u6gck!|x(xg-)%0#dLrZ0}F(;~;chCt2`t2c^$u!TnyuG-QJ-j$P z$6L-e@0H|fBx?JGFv&|S09!UwG7E8~bk*^5}^!D2b$CF(C7@#beAwUpeDL+R%Y40IL47eW0p{FNN8xU`uG z58clLsQG_!?S5Y0GIkdjc7*5H1v#`vA*)GLUKdRE zU3K@Ry8^GEo6V33hl1(%(=BC9Q)94yx3t3!Vmwk1KH$nPUa>!ch%`wUlErat58P}h z+@-`!*$#S+B~K-ng#4wGgUkauN<+`T#-c_BYxrIz!8?qeb_mjJrOjdV;&e1qS;W*W zPK;{7*K-n#x?XlV{egzNl{_sd*jTqlWoR(`-`+!)!bYcMa zBlX{jZ{SRbAoR>lr0dz(#i*(;W|1^iF^k8s%Ufo@W3dcwWEv%0ri_h;G)y~|es@a;PID&hRM%9rTS&cP~Pi7Jg z;OF(Hf=u`S>McbC@_%IOWFGJ3iA6}Dc(`SO8PoetG<_WBBK?{VW(c7K4YJ z0%}Y;em8W*#Itg!(jR8RT#_^npvwEg*Gp&xNFB0x(@a@yv3sF0PEJf~3I%Iy{sM?5 zQjkCQt?fkZrDrQX%(BMNH+gfx%)YN~j=@+v-%FW-mtJ}7jQ#>QPUCb1)V00H-4Cc- zAs^AFxuLFU-bSpuCk$=Z5V=fyYh&J06n`9cm9k-W2Yf~uPNI&yL`gW8jmz9BOfm}b z0WLi3+lq{wW~v=4zy)}SF&BqwGDZgh3$D(%C2xI!=2&>FE^DclVcBdc`M}~+uNvIp zj4$hFX?ijVQZ9w+-g#}tWVzIX#7wn9uNu{&zQ|DEzEP{Q=!v4nk2Bw;?4?2Kn*<&r z{ydFYNwfh2nTy=2b)lTcBJ_->K9M}rrafLS!Q7f4ZR{wXISc}v4>n_mfoIbjuk*cb zy~uP?z!Nk6V~aNn4nGN!plJVSYH?@cR-kbA#&FO%tom`UrmFZ$T)&H~0nBF}eh+h= z21i4$k-&tI(ZEq_kuCwm3zjE&f7d1?1Sj~e*niYjvYdaA!PQH*fR1(C@I*Eo{p~4g zF#>Httz&7+Ar6%i6ty{ghqSc6#&-y}BVVWe4@+S$z%@M?d)rRw*`9QP41<&22+YJi zD&dOaDtR<;Gx6W{R0;-#Dmv ztT5Eu%^03q!cYmPDP#)W%o_=V&m47JiZNJTs*%DYk~0f$zo9c@R@uifM0GV{mn12syu|W zTr}VUG3;9&Rku;FmJ>i|)@vkzQ@usCd(*#g3Op1OyH^wEM35sLw27*SwE#rp`E6%F znp4agv-ZIdEe$b5w3mP~;xfpd*8I2RloG1l?kfO%>a45KX7H6Oj=&A8V!p6tRCz=Y z@T5sL+kmA}%$)tk} zce-TUV5aF1q+>V7;IRcR76*k0G4$a~*3KOS(47h`zJkuD#R|T^#&k)9qC{aINIR=* z{M8Z`rK07NL<+j+5!t{r;jFuz&ihii?5T`sr#$yXL4&4{ThYDC{(^lsg=Nb+;;T-T zR58VUxAZD!utmeZTn@|9&Gt5eq*U`(aSRi6MkuiJhDvwh9hBj45J+F1a&57!m?dyH z7Gar(<3c9aLec-QTSeQ|1(HCo!t8Bv+VAtljph?@(C;qJZedAFvxZ|-*-7c(W)EWj zyjv$6A7O%g5YO*W)Yo-aBU7myC?5(j&2xFz-p zTu74{26(q%F)u!tuvs8^TP~%*kxoZZj{qP8IVTYKxd*W&dSQXXxrwO_=SBxaSeA-5 z*MWSyQg#_m#0)+8MbA$r{SwcOGd)H~BuJ8~&f4!iiX9KMKK;FS!m_ zl}f0S?uBe7@sp=*>x*BKVEqU8JDDUL^x;)>n8nf+EJviwmDOt#gwLQd`+(m}MgU+@ zcw>9f$508;pLk7JkCn&;g~=6#Ju{;Wm{zE=Tn4c#e!`!bRP&NqTQr~G;0)D(rGEDg zvgb9vvtmZ$^nOV3`LI{Yof9n?_?sL5yoy?I=${yS9NE%2F6pn2 zEhG!ZMLVo2W+*iPYb2V0!o^*f2RDP(#C~cS2HA=@{hBiK(;@rJO^SkDn~o0De|Q$v zB51Hui;8A|yM!2R$uByw;DK6UzA!&zlW*K)2>n_XSzE*FF!*aCOQ!LIVspl-!h|q* z;$KU}A2b%u@v1V6|~G%+uO z8^Na6;r+WBGu`ij)&-C-&Q3&@ABMH!b=eg8<<4+?3yu4k_p?w-kw{q_sod%io>Q`p zybAz)9UR;k-GxfW1r{OWQ@m86Ign>=5-eoBzye69V$r`{R8y$t3U=BOhQ;Zuml-}O z_tRAPMv=XF%ENZwe{sGXWC)L=pDf7&6~%N$rUd%S0K1l^ug~*(TNlc_*rf1qLpwHz7x^|UO%FgRi{OaR#umzT9iDQ zVT$NV_?w3cJi_|qI!-!}&P={0t5j>N$cnRPgY0z%P?6B66o&hHU1^b}%%fmBjxj4w zBty1GP3iZhs~qg+6cQiAOaOz0bNN{#cHfikg80bc%f7v|rzL9nSKcglyaWw<2xsNz zGZY5;MS>vW2qq1X%%Ls3)(T@EI5%_At3jObOV1}g6mO+*nG{}~-$^0D1nPj(qOP4` zUBNTs(Q5NN9lm>k&s?dBuGq-}%pFqpvcZG7uHz ztV-ID;<&KO_5;BewP+SY9>+#??%M;LMQ~7Q*n3$$Th&x+Zy^lNk0fL=m>JNSNRcNP zenor<1ExAEo2qh13})X(PWPu02nM%@(g^B$st{7MU9#C@-CPHk^RZ`7so;*2o!Zd=1l-d1Gzib zKB&-x*>gCCZi~oRNil$2_T=AJz4>!1r7VHc?beLE0Q{7K5!UMP>WXH}8ZShjVPuff zxBGM6Gs5=4W!Fv{-tdcuAP4UcgkIb8Y5zN#GXk1fk!7cJ{Wa%gE&85#q)$CJFX^^??Y9CP$JKmaFLVrGQC&AZ5WYs|?#d2rQ0#4^=*Bxq#;jltQGSKcFMU+iS-=(i};aXi` zp|egiXbrEWk--Pyba+EX+YT+~I|;e0%_r&7!gX1hvVR50cnXbrbE%x>7*rAsVSTLo zX_Blq7r34ugOSkr9!t<9W#_;+B4K8pEs>?dEbGcD_32AQuC?^Dqxz9RhS)ZO%@kVV zAZ|Ny{QkP-jM7gqCo|x6Mv^!J{&)^t>(6HJH4Sx~`BVDqYnaIl^2J$OG#0;Tf|g5) z+fMw7S1C0Mi(j%bOb#;LKP6^5Rvwo&eA?v0{&G?PHXaMsc6Bw7Dhu9=J=7!(HU zlOLbQ8px)q(8fhkto*F2QmmNicf;r_T+c$Ej>;MODUs}>6cvCoMU98ihUa|KXU%Ot zQ~&*`T%OqP1H)@!SR+2}7=>6!HWtxuBKwe#HmAVkq=&4fia(SWw3_+oNVBv1Cd3Xo z%FH0ag)GEkYFh@}s?J7kp41`3T2TSru$VIZUFv2LyHAumd|lu{{O3Oc7r4jdY~FG+ zq=t|~sgR)3$(bNCQQZiN{V+*h=7&DM3Kh0>wL~fv$DbRUv?#*}jL2^--xrBGept_U zsrOT<#xm6(Sbjw{sD&ZUgD@T;N^6h}^ideaLYOV#eBe82$Cz2EEfXlO!?<} zqUpn6>P6;pGQvO>K$yRAwJ5D0O%N&cZNn}IQi5s6#rtxJNreiS0+>P*^ty&+E!-M( z2fRHnY-onAs2tWd1w9p_#88VimqDC9mz$WZV(loVV7wPi`4P~l@+hpx9!t2MUo5AM zkY^t?xrmJoO%{sX=C$^IS!IaYa<@ufvBnnpI4>@VL2TTE<9$#Pyfn-n&Y2Jkauy|7 z&=V(K*%aF-**8W9U;_zUzCRVr6bi+8iR#S*USER!R9?WR+_`}O`dv|lo(1~{pub)2 zPhVpv<9|N;!VXztOrkwQnel?+D0DOp0!_s;br!Ded+e9EJ<2jTL<7iruc5T6hGjlKv*8lpsSS*_5vnAMUKrYz@_Y zD=j0tNPN=?eZ!9rY&~B6eqa&&f#r#smH*pki#wQlOC&M3LP&x0ftRS?42m;Z0N#G8jIJKhVjouwsc5KxVht$r@9wd? zPpgt@e=as*PwGx18rGumE68}eOi8g#*sA^lXR4SeK#1bZ_>Y!L@m+D4dgZ%_#DHw+ zGmSz;z2^K~J&d!M{f;e$h$lDrb&;ANKx9hxjv~ zeiq7vM{gUGD*mbAK9YkAD+U*YuiZCeL)T^nKZTe{48x#(q&HJ`klQWi+-|*kWUOJu zSfF)S>iGJ-D}&dehwclf%n~hpII2zvK2v@k%!!f1>!!eV zsp*iEuD;GyrGmZ4#j=9;<@%r=84=#-KwfAt1qRKYIyA2R?i+TRApPf;BWsP=eQ)rh zo~{lmn<+&JW%D`U?b2Z#gjFQOP*I<5UBTGMo6jsYC*+{LCxi%t|Wd9Z+#5SQA$~dIKgyy5woJdJ*>42kRWyScuob3{jV#Y zh<(RW(Lp*g1U@p(fP7>z+Qz8D@aiY}|ku?O*dYtMdxC%<`8xN$#n5v%DQ!)(qk#8TrBX+Har(MyLUJ})%t zqy|IzjYYNFHu{pU0+#1HW(Axr3mCR+T0C+QiguOcuLn|UTTPB@!HUvG7V-P`661D6 z%=7$w03z1Ja;_tL8i-7&4;6izAEok(4E^L|Xwn#r;{%Bt_i@khXJyY{5ScqXBM{@c^R*yC$8@I*R zXm*R2QbT}!O$Cv$*eX@eP&GG5c2#BpFeT(lG9iqLzCYmvbNAQQvo&n<*dwT)N!6{X{VsRCiEHYwg$MK>ZVzEut%a&O9B zAF6^&8qC?TvW;ss_Ah={p4C=QN4^Ok;^sH3VfS^TXybwaRno&Vf(<5s+%PkFOR`1Q zxM2TMX}z8ZEm;*v`QXhETsdnrD(BdXio7RdptiB}iW(f~vQ7P6_zlr#LBR$tNyK07 zJ&ve>eb)iUqlj#yQk}-1oge$e5Xb7;K#|iM7_wrD6T84~2zdy(qp;K%qixZ7w1+Dw zW8$M51Wu_tvs#!R^5~hek2`ie7LPE0>MW2pUv}9SceIwL?&1JcqBN4}JmLwo zrw`` z@6|TfqZc(_ntd-xVd0yWqfc&FQk4_v(5rcY=1814g4vb7)+q`!&9g!M#jXcS(^fZs zfk`gOr)5lkQsu%PK$~hr<|bz4ZFKqQX{!!}O$WcaL}P@n0OHbe@1YCDeVhd@0+R?6 z^$cwtIp!IY*#e>Dm|?o<*h&Dmblydn0e2XPwiK9^G9&LJPjCUfl2&MEFWGoLh}rN^oP>|Drj0`rdjq4pnuu(XT29>8#Qr5yP8QLRHM=!*E9CQ@R;${3r zr8#C^Ht+}v1C+UR;m=~L7n@ z>@j-0t(J|~(jP@o&kfrx`WEUiFg(G6Wl=c2klMt8 z^D!F>HH8_AV8TM8n#PXkq`6qxl6Mjv@#Ty)%Rt$h?j`nN`b_%C2se8_oGF{#BU2*x z-H1ly*+jU4@)&<7nKGoXSuZ80d04`BIIoA21a6ecgbtX{=%Vils#1hA(4#- z$5Nm%jP6}g^Rx%!UTS%Bek!p-FP7(%2^QtvP1JJfh(*6ei!tFWo=}9I%1C?HbQz5w z0G*SW!fCh3nTB~ zSlS@oA>}V_YMhF`%`aIdF)R9S#$-gEw*H06Y>YuwT0$uypy|&r+PS$p{YB{Xrt9z9 z>4Y?@qawUk$&%AeyjaURT-HMt8=Iz+aTsEC z4-V!79z+yzd-?Fn30j8N<-0TuJzx`OqTq#%CT+<>+((=w^{p84+AS_G0f6e5U9Z91 z{OMQh3VEDMC!}ophpSO2x-}eIek&|c9U^-LRfk?~4ObzC1ls`-QNptLy`%K_>@qSk zrJ`VC6X^XG&`ydJPEBYTXBv`=?kxYP3ia_TLyOM(6T?ZPq8G?l-`+XoVBE-({WTlD z9)0KGejiXd_7Pyg4R(6en^hSv9edi;M0)TtJmeJZ2BX8L*Z_hFbnmfT;B7m8P47G5 zk|NqIq}%_jKiI=Jm#JoHICC7B4D@6Ay*;NX zb`Pb*B%3!+&wZYHDZKPd#9Mw|n=RUTZoF0!?(=8gJ^}Q^qSkud$+4$)AUjt8(C(8% zw(#!uBt;5;ni1P4M{%o`qw|&rlMJEOyQGy#Hr^sWqzk~M{VK29m8n|&dr|kkR~EG* zc|PN^z*LPnp{@5-*QWw`bRiKAm5tt;2~VAN+(BBnH@)B4s(0TF3=keLGZA|Z7uHSZ z>XP6M9!ToBMUkrUI;M}SIG!W@&8I*bg@wm?pfy(fIC|p{4Txr|DAO|ZXzf^8-G}`v z0rl4@an;wCn|+;jA+YER6=^sMT%Ddr;X(R3&}B`EbW$JD!N2z{t$AJjC@TGY%*r9M ziu<*TDi)@45O51h_z=(_& zYw6L~jK#g?cSGrE%%b3Fwie<1FTE}Yf-kt|DBMoYhiKF}%<0S+{u50KjG7Ey2cWk< zEfbG#bbfm&Fr;I_Xa9J+C=}h1eJ)1bz{HOr+!c9LMgafwfXk6NlHzIXwsKA zO#%UQG1@wsb{l<*nj9KBM`hFCSCqggqKDJ`2uyUto0=4Fe|Tk5MFb?*zQE5 zGkUj&c<}r^DD#Z$IwkhK4cpuk zPYq6(q0ZyG9@pD+rr&B}pK!&c_lomeq+K=EX?%TA)WOs10BI){q2rvV1&_e&t@D(} zSBSlQM(GZXX0p<(bAwZF;&>MbUOG+f4~IYl&ua8smvf}gq-(qLy=Lp5{@FUo(%mkl zlBdaA_l}G7cjx*5%bAc3tgTjl=aWR#?)g1oMop~49)hlhG41XQnUE2nfu|-f! zUU8tBwNHOE_l!VjFz?CWaANZrVCtN|sj}dVyO69ZYOk()^#MQQuA6s3P89qPHa%QTTSMtlM?#eP?zbk2KmjG*=GuB&3)WQAtDG0-Wldl2N$j~a0Ki*h!F{- zQ0+?UoJKiim{dQ#*(2O^=-YD7*PTiI5Zp5An=X3zG|q=v$tN}Sjmn0bP~Yk}s>`lG2k0=E@K47H~D>01sR0#m;( zUa3Z3o~0UcSYIaFJSGj#F3Ze@+tXC04W6ZT`1JH3d3aKc2b5Awq&ze7#VIbd2^WbB zzF9KJ2fh3~2s9j!_^tu8ir?1QTj&(=O1S1$tqAkcsZu9EUN~Ydp?sp4`l56H2gN#qJ)$d8Sl5>jAMWChhS}mAu*bY{+I)JMj->UU=PJaWM z1F7wI8ubQg=M|E#=qiTCHKxnV;xM@lJ;yOzv`)u~g?5md9I!@-UFxzlw`DpQ8!ZS{ zQb;?Sd7=$iVQrJ!hfIRGLldyazgQN;T~X|$-ahz7C^0M zywzd-&o)RP7N{Xm9WYJZA|_CJslkeMVGs0!`~la%yO@T%=?f_;t|T|~k}ZzY&^0>G zpG9f$$Cqaek~IZu0kJb}Q%jt;<;TLxtfzKyC$b{p7+8!(q*fFtN$3S-w>NRNUBovE zHFxxp7xA@EJf8}nDa^Rq$g=qoVn*Pu2FC)HD!q&JMrp@=zHYQw)+9D(fq zm_(ccdA{uZ0AWIeCiG>6>&7o!8ATtCL?C!hZzm|0A9*|*oHfK7KBuT<%x{WoLL7j~ zHJ+uO$kz3zlwHy?AqTJwdF5eC)!j?J#3B$Qq-uJ0erjAPxxP^Om}?^?TB#UqF%oPy zp|t`_UiRsXPr?-%JJG1`)nr=#eS=>>XzV4ct{P8KP#rIDs%Euis--$#}yy9Y^`H9CK2b2fEHGdSepO!A;{(zSOM|53w zRmkm!KaZ$cxKrZWAi9-UEiTU%Wbu^}RzBm{@8fjISN7yg$G^)S#NObfeON0F#jkqu zQzRu;9arrUjil;{a;sl1<*WOF*wd>aA^BxG5Z?y$FE$$~yts-o`|}ra5t;8%xT@+hHw9MoH($O9 z&B{~p@T%e2-)*0p7s%vm9)n&I)K>5`;|WT~V|3&j5X8zv6YwTsTYh|88SRTyY)iEI zn8)^KTCytn?Xt1B;@>pl%>to#$`c|Uz0U726j3W_$^o7qCKytlaM8=Ytf)k&L=mp= zboqt^mqr{3EB;khJ0iXp9EY#F|seivAC8f1bNF)|BWa;YJS0D6d{tD3Xse1;TFq2)6YMN zBDp|+b4Hi?EH*KiztNzD(aF&RBU-VH&?rg{Y|*bt*KZ~$2-v=(1jpl!5;mW( zo|gjgaki+sRck5UtRCwDrF~EX8TAMHjOw z8$I*sLS6c1ZL^avjcO15Hx?=jNi_x~QW9SSSz#sU%(@1>rBwm6po72R64m9fJVH_G zRr{h9Q^63|Z>p?;m6VdF%39Dp>rx8AEau<;_U_@zIvJ?gVL=x-j-T02h z55(ZN5hEL?JKs;_p64IW*v=@jjLJ}tWL;Mpj@U;9pr+6ArDI zF!St}%po%e=1`lF=m{M<=Pl1DgkXzR47u^LF%s7ETBI13WgV4fm6^FjWVl&_)W@aR z50^5{0iF9i;i5I9ZXhgklrVFCz*C6k^W}XV+@u&EHls*F@L&Oj6M5_5xFL+ywKzGE zuP~&Pe~WL@$<;3>xsdwou32-?67HRJ?lur8dM|^i1582UBYLT?3Gy$Lbs+T_uk{nt zAkbc>8Dnp!hyBu$&;q(4vL-K2FuV(4EJKr_;m>2b!2zqq^>7*dznMlvu|d%a zvo5I~he^j(=W}#g`QS1|coXI5DSd!uj18T*u2mCNQ!BE?sqp;iZd>CJ7ao;msf(po zJfx4j7uS*o=wxEWR%MV z4t>hd{}T$LUhKvcK&pDqqg7pON2ju7l~rF8JJ(9pxG8FlOm#|+`954<38#v#x5@MKp~m!9TH4$?34h_ zN)0HR0QMsLiZPo5MrV2Q-SQ)2O4jbdtN8Dv>|W0eDkpgh5?bi!|5cjLFPOU-g0|}z zDx4-Xh*GQ23P?tl^Jfw=6wPD52fHJr>djf%E&#>51^&|?arz}ckFbaW2qj851d62a ze-Y#jt5%o#RhiF9ius|)BvwaM3JQj%n%yiG#%vIYmT;w-aHq~HS)JHozIAG}*8_Z$ z@meAd8?6457F=*mG6+$xD*M$>rI0?y8I^gGUcAvcqerLmWrD>exrSC|AmqLk%YpIuw@NA-!8abe`5zNV|+T2`HSc+L_I54Zdi+0Lg95tBtZ9hu6o-Y(bKng*Yk--2_isyI0d zd-GPvGSkE^Fow9xiX9CG1vB;Y0uTTXFoJ+cv<#G&zd_iH>yyMU?2*bWW%=|`8MxGF zRDok|L0alugaoA7-MHHSGyl$`tTqA`f1*3CNbZb~{u7W)ef zO$+%5*O~XTJft}N{;U^V74FR1@T=7FjxCJyA2<~i!dteih$?;Aoj?-C`v3|=cWCHP zQIvVa(JbGw*dkTAa*6TNz}OBa$ZNKtZsj8p;cqH|+DGF1`{_l_h+b zDa#Yy5|)yb6Pu0U@ENgS-}o?Gb`C+SFxBg7z+W5ESa z3OB7Z)@rSB_--}vP*p;q!;g*_~z`=eB7 zB?67D$G3$IyG2Gn;bpVz`3u;j#+diO9FI`b_vX+7Rog)naXSK9Bt2D+|E&RN@!JYY zg@GVh^|3A%6!-Ujbs6m~@Pue^foO0GF=$Yjgys5e>>-gu&@R#b(Z~5Uz4I|0+EBwg z_@^uK=b)Hg;rYHm^bcNl=H7Sx`X~MRXS@1}@ZOsryf*)Xks9nm^mivdpZ}G2=&ES{ z548H=FZ|%|{Z#Dzqyz;>PF@R4UZYiApjF+wR~2jE`u^_`Uy27`)Co?3Ud}cs?(1+) zazJID$7`EM73iIs?3SABb(n1SYIRG-)rPZ#?{2pHA0YPd_V&M@_KexWy4~bC+7vRE z-Zq@>T2Jbnv0!R)9_)1x>;+}jUjGMx%sxBJI_nF?+j?x9d~Qo9S(bd=%6cf_auN3a z!O+CF?m||Xy8LkO{eJ)HM)cE7Sk*(=WjEH(^;K8qA@5sF@7v}Y)$N2dV(cXf$0rUbQ2shyl;hc581M>x|?HA-u1wp`oV z%~m6Ya}3+_f7CRJ=NgLH-%YOJO|DQh=FtGEmbAP51{IX$>-`q}b`Y)qI%lwEETSD4 zJ@Nm00o*-zoIh7A1SJiw!uU{tY@n}re{mANcfuCDiD78(CT;7^dW(UpW_yPcUORk^ zvM)%gE=eZePA0RMB((vjcF&D=Ca?WaVzCVdx9wLeIA(CO?hEwUx0DL4>3=8V|1d;6 zmGYUL{$K2eAlP0q<1&ksCw$xJdR@+w&S*@+bF-rGybvkR>>og<(4{B`*gp_X7r<@9 zZn)j{@2>f$YGq?_eiE>bbPfPqA^e5q1`y|T)8kXzC9+|~5@_CxjjA?lph&#i+c zkJb~FJTpElL7gj#3Up162O|eZ>G1N6Of}Fy=0W&11$9ikwn5_Sjww(}(F^*ms>a7x zI!;$QUPe0wb{0Ix7Z$5NU;mKc&=9)j6rRN#YBufNvA->Wnr-DecI5Pul^;HMUw-IE z3lg4z{Vg3~uhE>liN$R`ne}Tp*;B@Ao{W`MPa@wXk&^^Z%kRfYZ1D&seF#=k7AoQ! znA0aC4s5OyRkwr9A!gYnuiGH=jVJK^&hGs#5Q_b<`q*;$A2#HY{3wlA)K+Q4aGlV0dDH2cmnTk>wS^JTOyvE0Hh&UmwO-Qfy>k~T!V zEKaj9Ne5D@vo#y@po%8BRVI)rx$z%z@-w)jPp&OMja`T_)Qk_Ksxcxw1L{#?8V{u1 zkC$~T+mQ=q-jYWfK$w^*F&+~slQ>7QBYP{L2k}of{6$<%{@UYDMHL_8j>%_k2M4|8;TySi70j0dt7Bopg7M|jHgWuIR7lr)@ zPuC5G^B;El!bTi}WqBq=`H@mENa-4^1V1~3AYHj7FcISxCtCm6m`_YG@xL{g9BS1d zJ!hd7@z2%8k^v6GK#9Eo?b;Ih|2vP$J7fuX5Gd%!pD*Wn^n|)!paCN@nE5!vZU3j& zzNA^u4!9cS;v*FCS!8>!V1a5t>UJRitet(UjWub=#W`!#g&*v%4M@3)%#+(D$<#;nRI)xZ2lsjH;pma}^qu**Q=yIhlGFXa%&jDxYr zC`idyV4pplZ0B7f*lfe(li!wc>25k!1`TpGbK5_msK@H{o z*2Lil@*YAfXkTaO1rKG|GPBPgv!a7M&f7Krw2cjED86-5y!ehz{SVlB>`R@B5V8R5 zPm^CISQ+#j8+4847^nX>-4xcs%${fHBQb`v6=d-TI4tNctupGp0w3?`+HkgpaeK)( zx$wX1BdEr+L13HMXwrYKL?HR*R;~WGH7OpxzJ7OVM1+)9RklJ*vA{7rvNGZx)dZht zeK|fsKty_ZY8wKev5(RLUN6or`?yV=qXCA@FtZS3R&`~DZ2;6fxD31>WrbC zkX31V&xrubCZ4j2o9u_s><9j;YyN-!-$Ja*a8y-MYD9~fj;{oNk?w}f7AtiQdbABf z3lFSRAUAS9jAv4D1H^@~XR#|}!<0V*L3f0_uw_6woXW;B0h)d?EJ0qw0{f}DLjPf& z+Onw&RS=L-MyUmx=fFWYFT~!0e%Dx(9ki$E*{@0DwFJ3cu6kYmQF}ae2_436uI|ba zsM{c8v$RJgonAAGv&s2x*amt1_Uk&NzI~uCSA>Aa`ZjMjY};sJ0~E$cWo13ocS%qP zeIrL0vc7|8mr6<53}QcVImYB!oM3#>SQR67<++Y7^fsTysqKie1Cvzw zFMe*?uObvTBER?5-EQ|E(3Jf1*705MIK}nCG#Wl5YPDwKV(rI~(fMQ`&F@-BLo8A* zQgE9jEUioXfL7IKdmwhd1cO!Y0)uSwjAg^-PI)w>dn6kD4vX^X+fd(3ftC)iEli=W_bYe z!{Kd1;VInOFRM_bmGE<&@F@7g>!rYDAAy)v^ZNi>5XPfprA7%vf^U5)W&Hwk{X)Xv zMq(}cOBU2O=4x-LtII$p)`h@aq|FL&FDNbBbQqWdb9Wd+>TLp=LMihPHOm7( zlAk;1@E^MvQ8(Z-&fm_?2ObNQo(i5wyvpsai))Rpf|2SG_@X(xnqw9DE)QOaZ9@7& zDF-$$JlTYd;fd0sHX}p){=h>nExN5-oh=)K4|}5I;1<6 zTDrSCmJVr2mk^e2q(mB)?vQR+y1N_ck`|DZkdhG4m-~5+-}4^xCwz}x zYJ#&ae3IAU3vj_UPlWqV5qGo@hL4y+fBh`-)PKcc-#7oBj4Iz4?LU+im!d)`4DtHy z*#jxeT7RzRqhlv(i6;#P5*+44WiZX0fFq`I4?n}$zIMJLA`586^F^sI`>(ov?KWoc z+duSyc)yxhae(=^vR@*5uesVF&1E2{GRCyGkMuRYw58x+Lv#N_$Zc)cg^s#tyq8&; zm)M%o69SLuqCGNkWS(oJU`U7(W#+D=n`z)rufBVI1#as^u1OF{qe? zIbUVD)80s1J)7-eR|(3^gdGhquMrXBOQPO`&n?>s+h!IEg`*qv?T zLyl(i4V$UPt^~PX{w)pExPYX2*y=vDINB{QIv7a>pV1 z?1}xn(f4Aa3tG4Rhp=A)4_6|ZfO@l58^oh(@|ZwGxo7|7(f3qiGCTEm5$?C6sr&vR zx3^)^jJ7eP{BW~O~4lh*nTM+h}C`}J)C>K_%0KYk-l zV3I5rM1`3g(H|+;eIie<^=|+#jlTai#^L2f4sGs(W0qnxUA)n{Q*~ZV9Ke`?BarGu zo=#_B?6vRcJ9OwRi(&YSck3Cj_3cw`7XgF)M|>#v^s!6(TmBEFVH!5mSsF|8d5Tt$rdRL8@L zw9#4n#ZaxV!2<}OpXM|gkgSaiq@n$c$X^nznItUwPG{GGLH%nOrIzq}F7Mw&k`K52 zL@bTYk^;liWe)iIE2QAQA~2G?O@#;S!=#!k2YmVrBd)q1vvr zvb*pxa+2Yvvoxjy6bbd_;_ch#Ck8|^JWvb z6|`f1Xb!V~m~?QICPpX->+m+SYOpA{n_O(qNTy(!Ei&dTO6UB`Qgi@Sz^;o1>xd@$ zP(X-c_+_x&S> z%AEqpSm6*N3p_wEkl(}I*3thHwy6J|;Z33!_-5fr<}9M6AI z2v*c2Ui`sX5|+|R5i^v}=u^Vupr+v5B*&18qX*Dbjn@2%a|sV9((+IpHN_t0@Um51 zAy%o8QM)9VTJR)UUg;Nb+@;i1))9eLg6Vj=#6jdXO6Dv)x>5>B7+6DLrb))WIw6%J zICF{2;cp_OCEa#~IX9WcZ~NcxIsHNG14*sf9~cX0aRD?T9q{{r+53|7|5~W6>E-B` z2SMA}ob~mC_4O8X$-o(z1fjs!2FPELH|r2q6sPdxt#t3jX>aX(yb*u?V^;oJ1|s+K z>(LkGL5?2SKoch6AriYcSK0HxG5CXUf~LjJU_X{^Bp?VV2?2BDR~8|(OYtGturIQz z0BsL%{;Mo`dUiyXBe}Wric=1)jWYO#CGDWk^RsQYOC~vN9PsNn69ArD9Xmli5aR{{yV&y*;qRbB{!9n?3h6Rf}wZJ91KU!(bTA!nVh(WTu^~_~NYh^E@HVT6ik+yBJyrWe~h^P?(iH z(_-Jj_GLzBE<(#o~OKGJmWHZaEAO039#`T%!??%7T7BDet*8|c=b$es8pYc^l}qJ8LF z-r_syu7g$&E19RQQ-kj`Btt+^96KRDs&^Z9o7+6`(fd5Fw|4JUG7*(%u zXN<(NpT>V3#&C`IQ?(E?|@geI0vDhyEj&ie)z_^{K2#bP%lVP+Q zAn&>8&$?1~3>h^TsiYbAI;2y+Z2>}GVbDi=M*sQHHZVdChb%uc@M-5)A2dU0gN3g< zPTfA&_E|ZbVB-_mzDJp3hmbJeY))b%qG#`7Sa{_W2Zs`C-SYSgVJvU4+*ku**)BES z)$P(;C@V84mV(hh5%M0FH_|H1RbI`%xY^0i- z?x+%j8>6uBt){~S!MEc;cE9{;!(3*~oPIa-t3s)G2ieE=@@S-`FCG$w)HGTZD{iwa z_B;TFqVHK{o8Pz)kDDiEpV2BBG@6Q#WklO^!rI#$ z`@f10egC@FV0M^{FW_i1jl~gE%2~S=MS8sc$ouC)Q+|}OEj*9TjWN}n5 zv9%CS4pb=9*C9`q>eD`ip;4%E}(6%Ws$=fl0Zk_eW5zi=K%0A3|K}95ia5 z*G;>2g~a)-fLA(KEy29F@r&7O6A69UJ1lh|%`bc$5)%v)S(MltzPC$%sAywCUPBt)3@|Zc0z4yd)YxICD5i^Wef?!1Fzn2d{ z50@Lokm-rhygWpvnFiECgUwf!1*`Hp6GA{VYFcXA=65&EIa*=?$(--UR%OAp3%nE| z<3uN;D$#~xYC}?Ran}yA*(zTdVZ+VyR%-lG(|emYP!3qE1tc+(+y|Pl4+|7=P8=`)d{iOW|%HJn!U=!G`PsYNS>`}u+;FePED0Jts@hEz7-u{j_Sm@cwLd#uoJ`p@kMROMwF!g z%sV?72k+|{9rn;)f9NV=>k?M*+33Q3OvH6dMjlF9Je=)*6n9#&6L;y$Ei-Mrsh`$4 z^}Kifb?2O}x8FNosz%T#R<^dx0+LCVcMp9pBbG7E*`{vua z#33QU>P%k1xHQ$E#$&mX+0fvZ2@L<2=c{~fk^QEMhgbQxxAmi-?EK22aNC|1#=jE_ z%}AXMnvNYfFU*eezB2z$<=rVyVkavNkPQ;H_1&kKC-Pj?h^`_7L(HgjEWhQWjkiz) zgC(IsieS1LV{6Irx|hChk4z3&RNkFZRi0?K3e%8buqwUEp7o>4!#vH)eC(lgCg3jP z!~BEi;FAaSXwcWnk~5>1AjI=uzu+xDIQae|(LZU}_Ou?#)ew z?f&e4&+oBun8{h3iY_g>_^%*|wv72pdy)0KKSm;C*ngqy;iwVPFr_roK0gUG2^0&{ zy?XYfktAEckF~036TiP4RV|uou&EwCI&(i#|HY9eE;BES}gBD0z-zW0J zt|;z7DR5tDJtRBoxnQf1{T_L`z;XmOQ*6ON`fP7@L!P22RYn(iIaQ>UD|RqN60sGL zGBly|X-UcJ8G=I-6op=*+w6rGNvzKZefOp1&kw;XDlDMjv`{pU9;{obKf%Ei)5kUT zQf$20=3+s;cni7cytWxR*!>6(+<1Ju^qxx8Re3~MA$Y*`!3OWA0=Q8CGWnuq9XPXt zm**Y=It=#8IYk(r*J+`a`9VlGsEFbQf8J6!5{qP1mnPFzD-UyJSqAl}gzjWs36Q^^ zA)H3$XY9PIrpCVnZM93E5u%J~fdHzqaFx z_uWR?I8&mODBBEQr*Hf0rQ>fm+91)9TPY;@{4Mkg4S?T6+?_|Km+IK2&=`W-)tcQK zjTGy4u&_AyyPOe0=WErN#3An>BJGC}{VyNbGo>x6DpW7A19gsOa>f1cIOL&yFdm<| zi=}Fm+F%xeW(oWi#J(@{f_o>koF+{QiklVJvC4@n;$(^z&#{AB0wtjQ<%@~2P5fZi z>#?F8l7f@Lc6Su|p!=vzR%)9Lx<*YnM^b>n8adm1`soaTE>cop9I4 zmk*mI&^Z>$m631#DDMD_r+&=T6F64A*-|uT} zd%?rEQGy%x9}lRd@ON+@f^aXtW4R^dr*1^}de6mQRUf11n+Q@aemUW-H{_YQ=@dY_s#j8`<7k{M0mzy=vy8BZnq9`@r`Zm|Zpw{sxT z6dv<{!q9D1!npgkZbAWG|0bANSVRUnGqU&oN?=WI)e9ZHqzd>sn?${jN45rS^NCF7 z@|H3`xhTm60fly%6RQlFWWW=wb1UyI?}8(fgyB6fXCYGd&SO7SSsgOT2qFLlj_f+_ zI<)?`?n4p0A%@s?MhxrNCIGppL`l6fTuk&ijV$u2XsQ5rKH-}Pi$k(;s|ptFsxs0l zf7Fij9wZ_ubPIXz{q7xfPGI9Xt@u3)XEK6Pg}PPOW#V{kpT8@A;*S=1w;YA1FoY+| zV?k(8({1o-WxR;o#2>m(J^eQ*L^x&pY6bMm9OzkDx`?x&xM?&Fi)8u@$epJ>5 zhZ-tMG|)Anbhz$@Oh@KQE_Nf*$Gi^e>CpY@Nha2JTGv1qEzB8yDZgChV7X%Qiyx8i z)d4!nttDrF+`2wBIYs7H&}TVCb|IOJ29l+*i3@R?%xQ*lmFJK*XIKeIiZ#=VO^4%8 zz0dS0^WwA|vcE4@6&9`C2oRTCuRCTETT2eYP8goREUU|kx>=FkO4t0xf8>>O-gVPQ zo7m{|AwIPxjoT0C-2#*IZ`e+;^L9ug{5JWb!FCh}#!)(`4QX(5}Gm9{U-7 z8_#>VXbF;JC2Bl?6o1WOH>Hn;vrS0-zUcb30t*TH1;djM*u5st*dX~s2`Kvco3;z9 z!xHZw;sqXfvjQ@b7CHYl-GTebpS$5H6dekz`|>jKmdAy(L$)cjxh+8Najo_4&*m1Z z-1qq%Hz|+9Sf>6{Rmg3iiWA_;ogPnPSlk3DXqM3S9M0*f!)Wj&6!o%RVdoUFUMz%- z0=R??NT^ThHz6AXSk=+fNX0`F-79Rq?Wi<5iMDf2#3tfG zXTG3YkA$h{hcLYh{a!NMSz(Xg<$NnaWTPD7Nm2F8y5npv@vAdgO<;#qOOehkO5kiQ zdc3V0^H5~{CCY8QNHmz}o3(}z`G$cJ!{$vsn0^r(r$sZbjC82(-^?f#_Uub| z5|heghXN&|XyvHjLDgpHuhu=!*;gnOzqO^#4Z;)1f44N_D-5*Kw z?-7>?g}mx-jq70IFBr`x%v6{*+Ly4fm(cjdX8e0osJLtnfqu06z!W)hF4S-Tnc+i_ zQ5w{N!9V9bU@8kB^|I4s7bz#a6d*K^e2`3@W+Tf}PtS*Tg4yC=K!JsE6z~F0K;29A zEJ}b~zn8X8MVKP5YGtSXI(F91rv#4}cQC`Y%Lz-ObTKVT$2B0F4fz1(c0j}?`S?rJ zZ;z-2l+QXrD9$J#84WC$ga|49Mv0U?MWg_H&n?(gAIG*@WfUjR3&WzAzt#`V4#sOhN>Ji=1Z72hDPs7Yo_}je<0dL?D%_+3XX&+Cv--w#& z;_P@uQn7Vo7iA>6H}`~}i@bOPYL%e{IbffdqOI(6`gbHXS6|+?yF$n&n{VL4%y-BS zTyMGs-Kjme*Vfbr89`XDX_1N6BOh2^$@VvL!y+N-+_r zY{k`KY^gt5IZ6}lrY6p>-poL`KvzCd!l8xB0cq#gd>{>n zZSehST98jd`W;in-Yb|VXevO#zkZAtu4P)J+Z@BTINJ1t z{vCqUmBterM=lN)lhn}UR>*l}2uo9=9r+%ysOAt;HJ^pj_2}I`(4Lh$=`V|2t58@v zvQVJnumPy1<~oaQ77~6X9?DG~QQ{1tE&zQt`9zR5T66~yUKhuAi0{=sRS%Ja<#Hr^ z7ZzoaGjGf-Hzb>&5^v?`I)Me}jhQ7Kfms7u+0d8-;eR*;4-}{4aXhP;m-un?m@Vco ze+m4*e&AnT6P*&Jc#DRV(Xox7wq2$?e#zhZyZ-$@hO##|qkKa)p|0|E9bc|sDYc$3 z8wMFMN|EQcr&D!WS&L%g{$pP1vtLgZ%e^AY)#t5Fc@_GX_90(_RI-P;)=xi-v5_$Y z6)7nnIw!PK_~Ff4Rw0lPgastVb&;9;rOv#9l8+~n`IEtNm-`m8>}hMedJbQjrQ8bo*t zF`Y!R&S55q6r!oF^e!g}5eiGdrdko7Nm^FkCnNEl>=z!!_Xj)u^a{xwWt5CoSc~nd zbbl`tj$Sv5rZCnw_FWj-hkpB`=|oi2O_GfPNNtE{F$anpyl)mxoHHrGAkkYX>PAUDJ@{;T+z35ATCxHvH4KM2y zd=t?@P}1c_$i>PG$F6H(RzAc6lU^M&Cfkx;;&2~EHYJ7jgDkT=TUzsp%W{g&Z|y6Y zMvwrru;ZrawFEmCj-IcbMV`77?=6+mJ^e$Tsf$~`UPlNoY9qTd(`RbT-lG@-iifoq zsu$DidAz!4p%7G7G8_G7x_$Y~Dr}yP9iP1bA)V@OmwFb$+mmQ*czL%7or89Le3$Ua z8*TuM7-?43iAnes!2rjyn~nOAE5|MOZohjriG_E$U$-coBffQif$!nrWR_8xQ&3^L zHuYu=acMY2#dOAm2YXDZBqe!1vF#zMfs&v+s!{_6^!R_7EtkIjzB8~Jv```~#jeRW zpTPmhw7}ziz4M7g7Xr6fbMkZEaawuPt&mNKvN3L7|Mp&A_&}x6z5Mle7^p=kX1lq7 z^;T$fVzSn+y9oQcb3sk4Rk;;b7eVh$-Vb5F6N$wVor{XYp0Lt&^A%F;OdKImmvT$1 zq(J@L{8zT78{iCFeX9Oo(Cx%dkq@Ti@M{FkdURoyi~Fmp2{Q2a-vJmIvR!+%Qs{Ueo-&~qOsf7TNd z;@Z)GfE#UQMc1KyW~Pmgz`Od^k?bv@vK|Kng;=llmB~$Ypg6Ma|MKILwEyQ{RBJg+ zvnpK}VN;J|5?1k#lpo>+@4tMQZowfdv$lt}q@(I$TEJZPh$9LFVJO*Xht`DO0tVK_3qPXlV4wyh)rT^RgsfjoMxCpIaOSsR4Z0BC!NO-p>!k0 zdd1)p$-GTc!6r}=+t1KU??1rX;eCl%M6OUkSQ;*NtrW#BTGV$b8#8t(c>_dGK7^G| zfQ7KtG@TfOks_R+boX|)5s4D)^Ka#Gx3_X6u?j0D{(j+Xf9Oh~$^c(=UrhW=fvV{u zQpV>pw)OhgLXs!pg7fZjK5*TSzW4hV(?Q3oDBP_vxycvfn9%IdmsEi!=z*)>Q~swY ztIn?-?~}kD6ED-eUSaz`#|O?Ltg8%NFwtF12JBgJv>1H+N{%^y{kzRv@-yKzTKA;h zz)TE0Y5GX=cd1hMA{xG_l>?o_?8CXTBa9Xe`<-@9aOmh{60ehKql5n3JvgSV&TkwI z3r!a;E_+ZlKsf8Dl~t2XZDpmy*ZJbf0nus@hYOSpr-9s<(-^ccA1sQ5uZH z$Fg(vCe`Ue_grRb`bT{6dq`&*yEVpMa?RhAUxx7nX6q(ALGLK$LRLJ&A4Z|YeKe4& z@53z(>w7W=+CO*W_ActCh?^B~krHU*Ssqew`AY&)yIcds0m%;93|Sud=d8^mNG{o| ztbKls{qB@|*B7Eu+M0OSKQc}SlG!G_1Riaus6$fA}l!fGDNXyL)@*1qEa8X*C69*X=?~Fel` z#?NVo<(po6YB+ahQYLPWE37Z^(T#C@aDzyiIGlgz3*&bqK_{l?4c_KOim3lGf< zj!xU~wdY1KyDt-_^vi(L4-9#Yom%qXqb{{=a`~Os3v+Ja5aWV78suqa306S{pK-@s z&dNr}rGgwKHmZ9IkHJ&+eM0`@6#AGX)=?Y@9i_;81W;7oW`U#!z3LAkvnFkAXtvGWSOHFt~x{g-hG16;?iREI#DBD=x?Q*FgZ#+*&FqvUF)5F3rNdl zuCriIt0oWeco@R6uBG1t#2H}%TqUq$Bz_-J*sr#CW!%s0K=?-CbH~7&Ugj{?$1~CTy+mBp%K0(eC6! zu47dQoi}d_2!xY#m;o12ZzadQ@2<0ep+w-?T+!Ngs?afhI*qUS%H7x9qNKs(a94MJ zC@eBFjT!STO?(70dUBG&Ucs$xxmIm}U(HI3pCmlt{rFB8_EB4XM!k$PMS^}ByAd7K zk~;s)aTrdD7MpC6iphdT|1w*+`JW0^4R~T)s&K9PbO%bO9WSJ8Q;A5y}xp{|85D4QmjX4(*en-2krtRT7bjA3qi4C=ip z`%I!`-(`OCeHyke#3XN5k1p_NFr0cTcF%%5-6O1LvQI)czn#4*(?*jqkoAI}qrTAq zerjo5+Fxx~IIq56hweleSJ9ggtb~mW)L{&;I|zZQS9^Y60)x5fDwcnOn97lLS?rmx zWYs&c8&fY;Yt&Ft9i(K4vR^m5RB z)kU2;Esy4P*gv-67C-VVO;%~(aV9PMzjS&DNn_^iv@<_6`1j*LkY=0)W=$p92(qQH zQV4uzOSgjc-TFTT4M77|N)vVUVou;H`X`r;r{jd;4k&oiA!pCWh(Boef1kHfv#fm`lVBn@{iD5sRLXz<%4QI14t4p#V|$#` z)r8UY@{x*m%n_OF;FFF3&%1ju%omx)+DU2>9BiEEpZqOcZHqvH_o_FSZee1$yms>R z`w`l0q<#=rQ==pWGWc2DK{A^(asuAvQLScck9MDFb1K2h2)Uoa22 z68hGIcyq45*;uRp-Y+*N^$72QkrT1sP=Wq-=-0e|Kl6&)HUP~C`@)U{m55J; zy1?(!Oxkj(v?w+QV(IUyhqA9_nGh2aXPpdq3DF-yTCa(N|2Ky4SEpouElq6Y|8xFB z5~)}aK@R-*xp}SJZRWBV{YQm8k|G$FW&O7y_;nb;PYOtO3r~h=HR7CSUsSX>9wXa? z#$hNZj}dYx&js*XQ{dp2;VZ}qDIkbQ<^F?{={UrZN}{1HRFW)EKyXBc^K1Dz4#Jt!J01>t=|ZNwzW59JFjn$HLG5lpmZ-V6c4!bh9WIkd7!e^MtcMTQD^S_%B2`judpsLQpisC>%xrSdvaRJgTF2#6eC$zT-uypb`$ z>^$?0{3L*X`X_IDlu?SVgP-+}1S{*F7mGUNg%sDL1-s*laE;(Uc~zq%%=*j79hS0( zL(b1_7vHO6H7Q=b36gC1M86Y))t}yE8o6oIbzO+v__K=vTZ{|SlLIy8;ds-Tr;es+ zB9)vyo?0f0>F zjIkr7et$-VPO8&D0Z$2Fa{g|uOGJ;pdbP^^Y*ke0+oVV%UYWSl zRh1xgXN~pObkei_f)UBh$H3P-56Cfw&)ExKbKhCcyjd+ACl^KYi?1;QE*d&b1-lIy zpdVz7*LD+M ziaMG)97$#u?-&UtH$NJdx{!d!qVw$bj}-JAQioZ{VhWsl*Oi#r-_3`*Tk-pJC>19& z+j7!@TawC0g`f4SLZPXMUcHKQOd%9g*fadF?K zm$wPK-K6x*ZaP%HRnoN9o%UT-_rDUX6)8v zCuGXEXBcvR7Sq6>o0e2%r%k_xi9J$AgWDpIDUQtk@ZFoPMFG$uYCjhXV6D3NMK0^Y z7a+l?$lj5iXB?OuF1$<^;%!nfaedN%X^e)HaABI3k17kET>gn=rqTB4%MCUi#nR7J zdk3N<$MAu2l4(#7zqqK3P}fDeGe?tjyjGzChFAvee5&3?Pg_^To1d&^2ub`$Ne?ap6Hfy`tMLjo9ETR7tgC(=mkOS_5b`34 zY9UnS1*k>A_3bE&rRM;Gvxq=v1qks>f1}@^MhYKs%G8Psar9CB>#xJl==u+w+8n_Y zhy3fVr3OX1ZkL>`c2>qUaK$T5Z!J`<(9|OmI+?|0<3eRh{b=Y7C7FXEXw=N(eVC1< z5JX-Fy_$rYrk`z`iy#o_z-9t2%cIH&s$?@F1FM5OEY+w4T8twzy(>Q$I@{}<>jiH; zFA2oVz6$-iYWft%O2^b?tk4#B57kdY_QQJZ0Cy8&iguETQZq*TQv!hsRsN9qL3GDP zmPn2gO5&rWp~$vpo;sa_4N#&VU&h0x8nTq)Tfj6^8>abGbWfRYSK-*xxZeoi%DrW0P+EjtnQ_j#*UgQ>{#h~wHoD?LhhbDeUq7o`ef6nbUg(MFAA+*7U8R!p zn2_{#C7AI|0xVj7`^A- zRDZ^LVX5u>$@V{ET=UOC42Y)j1xiI>e>0<@&H$}O=<9?evjSd)M9Xw+ZE=h&Q^8ip z%)TH|Cdj#-qCz2(NKK|paXroVE1JEw9`7fqlf&{MH~tltkbz^(AHBHT4~-W+y<8Ts zgEV>Gu|k79V&qC4F5a4P6a8aE2d~ru2jlYx#e_RH%sOD8cLQvR3lzwQfumkd(4Sc{ zr#J9JIjd@SXlZ|M2(O=Zo|rC%U#8iV7*{gCtyfM&_7Iz6ERXqvfV+#+vo3QuCT2|Wd@-+QvSm0%3$S+`%@(&AZ6&`O}kbv}!KKK>`r zMS4=}t>UNOBt)Iov=Agm7YoXjsFVjW(JYfYO6isCJMETt(F?qNQ6@$ zCZsEry_cmJ1n9=t6div+p`u+ZzG3-mo-w{2`EwfwLZx;@+o6!X>Q}v>q~-08FU)t& zF~yi+YANcZDnGoPUURA8Kl9J}i{4v(wNF34YIaUwm?&)O*LECI^oe%`d#XOT?@7=J zQlaYhKl0|oYWdU6R>7uu(p>zxeib9}x_-3M7Gzy-gQkp~fRk%%C zs<(8{3mjUvqP#nHkpT`9B3$Bf8vsuC|3KZ}>_j;Jm@kZ$!Xj`XLV1vWAXf!iW~HVm zNySYXdd%Sj{a;=_}9Ma(_ec13Pn|_}=b{ zQZ)AJmP~Z12%)41l|D%@!I8z1(PNi=4{zWqB!M{?8cL4+u!6g^Amah#p(KanrB;El zwuc0?%yt|!I;YFBzDy(QErD9wap<$@&j7)I zFD>IK9{PIZg;e&mmgv&{cW19Eqe?KomVq|h4B!1tvdDiNrMH$3$Y!JwuE!jI!Nf-- zrNA9~sjn~(vPdgvLyO$oTc5WcKSdoMuX7^#y%In|-jAX39`GPyJ9Y~jFg_h=O5)*X z)&>zz*&7X~@zn8L^23B4x++qi1Wb)q;B*TIK5wv&iF$ONM>kfM$U4o0C&_e4&90OG zNhz2+Wx&M^==s7O-OXNvOq`QjA$EZ|MMV#&Cx)^Xd&X z{DAb9`vW4UAu-c^G-~|1@mSc1HSj&5z>_-VJ~D6={1LWAK4(B--aJn%Sz&9YN5xow zx;s&09D%Om%ruEPO%mC>Tg)oV>$N?MD?2ZgsMMQjKjEGSsg2JofdY!xOoGd1aCS|6 z|NO=;RAp{{v<%Eg3Jg-w)(O}o#9>Lm&G!PASxv&;v|kKc{tW}Ve(~ae&bu!Qh9oH3 zH5uv*&BMy`H3Mw1bA$`WY=f$?Z+&A3Wl2V3Cy5XIsy}-=9as_DUiG6Pkjl%B)2I?m zRApYTD65-zjh_UM=m7^k74 z)>F_`>>6nyxe)UX?DfB;WRh2@tbvn}ST{Q()lEKSmz>EyM8nn2+O*1LC-niEN15~$ z%nv9X)+ODr#QbV&irz3n{#i5DkJ8O=SBtJ^GDZ=PeeO)U*=eq)#is2C}U(<^tg%Qg;LFEv2?DyixYAdpoN?YjC3E3uDeRmbG4M2uOv zq*l$+)gEz5;c5xY+b))PMWvbm(SZN1wuF&VCUial2&Fu73bBv1)nYxSx#n^;pP6jPBTFM-@s9Vczo>DEguC81sN2?VMaxu`97l} zS%t2n<~=|m@|7T?l(gLhB*R@hO)JD7wV!^i`|C-)pmpmZC&d&km%V zL-%zH__l=KrH8Hi6jz$N^SDUeUyv#pyuJX*HIN^t)`SQ~wnz9NMgce>Vasr2e4CAaWs;gn7O+BWb)90|yv$K%?C@$)mDWf{B01dok~^|DSgn<>nH(I5}+qDUof*jQ!Z~v0&lvdi?mDW%>jb4nU&Guhn{g(c%btJSknQ#PKtQKk z8>wRC108!4uqzUpdeua0f!zMT5z_b7s_sJ!TnEH}GLV~r&feVHneo!4nbRzYifB@K z`oCxaB21v90Xtu=jVSwx1t}lCx=+jXrF$GEhS2K*{G6k$Zi8?{XR)8=wrVm*t{`WO zo`o?IyDARF9E|1rwNSiv#rZ8gQOB<30LwawXMCdxb}SAS?LdD*+Z{q~L+9Zahcl)w zPyo+m@PM)jOp=@tl_LU65sq($n84+}Fnbf!7M&Of50L-Hh??SKD7sHwb>u<|Adnqc>(Q|vZxW6!7gkT(fzoB(x@z>mz$0=a-Z^7EJ_u&+IO4w;#-?pPHR*fD0 zC()Rc8N7x!tfbBB+03`Jj?|C4w4TKEneO8ybIGWa1CGGe2dO7 zn={aZM1>hazMO`)CL#>&5Z8Eea?dGDk+&sP0mU&H3v+2GW5t14Z!#Ts8_>?9j?VSv25IE`qR*5w{dqVD$CT5 zzV`r^kie0`84Nau&jDrTAfFb3oIo)ts2#S0qN2`kqX<`h5i)@?x20`=Q{s!r_+;^d zQYWy#xMR@lkT5o;z6A@NigZyd3C*Iq_6YDVGPyEUN2?H{gJaOe|@ z@Peup-pYB3>ge>FkyhFkrfwDqh_zifk`SoRB_2l4AI78WAJ$bfD^LE^)YjLWvRQG~;N>KsL`!wzF)ViqV0rsqgT+dO`($W2*x_FSE;eb%^RV*DgbR-Gq$l?UGi} z0Tm&Vx4bS2RtoOm#y`S_zC~{#Cv>ttjrR&nBaEu9p75Mubs$+`g;}Q(GQ~Un9Vj>; zN1{9nqF-{kfzXyHR9!)U2NL=PNghmPDU{_-JGJtoa6^+^-$jg=SZPOiTcmVzPUs}~ z^8TynxlZYE?Zy^2Vdk(J7Jlj(Y(s(on?8dLC=laMi0NSN{r^}1tQxKjkZUF9CK;uo zfWI9CWgu)KMRwy2k13bMhl*%AQB9ieFm>y?EqKsPqUIhUp+Mj|?Xm$DL>c}r@5uRm zt0>@ECjRKx?=b^(`Q#cr$&&qyV>S=P`70WG1c~>%xW(p1UYr8KgKlK!ikyDv{=jEC zM0sQ-UNJ;wsp(ec=O0Z1CJpqNcX)#+jlzY}c&x&yK0Y#GhRwU*%p>L`U^2QOR@Db9 zoaxUx1n*iBnOkQu{Ad;Udx0r*KG}n{UVj{7Op6$>^m`9>dnAC*am!A8N|T+rQU-d< z9y{lXl_3egJ~RVY>`8>zXTOJeWJ=}ms2pg;Z4C;Omvy;0>w(|mzdu5_%yxIf*Hf{f z05})+U)TkTM41<>VPFM8@l+DJ{q^*tx7rZKnHZT%GdW5o3i&_q{QX8Pr8y0JNi2aQ zvWrChUu_T{{hzc@!YgwVi*UTAHk$f-xILPq7oa;lL|)n zh|rQsf?pqHJqj6>ebkySAB~Jzb>oj`MiCt#x_(8_|T92R60xk`0w2Q9Jc*mr{ zEK&k$V)AryCaxrm-#My-4XAMaSBDhLP2gY3N+b0q59F)%`atqtt;g;ydXM^lY!jRD zB|iOW=Au}bNuoX##QTI5NH~0cQJ5*Ry!XhH^!F*0O5SsZUiELO=2uh`2J;d3i71Em zHrxOQe)aEo7KZoU+NP$>Z1E_xArLdJ!`|7Su+ErLQIek;dlosls50+XJ>;GGxHDE6 zZ&F@Q57kb>3{x|B)H0{+ia{aLI@2@^>N|c^#vd(--s5b3HdS;;s@r&{R@}1H74`== z^Pp3Za{BeI^ZY2*_mx$$1}enPJrS6*F|3W*W8T^SywdU|zq?UCxOmsO^+H)r(t+R~ zUSz9Kz=?vq*tN&cCDyQMZWqoqK4nqcR~{W$UY{$UP(Ae&H9bb`)uQi&mdJOeEmG>X zkccE*GrdqUtZlc{plS*W;mRN-xlsiU1849}XLSA0C12I;jq-*iUTPlZn2txxt_QgxV^y{vZJ}}OX zOgd00@@4?c+jIPmFDbAqsWOOg8B~#nzfw_A(m7)hSTjBBaO%}mn7RMCdCT>P5kuBn z{N>yG&yq$HZE;)DbjN;A2pTW#LwO)L93RW|t*pdYy)h8SOK3yC zQZhP!5UB%O>omsM8<&97=00TZDmzyf1+5gWF@-k&J*-VL#x7wq)F;j{jx+MW zljo-}*QbULs=ReFA;W4JAzY=>7Y;)6uDjC}L<78L{rlbTVZ75OpHNW?AIr3OE-gQF zuecZyRp8+6wUsKv>J-)zhJwqd(+SHIY%R3G26C55a#c0x2A3~h@{KIS7NwBUX=I@_u#|dAy92-ZV|rw#xkFX;6CXTWQzqd$hX6xWJkb zbaGWc7#QbO*_!G4V4SU@l0boO>|=|ws@lH*p1+&|VcQ99HbRf}(=@-DIS>@{N!l#e zQ0|6z5;HNZ3CP>b>oBy~OxfJnU;+6b4;jW777Z}7dtmylEJ;wL7-~7*FDLN+QKy{a z+wmXcrx^F0-+HR=Irs&Jm1IV9LIFa2-dDvbq8D+ZBPHHjZz-kokwVufng}h@T{^9B zStK={+`DFAce`W`B_tcsOrIe(ddGG|A;fPSzYjJ?S|U1`AeUE47bmN^JVqB}q3ZHd z=zzw`htty7%^P247w0MG3iZ-*3J!_nS-_RPdFy~{z%?{K%hOr;9uHx|+Y~;z-XFri znD`}#no*u-_&76EmXw!FMLUl}Zq89AQ$9Pst*kNPV@;QUUO2-C3zzj8wUoW&Vvx;G zZ%ja0mdeSf6stXx>f}fHZYQPk!&sqDs|IiCee10riTs3R<8^Uh>59X>w(N=s*10y` zqC~H=?@3)xK4suZ9}RBT8j`Gqx$CCQvjn`tIwneH!fYuTi@M@ z7xkM&K@r}P0S$=x9M}rupa92JQ9gqJ1X&i>OG&R^W^CX}8$Z|cHkr+0$2LA>)~d0; zG(6LCyYKx)RINmJB}D+I?B&udp8TW%2yQdr0{77N@bsYeO%k@I7u5t!Ij0p14heKj zn+*hH#ukAt)`me6uw7!B2`e$ERc+i@MimRsw5G$S_T2dY*m~=z zsJ^%Vdw`Ly0S1s}=mrS^X=dn<7(fuDq@)p~q#LATkWx}Q2SmENLkZ~=kQ8Zt!#D2F z^Q`+`do5=F!JIYg>~pSt?JM4|y2-4!kw?d*Ktc2QTU)lFzv_yNat4N1lbTVY5>%f> zGcS%@>ICpL$}_03r3$NO0Odownb5bGrQ;CWlI;3UWt|B9Q zm7C<;_CCq-sh^|DZ8Gb`a-UwsB$T`yc`7|%7)4Y`bp8GN$KH?^>ahdru&d1=TcjbK z3dgDlR<{ZdtKqw|BWY5z%xY7Sd3#ex|} z-!`V7@kIILMQ%hS>6A%~RyG|(!FP(y#4CRze0j^YL}{XM)v;|tqJx9YylwabCddk+ zY<#Eja}1=d9r~#C9HRGc6ZDl!(Nyq_SAyCdTWDluF=C_I!HrEMBHzAb8S?!WdsG>^#Rw(;X50rIlg} ziH5pr@>G?RzoD$kC<#j7himn4K`L0~FOqkLZHD+A87AFC^Qz^@zm)3cre{Y8hBX9@TeTpI}oF{=a)N{De50pqEs@;yQQr<&TA0smD&F` zYy$+>QjHssl#t31e0%7#yg<4lo=vXomm;o-{PrRGjqs*oc@%@g0Olux!1OLxnu&I3 zxa>x7N&ICzCq<~?5G9upik0IbQ#N~1Mp-NR>?0aq77DmpkB-8G9IXFna)Yq_o)T!M(gy|G=7p3&i2~$hvKtm`WOO2khI3m- zJ~Zccv&QZj$uIwyZ1(mrZ2heW~ekBK7NfA<}?D35VEZgvEZ3zyh+xU45-_S0?;F3j@U2BPkIO2^ z)d#nb*MdU8GmIHFZ;si864(WO7U6(W7$@k{oUeoF+g@fu;^Mh~W+htCX1RK{jIQe0 zMY4O_Y)I#g$vp6*CU@DANkd`||8Ay^R@Z60k=vXtm;MgycM(NTthfK%*k{8J5dyju zx!S(upmb5qc=@!}1^>{oOW9Km=6#c!rK6MoJWhCVI>82j7p#|Yx?`vgMK{%e69U(N zH8aEz7206*+>AdeAjc<_B{u$^%SkF)LK&p&ps-1!@v?O?s{P+%YpLZaF5Q#$43ZIo zcYOPysk^bu*d6{%vhtnfj?t^oC&T^Glxr9D;mOuiX6!Af+6EdPfiXdxc7e?66|PW} z``IGxNJEkSb+lSh(ZNz4^SF_Nlda2g?l z7-T$YSThfV!dq>wH{}#f)XS$oPJ8PK+C4rQo*fbvk*#AH6Rc3f*`IlIgm72!{4NAO|}BE&QF|WL%mq~YRJPGC;H75IEbX7uqwk2uAuCV zXAAbU5RS>JH-w53MeZ*Wh**6o&}+%fdi7)ELrC=L6+9o!2_vD$=F%FxGf}<)6hKcg z@z7mhKe5!N&Bz2Wt&4lAGjU57uBxKNl{r^kKd~oqF_^2Ia=Jj|cNF}d_pRDKi^TG& zx;dLHF!SfxP;Ibo?DYo^zU4DHKCN5tPwK<5)IbHBmOobrgk;xkO z3q6r8+xeNl=_+eVMVbP|irE=rIe(CycAdS+B`8zn@R@yJ6$iKG)i>5AThqyQf_w78t}kNY(8iggbopPAL5G98rtU=Aox zum3DR7`gt}NZy9STB%sOOpHH~KDHuaH#|??&cdfql={>NqyX!U8{$i&cI@$M6b|fs z+c_>GBnu1Muvv*TqcG3A^4(L5f~qO~lr$I5Svvyd2i8zH*ojs$`}%4BcwLbBv%FRP zEAg>ldDOn%@5T3+(oUO}tHtQ!GJlEPa0sRaUHZ|B%<52icTt=*0N0?)&h%6J;|sly zfs-;sG*(;QN zY5{q3k<+A7bw)`N4j6!S1G;e`w7#9PtK^`@k1S!%EPRHlCpbeyA(b$cQsx1hOxg(~ z#nEt8Qf4kR3f|H0C=#jCckx(S7DqN1TW6G3gvYk#MO6LaH9E1$FJqbsIHkrvV4i7K;NmA1L`e9{S*kx5sm!%cgWd}77O->x-!S0 zyBsZkHg^KBHk-3CH@T~>Lv3fRwk9<5nqnee*VakxvMvEOB+Ug>NgQm+4AdQdUULLO zYJeCzTzNVtLSF3lD5qpb=rCimRoO8Z8P5q>6pFASXO}$Ch>^7P0T`mf1Z`h?(n=H8 z_rz`z6o8DY{AqFttCp}#fn+w(BUhZ>KdyS6Z|B0bi1*FqP`JpZ!;!ux5#y=lFG(%ybz2C!iNRnV8H z6wzx?gTXB(v|p)2sd5vV07JMM1e)lzI|-DaN;NF~jymfem9`xC!68M)+)@#8+jd~|D*>+U+i%BwBaOPAZ}32Ce_AFmU;NF(Mf;@E<&M$d{k9|s zK8&UW5mJpvk^(L<2ru5gzgH!kc|-)F9Qn8(Dn&;k z+rK+NDGQ_7^pMFy+cEDaHnv>?G4|&X~V^O_R0*iOCZM(V2k9B)T_H z`L+#Vi4ox)77-Gr{bXu6oxY{9M4$$(hv_uq$7E2Xu1QX3N zi|E2bm!9EVn~M2@V^<7khDSRS+m#R2d{IeA!ADcv7Kt&mJN9z)K-eNmY)}DAvy!9t zhMfy6gpUv@om27kqY+*v)^HilGyq}IO|lREPIF|y^%6Bp@@c-!xXg2bV+q^W+TbQ( z5hF_+*AzpDSEvStY`~HFU$iariFR*UhD2&pz}%zM_6<>fCTgrCUGH%kC4SIHZ~>Yd z9PF2QWaF4`DN5(+O|0IiZv{9KWY*tMW(^xq*+NgVW*=Es1BN1DJo8~N)}2-SV-l)1 znrvfV8m|L69mFVb9U~Lv$D;#QKyjYHEz{g78Rowm7vd6fkb(#R6LujxN6DmJf-aQZ z<3QVh2LxBvnk>+N6ijsJPV}tAq^raCBq?W6dXJzob;}ojPOaV*?;-{e~ARm}ODd;dj0y~&o{ak&8?kaM+6`h%G zZaCol1A4=xVtXryO;GbBQagGS^4BhCFBkXKpsq@9cyf{)@qpLbQIuMQTFr^+8L%I1 zn>7Y8r;cHm0-Dmq|5;jw#;(;vUKw!uK@Qkwx|kYW4nat8Ee4En@x#4@KR;mJarV)| zoYuauJYroQV&fVaX*Q1?5NoGj<0h{w3s}WsN56={tVqS)BMHI`xDe0IO~wRxE11Ed z=97^a7!1`Z7}ylLDQf`Yw`&_DhGht*STVRL01+zWk7^n<9L&s}=c;TbaXH_AVeRY* z10T(1sVkb3=@1qh3?8&Mj(D?@;YAome#$RdrWI$6)|GVyleUC9m4z!iQ~`D}w{hH} z_>XKd_j7kU&Tz~5f*6}|L*)5EG(FRUw!Bz7qhU$&u1F53(ke#+`#xKqdz6^6ohblQ z9X1TXl+Sos3Ia$JK|xuVA#cnaP1XD-ld^%SY-*pLhW#h1`1K9yNUzQ zSrDU^agD&Ovfi*;{AzE(uT>+_%ITd?ZfIen0&FUP!Le#s^Ps1LBC478-i*Qe!ZHC| zIbUNFxN%l^Snu6J9|qQZ^MYAbS&N6&RZYx5^Hs6%4OUV#IG{T`Q+7RVTg1EY6+gid zzxEZ{QJ>KbaFfu~2aAyq(LJtoKyo2e)TX!Ij*=pBIA-tF?}2dnY<*AQf*G>?WqK(?H6^2@hyrqE*YaEs>7W%Mz1a&QxI zF-t7Q9Ltvvk3w|eC+#j9!eU%e_U`J8;PvO)Io&k=`~xaviRi>?$o(z25~L@IQz$=Q zrMg))j&9RM>GFaQ+45YE$6!#HoPppn9@7rI39@I@HDgsG3QvEYJjK+vPZ*4&e5%)> z;G8+&P}6Q__5_%Rt5eM>7Mo9WZL{_|&3MZq{cbPnuj8beHEo>IM3*wuXI-W+NtGmk zhE8qNhKl&e#lw~^Ruwu)Z_TZ}<0B~*dr@XBIn?L)ekrl5y@xzTjst$N;{qTJNWvI* z=UBwHH#QfLg?X?5k9Xo-Vf@90z`dPBa~BVH;&l>h5@^04x|DqK3Mr_73Y-8N>;RCN zef>DG_}pE0FlW&;q0ARSMoub5AQtAlY#O9>gNUfWCg`3p#h$_x zvM?|{c_FX|Jh%PlCc8}M;-JA^7mLl%Xj><2-6tL;PbBer- z9BV-*)SDt1c2J`ojs#0xTTd5G2X@hbIHJKx zp=gNtjxPk6VTM{EE;jwR8juq*g{lg6#?F46E#{bRUP3uKIy6>2tly`|0u52Sz=>qy zRkadgQ`e-&InS!K%%i)DH2GRD)@U%3*vkyEvt8+2BS~=B!iR9kxya#x?zAB7H8J_r zhic?-hwXIkP)pI407-yTw+*2~88h$Uc~CoZq92rC-`h+t z1EW2n-5p2hFLR+AW|mJ0G2$33#()wKrNIf;$|6k?jm_lw%WiSS9Tb`>DE(2ZoFFJ$a`C5}3@5c* z^%U9u=g^F&tM6M!d!c$%&qnTtv0Wlo@T=u=UTIDPAc2@_lod%3Jz^Z1*-`_#*=m&Ml97~^ z3a*uyPcB0_S&(G-fS)t;!wTzq+w8i!Tx=S+ZS2R4tuGR`t4ltbu4WXS)YcOZtLTR( zc14yobn~PW+ePlwo&OJ6M2f zrxYqKTmKYUr$CSn?&k>le9TfZK8Yfr0@VhSY%P$ka3p(&sH?3|sa&wTElC2F46y%l zBKQ+CtEE}ORB&W}0Y3(%6=8qhOG3+JR~$;2PJD8DhoounyU#vDk}}5Z%vjiy9DOL+ zDXPW@S7Rzcr(Ly|gq0#q$M4)!Tp29iDG3D`!jZ971CO81d%r5o!2uBFK^eduJIQO- z6r=#b(4F}wCdHjJ1~5v1ueJ+hd=f!D0@`fUUD!Obgm0Qq4vQA!V&E%FK%#Ae5Vt1d zP>&WCABF<2+`8x@!AeJd7cx3lY610xJDmGXarZo<`+P z(Yb+Bkzwx&yQmfxOgM;Mv7bE8PN|Ol#9WA4GNhr_pnP{k`-1W8Acetp)CbX`HP*^sSZhkU{$cDo=j8!iXY)m*YPD~@^BbVAh zE%=|x;YB2q^SbYv49)L-BWB^gcGEiNDW@-@z1W{R7Ie4?j0@24UtJQ;tC!rX&vPVatEHhQn%3g~#f^HTI{VgrALd-7*O40@3)*C|Xdw zxWu5f`rQ)5zg=6@3;Br0o2d%SK`;42mzir@487J0orvVsC*2<9=h+j1jQK#2UARJo zFJYdJJEFaBm`36wT`YlQP*Ygq#}ZzFfMo8bpOjdFKxb*A`h3e7@((8^cB4XKo3{$% z&?qn8DuItH?0pTKZqKq5KIHJ*Tz(CYz)=g~jMT^If9ur`RfsIn_VrrB0)XGSX$L0oRXKJl_>@ILRt6yW~Tm~>7>Svsl{d+L26PF-DToB4;2>}Y2?iH9lWjF)N{ za`Ru7w6g!$PgL5OM(qp}w6Q0Y;5^-BCYU|teaxmKs`q8^K-4u-PRwm#Qp}&<+DT~p zjp(pv1f{GH<*nZ+uPmIgs9={U-Lz2I)_5e}a_8tpCAm}(2~L(PqTTXc)3^aX2b(*Y z4p_37Al0@Jdiv5^hy{TKbw0A7vvH^Tl?l+*BFWa5*Tish-Q+@l81FW*?Hvbw7VdMa zVKoi5e8h!j1J5_$c8%djthfaFeWDmE>2Ug}1=`+NP$LiAJ7{A!{hoGQ*$L)2HTTKK zD4B2WzrBCpKqE3VrcbcSKlPxbPmH_dk$gx04e4#$%^tW>}B< z=A1Tm&YPb?NciozvEvZ^388Z$m_GGTxpO~`A*>a{*MRlErMr4SX#0XmhS|xD7Bx?Wo#>>c5qN5C1Z%XzGxV8P|Pq^Idn$EdKl%!mj>lPgieO zaEGXcy@KbpI<9TDI`LMoBvfIB?%dhspX)ldVGGesEJsBCzz^A82DIGn^N#%crvJ~a zFXKskw)1O&-fW<^??=;smr0TwB;s;(HsX~*kRZo!*9i03jV#G*9$#5cfy2L!)ceAy!4dyw{h>^*lH5KflUaKlAvR<_g&mD>G z>c<-rAaG!%>iG87Or6`|pWpNJaiAFNiPHK6tsq%!5A5H~R{#FHI0LDwSNi>lYpHBB zq_RA`tp2e}G8SXkaGTdNT;ILy?a5h?`0br7bwvI!!m_ZOZ?VVn4pP z%x6PrdaNIuSH?|z4d$(?H+sek#%ax25*&#GnH zrU$!cbTJr!q!_7~M+vsxbIiNgt!VKkl&b=sv#Vpp%D4HWX~2(@t#x@mdh%bY{PQ6G zGiwk>r?BVnES3Bkjcs_^jH1&(Rb}_mAIL^}_UbI?JU)hGpN$R42{uiuHC#>Xkkr-q%cqBOztN3ijXPOr%AS*+hWGvQ zbml!ndf&gWG?-KTuG;+=5a)Sbn{$ihe}okgcl9N1@>|!6g{Yd%E($jD4)QPb6wmSr z59rYmWpNkWyukMOv{57CGyqFt1&%_9JxyC}cIw-yyo~>IDZ&ujL4UpxyJ}-WyM;=T z4_Dp{RyGfoHV<0RUmNJh-wOwhJ%Y;Lq3Bsdp?43IDHKnr;eLPY&c23Ig>YT^ zPq8M;zYr^j$8?8i!Sc*EDCkKgirjJgW3JZvfqi9o_`Z`g`_nI3BJc6{hd!@7@aCaD zm3KWa|5J4IZ-eq5vmypEFk*hnra}R*P@b}DEJ$9P%05TcAE59d(nwzK%QXK>eeVK& z@4k1weW)z#lfrZMq>{3ahAE7OYEpk2$KIpj50)R!P<$?r%FB2Awu$*NDF@b51UsX9=;qI~vg<~DX^u7(ZPXI6|OniY0w zoRUE--p?-iM!&{P*!UQdq>c09Zu;YH66h}zP~@Msi(i#@c4<%ahs^4z4&<7I;=HHx zvi~-3NA{w+?zB&vA{4#cQLVb)gzCpMvczFvNe60U_Zc_}J+v$vMUqg*WA3!KtLcCB zKy3R3tB0bL2V{S--$u9nk2!rXssA3%N&i}j$GLbIHkZ&e;b9a4J>TA+k65SzAQ&TF_b1cH%V@0UfqH_6=ojLL_W}0Kg=e57xJcI$J+SxHcfw9 z4i8q_4iG5jTczg%Q0W1#GyIf)7s0}tgC>+79zoLt)?m$G9LX;tbW=4_(frAm7D0bx zXE*n;q)QL?YcNk%HE!m<*%4m64(UccL)-pP>wc}&UTyZ}u=V_Pl~LgCuZUpJ^Kaz$ z2mW({HApo3?_JpwZ@;&c6Jt@xX{6Tk;*LDfX09GomeWzFx!oU^n{H62%tIoFL2 zs-R@Yp9PD1W9AkAX&wG&Pg@j5Nzc*AE(=?{l{Iv@oPM%}B3CRt&>?!CQA8m|XP!+T z4Wiz<)TgNmh^Mv{kMiLIS=bh*`4|7h0ygIEe|-d zR|ANzl@v#U3SWS8L~cJgNz z&aSoGFZL!r_SX981meRJ-orIXDh;WVgt)ret(tG*7i%xNkdG*Pc@f_oXxKVHzt*!V zePjxGqR+3Dim$#^UVr1eTjTRP%cH*~JUk*Sdtj^;986O?e(ksoG1fMP#y+jK`~NyZ z|J}c1%*IM9H@%en+XHxUvmDoH(NhbOH$3yD#lyB; zb6}^@-7j&{jQy=guwV@5k~SO}L1s-d)@2x|AsA5C02A;(Q}jP;@Y`#cFW&t{gW-p! z2R?N@xUC|(u6nRTr$Gx3WS9j3TMBzB>d#^*9?+|^2Xdv~9~2GH{}lDA|02!pBH%DT zo^aPn+j`*iQNlb~RATwCYKwoV=j>H9jq*mu(w<&Crx*Dq`pbSAGSt6sc?`2dkAeA;uA<4ib%aO4#ERx@?trt z6+K@4kR2|$Zzj;BCf1viw|_$H+WbKZ*B9S^VP`~xAX*%USz|<6;g|EwSMfDtkliJw zW|7cv2mSL$&+|x>Ggr`l;5GNN_+4cUigk@WP@u{KMGvrMQboAyD0(q>P}IV0j4$G? zIDns6N(G9%?sxqQl6KLItmw;6!?L&niyrIT$x$|}8@2o&PnOA#hnq5LoRBJ1i+hU# zN3m^h(7wJebh={2i<_k@moHNeSZByjm5E4wYII@c64723VLFgxZJIf{tsk1R@|Mt) z&@&+B-pc9RN^g3d5_B)V@jznYhrbrubVcrK|0c!sz%@aks;IZrm(SF9KbsAQHdLHa zE(-cCi?>{?MYr6pPMke_WxiW>@xbw29`2;RhKtE6*klz2h9ZcT@$=T-dDzW&d|Ob+ zrAsV z6V$eRSll*4r)Yj^r)*TRDsWtZo)_Z}`^A`cCJYO!`EI=}UFHh^!2U|6kp5L}{`3Bp z+0P3~67RISr{F$~X+Dk3n`#i{iD1(qRScT^9y6{X7>iQ4({3?H3$7=*Cb@yym?pi z`gmmqLI2Yy{?8^*hP{}IjXHsCM`wqI*C4}RC(e8d#t|)F+^8)Js#dZ6Z0?fFX}Dxq zFcUqKP(rp=iu5DAM8(bb9%!YvOw_(_Z1qTVGvpsU08dn9$C~tSm$+JPdq!OOTE6%5 z->VEx&wAbl8egOE8n>;o(*SqtulR|+~gFnS7N*x6f(GX>=eCn%zG`F~(;oq`afbop) zG<4$GVQ{OF07!pTALEo!b1y=JTybcx~~9e89`?XZz@1 zMxLsW4M0pQ+U~sRv5=vGu^7E(jthMKE%vK;gfUsgiV|y}5_aXo883>n@=A&qi3cKo94?` zO5}k~$K+0CMtzRxmA;Sk>^aOduC`Fl_1z5iJQ}GYAA=O;h+eHKMIeP(LC>g8qoBs ziM96cKGcxj#>~5FbLwDV4*RAZfkG?f)G)HokM3hDH7RM>8~a^up0vM+Nv-wMT>}83 z0rR7{hKDtv^%$@rusg{Da4w%^wI=TwLO3u>xfa9O0vXj(pz!~nK+b;;_b5%idZ=j@ zM2tcbj}niBD~VixmXT+n%&kn#Fb-M89fyOtOoC^e+s%n5lTwgiKGe_c#p&49NkLN{ zL3a`#ZouRvwV#PC!R#9%k(&Z{1hA*pglaavOz^|wcar(BTO}E|E19NwJFSUytYdcT zUO`D!nl{%+4Uc0z*aH-cE^3A%E4AY;oShU1W}NWtKNwCJ8Hlozhn#dPlV))9&!Wyw z-W*)r^w&T7{OOMppystJhf2H809d9kj=%xWGZm`Qh8c z4;$t8i!BH2B1{9Xb_}}bUw`}GUmH=nZK2LP-`s^YUBO!Ngc~T9@nPa;TT)3M(mPt^ zk!=J9w{O)_eKmLq>-iTF@oU!XgQDfh7FwXn3f?GMH272m-oB=QQ8)#Ros8d{M1D8f z^U1{aMOD-aF=%IpJ$SLJc>FMm?|uY9oexl{Y8>?4n3%7}f_*2lnU9tm8MBBD+Ory+ z4MP|`vjXLO>w(?HIS(XmHl^T(q|EPtDGxlq`X~C?Cm3Q?3TBWC`6AO!H&nsh+Hks{ z9ULj}nazK%!H+ApA@f1PW?B4oTGt(YJQLkr&XDAxmnPB}3CzND-5B)$@p|n6RaqBW z&nB^o6D5B65FvOIOG>lU4%w966&>8L0lR7QPB^_A;G4F>%@WajN!W&m)PMCJ8DQ!P zua4#tjY5QzN;O^d5GO?7jrb2=Fa2%xyTeu*y%16Fb7p75-uItl*dN6XWq4~6*)yr- zLF7~Of4ZGdrO~&^+n>H#fq`uY%nrlD-&9#!Yo%IjBK~>NhE=3 z@65`XLYaM|1&jc*ei~kN(_W))*%p1>dvsFR=<%Z^8@7c6S*`#W7rV#qa7%TpQ|Nvk z*scJj2E)k4`SQmju0c*}Cs-;%?%>w}j?|}?>-BDec>z+Q z$W5})S#Upla0d*tZb2=E^Hs_@_h z=C<$|);6C#y=Ca#JD-MkZc%}BCZUUO-Xh{VH>DrDUv+r|r@gi53U6GC;`;6Px5*91 zxa=Y+Y4ZZRUYxW;{+jk&(V3*WctUtHh6jgGu*gWxswb-D7IfvHB!*l)LeHwu9qedk ze$CbWM)(4e>mWUpP7Ax%kxMACLpx`^y1ooJ4#YsJYUS$QA&jrY=eWnVN3n<`MMhepY92&ay{f0S(u~5+2je%CD9_H$ z6&`zOe_-!-_?BD-(ej}OsK-@MKYLld;B2OPX8-j|L0`dY`6~#1?EJ)7ON@C?F&yS~ zsa5!YTOYlDMY-{^h{k z{mqnI!foGg_+vb_J)z9Gj6|TNX%z$aA?`tZ(cXq~08HPsPvCg6nkH(8>SWu(q6_Rg z3=CWifJ|xO4Y7;stD|Lp+Dv70S1H zYH7dj9OPr@*@(8!xqsS0vk8q2`PBR_g1z#KtyGn^K|kl@fU^3!^*qCIc?GGopgKt~mj8p)0e3l5L#BZuC9(VZe3klk?^+<@-#GyxWyR%5;HsmZy{{P49`G>_Zq&!b0CZa^@qkOlSHdq7jtaa1y_Ka{UlaD{r6iPFwx> zroyG2dF}-^QX4|ewg;ZD-NGc&%oOeLs%oY!uOZ2VsxD3pzw zg_#te*(Mq*QPx~6IkA&;Us!Qt@*8!X>G6A3>;A-bbfVSYsfel~VX70Uix* zZLCh5YeAR=ecw4+jOpOS;*d>AbAkD@_SEo_{V3{~1Ar%vT16C3hMrAGTw;tnMd#4- zJ01MWlcIo-WEAFaVG_Y#*3MuzZzD5aJOF3x&M7tZJ$gOst0AHfpPOP97N0oR`BGx* z+5*!UY~&FO=nfIoGwn1Ox2bYVHbe{2;&R7XIcOa~vw!l-7O-R2_?h!2kOsgV?V}#I z`Cl!765=a1f}_l1>bPRc-mT>9WfN*-1>x~GA#_m<0uE_Y7x4cT%yIMq&XNx_QW|25 ziA{=kf9!6HKSC<0f^ty$Su7R;7A#B+VYXT81z|Rn+TGV*EY*hiY3d9;#a{3avEhPJ zWm>b$D!XMDxk)ThbLKoDmI^!OhlOh?+bxwjMvz8W zr%r_m1fg%k?*#z%IRgSB*Yg~3+3hr7{5TG$J+>5lTm?V|*=J8c7p95k(EtktcT&=dbsjwa^psbt5X6cVTO}0xUS>@`v#y|ngVVDvXQ_h0P=z{3 zspdBJiJ}VFuODMe&*lH}A%ma~xk|jedtS4K&d%N%mEU6Z6ztrFzSL_np9EaEb(?{V zv|q}9*cHu8tT1ixtlGpQ+?=~UW-^vC_@I>+PGXF&zlGhT`nE;{wrg?a2qRc%syLNy zd}Tbzh^FRn;j@7}-)M{{>;kw8mlZ8fbwFwdy+ikZ?+p_aEJ`{9WmO|RV-AfO>vX_W zp+v8;1sxLbv1-Zi%cC?F)Xwp^J87$3gu3=r0o)#eCxdpUD09N=8)>hV1qn(HW;U_= z@jBP`x%=q(RTLYGX^W-eq)ooSX)NpsEb`D7Y^EsQC=t)EeS=5R#)D{04@%;! ze+v+ji{fY)gHQNFwcn=PQriMu7DDd=h5P|GeAL&h$)(!E#D;4zG^C81Zao*5l_lnD zJBbjZ38{b+<(xBR2kp$t<(;A_F8;ii@cE zWS&?!#B;t-6lkrhyx)0E;8@;*5Pb7qn~jAEI}%4^G+xDR?RY);(7R^mpCtM}Gdz+& zw>U1oh>B-Ol4w#@P&DLHg4gkdtL!{1C3LFH!|-e}X5dRXZa6J5xh>CXCN|*ie*fpY zuZ8;Xzb=)3YXWp_Rg%&}mVo(6T5@tY@C1e+g+!cB0k~+oy6rLy!~_rqVr-5O2n$g5 zx$R`W@?l8gc*dDyldtr)ueAr=Sf-$wl}6#~L+SGuJ_pO3J_pxg-&fsYH4 z*Z`CE@9m-!aUTK1L3O_Z809K{mjy!g3%Z;>+Sw4EZHlqjYE)I>M=w^1J8ibR)F`|! z(hpy{=zYq|YN>u%{%Xx86d#7Wt=^n7Go&a)v?6M1{=v#e4uxN=13`Re-9EuCm8tl- z{^X`o|J+ePST%N&s_8%@6xqJiZ{9ZfU4@7NV2^=&#;~p(Tu7R>;~s&74wg1UxStOh zrl4D)of?DZ&QAO(x7AFMt0((*BwoA6kympn^5^#)_t_WuLbXsEi=N8T9fD^wYatr* z&4oqp`hq|%a*75YgRE`FBLp@y{W@b(@BgG14iXc@JO^qTK^ zZpT0qvB=5ceQ?e)f?s;d-OC%4lL0fvnU2Df=|%Bx+#sl2Cd=A z!oQ`}XR4Qk;yVl<;q=YUft-=07EfL`mZ#}!dQT=rY|w?WG_?44B(csw0qmCFE8}a7 zv=uQrqveh!2+enF*$trv$zK#d1sZ7-GUB+Byu#QEH$I|;KW9ciuOTwUCDn>lYICCr zD(to7q1&!jP!}m}mrXJ7!VDMfFfSM*kSwH=Y9R2TCn&sV@YV59#K8n*8v|q?wtUcn z7;PgjGyYyA;^()7^1wfYpp|WjZ86m8)uXQaJn)#A&`SK#=N7@~=7O1I+~`s~Szr>8 zH`~iI#I1_s+d^2cs-`+vS;x;`zaWe#w-4C*?wDaEv@QVgC-{kzXBRaTKoG2{oGo7; zhaFUSF{dlZb&)ss#FxJ>2;0N9<6?y3dd>PxGX`aVpBfPVg+9}5Snl^^vpwN5ty=bzg`Tt&51=IIT&gbPfSHFJl)upT|sd}2UhUArhm$MYr^~&jS z>7(5?rq0%hzaBViX6_tA(s=wZE0B5jZ7Pd$Gk`Dib)3y%|68RF3 zuO|n{orcbs!-d*0MBgzaoI=|Z6zK-M-Xgr)P%bhQapZ6lOn1i6$iniv%|v-`q}cT$ z^~DvIa{R(DBLk}D)cJ=zdAnb&Lgf{M?KWn>XI(v+WS=q^BnwP6Y=@0o-mtn`l#^}B z_2iAHL?2lyqWKp$5ozL&CFCm3N4lBM1Yi5VfPQdjE|W=WmU((b&7}WCAy7N)rX3me z?+OS+v;4x@iQripqN^ciqS7_CQ-#NiI&W4DLi|Fa?f9-{q9d2f;Fq~*?H%-$^~bCU z%_j&FwXlBNnynEI4Q1jwpfsb2bpB>EhJTo)IHDG{CMVdD++ zYC_;}eP}NUF)xe;a#LrJ#%pbG%=Fc`!jEWyXa~x5U;M)JJW4Qs$4N|$f&KGyr;CM^^?BV}4GO0m`L&zKr{4sWk zni|Pk0pw$IxoKnad@$ zxkPFZsuzYa8^m%n$kBj}rw2mg2rdU{ZBuP|});$?OqKU!E z!t2T>qjlLyBW{e7Ip@@Ni~F^fjaTm3o@nGe@YYulE~B)G8A>BxgvBat8|zrmjr5K1 z!0z|PDY5kOjJi-}%i1vb@M^&Y%~KE9hSV9U-5l70poKJ>ne5*%tm$dkh8os0S&h2QIVL7fU+{C1Piic=K zo`BY&$tg=wEA^i%m`{%qlzAlcjzgTvBf{GYU7bu>1(^8%*Qhc98GKvKRtFA>TXKK= zyg&aUPM%QA@_q0+dkBSz%DIaBSt9Rc;x=@eJClKu&!sy}_EKZP-j<}Q^3&!6c!E9S#8a)cyy9nFmSt^QhNiRw;5CVw+@M!iGtoUsQe!diCvfgUxd# zs62ZD|KRg_)BMWTD-qzVj+|J27oYYSoolt)Ftiya$7zxkgpKr_I+14S4pikst9aM4 z2PW9c3}_c9P-bdjz@W?-OHLNw!e@=8yAx?>B%snNexwzQ)qjwv=}R-{_2r0hRLs^G zvd$1}hz|$sL%#Clx9i%P3Vorv2&CS&IRPdmBB|gSII<-9-(|ye%AFrxsx^b%v>|aIn(HGsaW^js>B4jp8fCd42@PbNMbV8`t@em@Ng34&Vqe5 zvCR0Zr3Ml{aYpm=V(Q{S6wA>l&34u6?RZDGK>7TAx5kg7t_d&f*#P;b6xl7jR&BZ*G*#~;$I|!W}As6i!>%XfiQN&F*&M$JvxsE=*1yH{%+*{^d7na&-Nj#8nO=^yU#Gmb2t@2SyP2@~pX_`)mHUwij# z2~fF*&5;YT!4!#nRz&o>r^>d`CJiYJoVYSE=(W*$Yb*<@H6dK^@$0ZK+>C$**DYQL zWtt>hTBa^%XIg}F?&!*ViN1%K^4fs$;je(1HCl(RZ5l#I2cK{K3dZURFoNg=(CEYWRJ*| zwqc4ehZ{i0Z&QxXKt}3+>AL?EQ})27<&X$=cC6xs8GS}{WyfjyKK?~KUQc1AA#UIT z-n4BM34#dI4i*sXRgYtN6Lnyo1Mn)i}VjDk+isD4w?qfCh? z8<{0R@vpWx0d^Fz3PNmJ_${Z5vj4qmuifRo}D4Tdfa>;@EkzCWbWC&-bn#}ot1rfYFMgOr# zZ*)EjsdD_Ykq!`BCgKj;lAC)Gj-Il^;GSKlb6+pw`WEJ5vrm-8#@}L(4+VHDomBZK zZCxPz+y<29v!g*`KjL}--7EtmXcPE}d1=u2vA`^}xkZIq%~hZQOCiF^mNLkEMI>67 zJhMzQ@vv&|j-$9giJc%3LHG=nq>9N>^D=yp2CE^?Wa$5uqrGBy$-Yr_wzaZLMmU0m zfg!r{hbsn7dG4BX9MDbOz`#sXFnK{tmGqRM zL#$W}v#$CuaKfmZa9OC9Czk;u*kek#pcKIE`*F8SU?{)_`I9pBD+e?xS1mWc9b85> z@VU*HW0Ml`fZza|PGn(mI87rU!O95@?1?u#z>FikF$EuPRdtZ9GVSxBb3WEimLJh1h9sQ_qV<&K}CK2{=>Pkdh| zZ2K6$b?!{?zn063ctjR%L~k@=d5T&6+vekMwQrC{ih`O6n>3+s={pBVRaJ#{2Wy0( zYVzi8`iMr^KhI6y+mpU&%O)u|cz4c8Farr#9dN?g!xWw0ei*H>cy#8P!o@v7%vl-s!dq)skSxMrP?h1^E&n^|1kHKJp6x82n%A9Y~$OL zKf<+gOUemA?T(*q`+|FZAyGq(I@f8T-?EX8sRK5{QD6CL>IAifqwU|>2PG`_*O@jjesEAsZh~F_$$$O(rWm!Ff$uAE+8}(TvJRJS2 zvo2;=Y+gR1{}Z;k7(C76d)d?SbNqrtmJZcO)<3btWeGdS_LwPn#K4ke%9e0+8j>sT z!mT7aMfh+~U$2w~bJ^br_frIMH|Pa%kHd5GgT4&jX+ay7{RyiZYs~sR&)DH`OqJnP zwxvp#rIQ{NjFY+^nYVq*z2Cs`y6=EUm~#A zV$p15`Y$jyyFELJ;dc^aTda>raaMgcl$9!AaF{!)b_Mh4x2ibb>dH-}US@fzNvB+3 zVBLmL;{%DKvP|L1g++ZptBbQ?HJzw|uP3@zax0KSy8cA#X!|gD$-bL2vg-f1;=WDl zeOU5+yVmYA;l|<~=*mp`yF=SF-)CZ-r28=y&&T|C6$j(OguIvh_3MpR#pCM_9J7bU zy6=I_gp9^6HL*oYfY!F!CHoeOrT#syc;FgnstCPn zA^&xL`p`FI4o~tkaM5e7JHC0J*_LyfyzG;*KZ|gbm|PuhN+IE(N(VC)zFGa5_0c2^ z**SZ)+N5+{^G!1~(sEYvNf`&u3Pastq>U62h4EQ?C7$3?dq%6Dp{eDaq-f6VA8Qu{ zKf^tu{Qvsim*@Do!qBjyo!Kqs`LE%b;78hqv3>6Euifiu95;^OU}CM4Fr-)>!)8Bs zktR&oTM{((q#^6|L)n31lm{sL2h#RtPNv%p>44;-g@1Nj*c{Hs455_WobvvXN&113W7U(ykh++%*GUA%gPPP@h`Gh~rEU6+ zNck7Y;`F<#qx3k|0BvelGQ>A@H2K7Xy%i2##HrXUW3YxS-6n1Y8o@5}Qs#$LjIRC+ zFx$BdR~k;rpsR?!USkA$y%jWg==d|ax22a;;w4Sfnv*h(L&zhQp8*iTf-;$7K5*Ar z!xp|z21~2pNzjN~b-;edzeI|PE_$&9>^3XosU-@UTb&K7LcmfT1RSf# zvbB_a`CUHbaVO>b;AKTBU(96!B&n~lD=z1hd?ZWeh;4d{?D^U5KYe!0bWxc7Y~3?r9#YZt&lbHGFm+ zm#rJ6Q2Zvoz8`qoVR7bOC>4siEmGDsfpVOpPr~w0Gs{F=H1X&QgMSy8z=u}|IEcTj-MEGo+_H}vf>KyubNM23YFm* zCFlpkY+K6SAiVh^)h16B7J*j_0hFO~!vHP_)}ME0_-|Rp93T06YcoYt1OYxyl=S}& z7h`|Y*6JE|q5o8Gu|R9O1$q7DBP{Wx3;V$)Xh!Tb;Yd>Bl8A{yYn(Ej@wtXtwtwsk z26@~M0XuyaPt<(o394u(_~2u)+q-NVnI^g}Fj8#_by!Hs8^P@zXQyst(>Jn~LS%Uf zMqr4UoFl3ETv)wU^c(m6bEhyqb_knw&sntZXUG4v2#w?yix}%{>hN$c>WuM|c(wK@ z0Hgm#dGTGZcB(v{K;UM*F)kZY?#GR)BXVBDCIbE7HnHalWxS#rxn<{F2CG0xbkY#z z&}*Q4zDg=_*q%x()&RwD^2TVc#p*KJN>*eZz5X~shnioPCn0a?=}fF!$sUL?t_U*> zf=5^0t6m#GR%~Kywp2_ScddJ6SCD1%=H-t#26c1%R~G&1e;^)AVh6>{UecjrUC}K} zA^MuNtbw`AzQ=-N8O@>g@JwTU$)#NbDb+pzfT|_}0T0hW>DR=07>!p}0G_FO+|^qB-rO zMptUs*A^eo05yJVUya4hf)(cBSgmOrYbMiNWzQ&azye-5cc^3OG;N8N8+S6n9J5~k zgJitZYkV7$CDGb@>ez2x4*6K@qj91*p|F$CK-Yn#Z^T;l9zD&>kiRGJ%&<6k-{P-V z6(yN?5wDuxZE7LVY1FC-{5f^-dJ5-)uJaSH#TS~L{(C!{$Krm304vTn%e|tw z8{a-j-}(EH0$+z^d>~stRiBh&Kf&BLM0#*9$g&XtsY)6~B?$&McXR76?EPmt5yJ42 zO5~iB%6`u8GC2hKuP1CE`HS?}+$<9joc$HsMU>SG_VvF*ySagMat901_N5x{2R34V zU=QB|xs{dj6)Dlh=EaApeqFv5j75-fSytmMG(rs74|L-8X z11P`o$(KBVXrdv{`w?fuwxTdVf0*NmS>kT&3=Uu9k7AT|ih0@*rOU|P4fE@pskVzf zjIiv%?o>n-~w`yv-u|6>UF3UXEMa znyP^K88kF{7B5HEGV{uVK2m{!&lr!ffwv({VLo*Knf3;m0khZnxZzvS!rumOVb*_m zJX$lkGCCW+f7p|F-hKPw4&Fa`n4Eb~wP4!uZ|XtvUvb|!@cpOGkGS^&*1llg>r;;j z77yDAX2;g{

8M?s4lcTAYxpyp7-JN2cA^bsiyOD&{+vUW;O z`V}n<2BsDNL%DG~tP#9TSa`o!A-Zg(;2yj~`MeF+5*R<#jytSH>c0FSj`(yq4Q&Az z$M!+srGOt6cjE@3_X~xsR3di%gIj+DzTLh5?DMHb_OH@Yu0z4y3vh4nbzIN*7G7Ik zhbO8#FAo{|@5G_|?>@X2K}e(ZYC>UUTQGwC(jWdjBnU`fk$};hbP=ED;l$^A2M&S4 zk8Gw}7=C-kKRU8qpfHG85;Miah{V%{f6vt?gO2&E&p~TP6g@ky&OGJ5#(QpdvTsSi zigE8GHox^5e7pbt!cD&vkv)Ron)pNQoL>kA7T47VA-ilAU(Q!Hp8k^doJZp)SK-RD1qKtoDXK#!=l)nu*xcT<{ zXz}>nqPrKFbF0RqCd^QeN2BQL)7lQc^Fm2b@&f=`}aHAB*B>Q*|Bv5e)s5dpbbxDh1iJ`tbNQc z=tMsM$-R*=3XYq7*fggite~2l56iF_)8&_xkjc8> zh*T$(*vc6^VKGLs1m_EKW#vNZfCx3lJVoI-wqFZ{Pri)k0l|;oKl`<~?ZwdLrZ#sS zsClHf5X^i(gGAdzwJO7(=ZhP>7qt|WN@B~J7#+Tpk&03Lg!3fds6RYhSeCrb>AB3RcO{Wf z-v-Y&z4(W>fnzEkzoSZ4+CENvd%(Q#=6~wlaS`OGF0SIjRyd^eM#|<-?duDtN_MVw?`)@7h=TQMjPy#^N^GVzUeo3ZDz4 zCDPymu+~#Fp)HNw()ja6H}x=9v*FW$la6cvd=`xVVO0f>L%86_%b2yVPuKO&KS#v% z(G$^C0oYczjk|uMbTMzzHbt*;XLPZ>&{-mgFd%SXTtcgrlKXC7LBhFHL;UV{jOcH= zaVlM6;ccs7>1|s+AB%s#eo$XmDikYiZR}9rjAqymebYdI8izfLN<3`gjwDCjy@=m- zl0{DC1v=&$8Wnq+>tg+c9=VKUA@Sy~1Kca+fHDf3gmZ$Cgu|Xj)>q8YYY6g??>t90#uZ+{5{c;X{gbeMHGkqs{nSP=%&K|x+DC|!Z!>o0vY^TXLft9;p*gGItOn{ zS$FDGsB!Lzn;;GV5f#)!Z?|GTz&9fJ#F+gl5^x#+)aKEE4|a;BK{aP1?MEP6MkZr? znF1dC9N3G1TG?NwJoBasO&h!qn+=<-sorF8hR8@vv83!sj=+V1XEh?0+miIgIa^B{ zKCBFlI5nRGzPxkw6toQY?vi*IkO-bjF#p}@&qUIM53rQSC*diftM`$Q6k;ZtynmwV zy~E#V8(=Q?_MKN^wo{RlfrccQtd`VyeU^PULgbT~$?Bj8x1JVb;DWYe^P_ZO=>35s zw%Fi6&7v;N7mSl?gYLt3^?_>UNwj-2@debD)o-LAZjR?42$4pE zo;`%*o^1|9bNbSCOKug_=*;?@xL5>rA2P5TXFe?J^^p&_nC?(iQ@`l1CY5x~KtDRm zmgd05N84S;#+AgNk1ra4Nr9|J04V4N?uF17z7i7o_L{N20{W z+Khf?&`Rt-GIIso0){vuAP=tNem7E)8q9{<*qu)!>{WbuC->Zgi)bk7kw`Y35*OJg*zK^n zb|y-&ECw%zq{cSX8--3U==tBqOgKQ$)egg0QXUC+n7>95FNnf_b%j& zAtgsOCzxZT<0uG6lW|mZaR<0CvZvZ$>Lvo4aOYIRULNEHPHMOVW^EKE|M2#uf?3ez zx!m0FP|vv+kl4{^9yZ}~V08fV?=8AkG^@$Pj%u6EiQ_8MOSgIJK0A{38V6>dzoaTJ+T&Lu+Gw5d2ynzAWS->c`EQbU z<~JBKk+~PjzHr43P6bUxa(s9g`A`esv;A%zd`PbM$1@bc0=EAJBdRz%Yv{cdC`>&H zJ>v1-x984$i+3^TpSNR+Cw5)}Zya-hVk*kBUnpFmI5d>+P5RY_Cs~WV+N>lU3PkecBixP&Eq`w`oxcd0xwr7ovZhsF@8mMAIvXJQX z2D3jW`A?HG5fm#MDbQs0NO7x#^JG?A|6@&}y6udZ09p>-U zh*5@jgalRn@5;khh#+?1S*rFi8 zHlQRWd2~U(+mwAZaN>KISME6*Nvnb`Sx_6|Y?Ru5 zF`el1&P{)-`>LNgJyd>o{!#`%#aKa2(cG%K2k*@Y(-_&tM&ep6NO8Iw=W3UW`j70uEsO2 zVlhV@(7Ne^RfwWSe^{g-l2rq+)cwwEH*SR*P9N~?-`b4({6RF6xqO~Y_}B4pM~B$U z{OU}s7O=}Zrhu%rR&$?#J|*&t3sL-N^!`~U3j32!uob&M#@f*c{FqROk9*bpUDw+t zZ==2RG$WK&dDRNxit(R^yA*ra2%2w`)8r)mElhO10B&_)C1B%`dxU6X^JY47JG z)AxSzRE~aGtcW1{K1{pdC1L+k`W-fqV*TcBI*p~bb|iZs8bAC`)7>y?nGsa>r88oz5t%!>&mYaOcQ-p!N0}~CZ$RC;XE)Y9j@*4q*hN453OO0rFqjN-N4^&JHttz z1%7}($bME;a23vzp^UjOa+t_8mhVZ@&>^kXXxpLVM413yXPV1#~rEBbLdUDPULEok&SMy{M z&O5)|EXE1Q&M$~{Q*X>f1bmK?mQhq=qWw{x!8^#Z=yE8Cfj2D$TNR&)d&99i$x@L| z{}nmXq9t4lNAfuqX}Wq7mYLD3%b=KDfVNb=+|Py#N7f25_nN&_Q5a)p;_2Fufs!D7 zrOL3`j4P-kTd}pv02o_Ur`!W0y@28uk$9Ii2KLMKv0OV|Rxm%!WsIYbk3O?f7LQw3 z(upSx7+g2*egj(aQ&741d&A^7RucIf(9%vs(+ZMrgdg$>b-K?&K2X zE7PT`i2*t$?YC^(D>IJ9Q1l{~(*Jp1@h~Z-|FdzzzC~0Q_tG@wnN{#kGVX&y2_N;F zox~6Bd^H6R>t~Mz{vXbP)aeAz&Xsxrml`u%Q?jD;zgb={B~a1;7p@Of$@~6}vX_1i z+!K$Lyl?nt5e0S{Td&Mh7wejsWS6myrQJUf0*}|eC=-nPT`c|?M?oupjbvO?4Fjh+ zDRc&M-Tt$5D{t8*YniRMBy@c0K5?^uqW&YP<=>yvNdI-iw(GT7|56Ph89{GpeJr~a z=qyD-Pc$cL7`N}+WKH>tOXPAYDs&Bx;7rH4Gs@60$k~?=aurG(QNp?sb72G~74B#T zt+aU&E1csoWK`|fa$lJ#ys^ForV(h!D4vx&s6)g0R6@&Sh%U2Pv8D7|tSPs~Yras? z!`hnl%5Up;>&%@8>Dl&6bO(QlY&?$7A8gUh;J>U!Gt~VHg4Y==%h!dZDf_&BhZ;Tp z!7Cq^R+d`+)nFxa%dY(sPfCeTqboFvNXGoa@y!l_Hs{?OUOB4HIfZEN7;|BwrX=_7 z7=?%;o&sVvQn&$38YoZvDgArx%Kqcu6^;I?0Q?H7pKqsGgJ*lmDN##_O3Y;<%e3x> zDLpH<={mXYDSCfmV!Ue#j4r}MosTX?l#U#cH$@6NT7N(K{77*3LV4BGc zgwUv1F>l6axcC!F3rSb8=@4xCxs0GVFmI{MN3z_1Ltq^fhM@vRJXS5NGgZU=i0*P> zZ%yU*>gN{juYZw0zh?#n^RdfTBEXkajUWV&C&`hs;>Ak-;Na5BuBwvg5KoBHa0l~P z@xS&0+Kct$6C;}ft_2(Y+Xr5O>Kkx-N+`RkT)&Qf>Tl2~+cywDw=G;|eJ4}ATt^9! zr}X#~@BMgd`cLq-;|MiY~wsCS{4hr|72#+B-yExV}ZSp zpB6~*7i17KJi%?7)rbSa5j~guiVDpO3WWM&Xxl0Gz~GykxE?K45|+~^o9KN6|3GXe zl|TP}pH8Lfl=3TFI6{Czu^f(+g)lzdj!2kONSg96&`em}CB`V11z)1x?*@b>kZyga zOf_N;G<+Wem>+P9LaD987%p32{{ZVPbClf4ldE8cD!g8#Jr@&=*jWixpj5oJjHuxM zoq8+i1eN7o9N|6+vQeV#Pv_97Jzgt`=@pbCD@5wKVLCMl!J)+w+yqEL^@Vm~8Ilbf zzc5x@`lgWjReRxOJ_#jz%}Sx(fh{JDfDAp(HaHgNZWxF^p_N0;+5FM!GNh5l1C3M9 z-OrC_9x`M7B3j%1=fEh$c#jFjD1C(br0?OxTe7NigqG-SR05WNh<*fY6CpX2%{m9# zn-R>bN8L;lz=d?S$7UBw#9d<2yo)SYF+^0Jf>)Egccn!;Rj6c1bd8p`#;V=HA`)^M ztsW`hXQZZqsOTm+I$V6J^R&5|k9Szxw~QalCD{$=S(N}Q?%jmKB;HF#4j0M@S{I0} zZ)6t^(o+jX1R2o%FVW`H^=jd|&`=zX=Qi`{1t_tVPT~yUM4UV7HUwArg1D2cbqel!YtrOr2yo^yK zM*Je}=1!mCMK*WTM&y{cQ_72Flo9vbKa!^Jj9|R-=-k>jd zWr6jh#u(k%Xz#h^&k6w$mMIA((&`j`7K9WfVE964v6o?r}Dkk z(z@S~%oetZ1V|Q56D5q&-x<0Lq}61-vz+3Vzl>13D2@kF=?aV+PNIwk-rV2}sLWv5 zfgUAUN~!a;4jO{hK{#^KvFC}B{?FP(Lx>I8v}}K)S@q6^5^2^nzn$=#BFf+{yD4(x z8_oo}g%MgTA|12Yg(b&(e#e{SlG-HVuhmF79<`wr{6!LZd}z~=Fo?lnUSwFXk4B{lH|lVnw;c-9LJLLlJ}7zNIthA z7=&25`*U1O9PA8)-*d;i@qey=XQd#K9*-d^ik?mFxB9Wdeca9(7lDbLxgvPY1aQ$( zBXcLD5-&qFZ;@sJ%12MwE%D!x8v|6YbI?MP>}Qm!5+UVN%Zz%w%!#@V`uiL7U+&%p z6S3$o%9`4EJt#D;4T^LEQ@w|bp4qBXVv9d0bI!>|y&(&wqE`>r)#fX2uogmVilGT@ z_Yp{>sCF?#?C&3=M-#T!*8Lv)XfU`~s}%>d2wC8Xp~E;q3+vZZ`Fl#B?3yZ$gNQZ! zeLt4a`g~5=Z0WR+?o+B@;Jh>~!}-F3=imSAv+yEz3L;(+jp3lInjPP+&#cwunbjMl zeu7SGhDg;1Y=xne1les2b_{^MXpVyd&hBIsohlKF7BZ04{XGUMPH!27l4JkaNjYf4dRG6wcTsOO4+xI16m-N+iX=lA8}EP#$?LK0a`-?ll|t}G9heUp zx+;-X=f69B_=h|!0|3Ey74$JpKa-UFD6@Js6mx;xPL^(LzM;($&H?j85kKlKUdX(n zH4#URPerz2o39GgKLE%N*g zQ8HSCF)f$acp)|u#V9VY_EaA{7)0CgDyu!GY4k3`Ss;V4#BPqA@fHkm+@82n;d@qr z#{Q-@8^!ShrZ953?$@z2EMTrP`(K9Q8Gz5H85CaOa$ z4n?Zy$u;;RwTz_P=DB)E+Qte z7n7+B)9}owAe3;tr^ft%MRk0seen}pv1$8eTxs{SUM6Ta1GUsN;C6hfB&Sfl<<`=A zK5~Wz?Ajsi5Mu;3xOItkzx>ZWjy=#%1!x~l2}_193UZ8Q$g|BO=$fjYMr;~8RuNeB)V!|A{Iq7H=pD}q?TKC;gV zd_mJU+ym!@-sk#+QQ@ze^mZhC9H8_FKX+-XO&BQ!UcEwS%w6EX znLp?Gp;TE$6J7`n+vojPh0nqGMYGb?&gex}jmOUGTH_>y+wA&+fMyn}Sb$i1;4W*h zOap*#^)kE7w=u3=I*tb+8MQ@S_@InhZfwxH@R#6|f{~)vaPh+T@*lrW^UL%(hLO*n zQzr5Z?j&DB=w8>u3hKZ7T$+y=`lk5+9gXN~Ix@s~&Dbj_{%0%1d@*lx3Na-GEBP>S zL3q7e8|X-=tN=&0LgZc5<&Dd^(UcJ|(j+#^$RyPesO@`Q5eeDIUhxn8iq z27u+bO91Pkx*#|H1HAJS-kYJfk8zTB zVwgVF?5F?5{{_vmGF&JT$^Oo~+FBXAzax_?pI)o%d?<*^$mOU!iU36g3An=q z*iT4>*_F49^zp&!LUmYi3Wm7k=+r-6_tWd?r>6hN8qiU}BT&8H$WO}oB9Fd<&8~s@lfN?EIWREkciKEnXPXWztOXVX|mi6kDt(xe;EY;gz02gr> zS<1Spv1A{Q(K$R52Ng z%230Qgg83WDLDzB)VTg%uYF8fM+2FOe%?DHOUpTu>36>Tv6Qrjl( z65Wjg-&nl`fj*9u%wiNhQC2oqnKo=EMjoP=v_jZFEjycmmrltPsgs#^&PfeU#{He0 z*f;*jH*wzLI8wnWR%h*MGFZU5@=z4GjZ&^~)*``%g)=4Iy)JaUCvMO;bnK@Lk(KC~k}J*W+uNCcrsw5xMl1PYVEIsNcj7id?J?M z8vbf!lze7?zDPksBZQt}efhyJj$*3XFv?^FvPEUqaZAM60d6B@`gjpAWY-vJ0M(9^ z%i(P8en%(oeLz(4q8uw@nmq;WCm|Brn^|K%nY*|QBta;be@{7=7)*iazf@=Fn`>{bU1LQfSs^MgreTsNs0Ary~F+-(@zg7t?a{qien4W zI`)_kyLJ!dOLI(FI-dQ=EQszySU_@I?afgYzn@FdXR}hLV0WADmSsnnC(%V5$W%A* zMAuif3_ygE*)Ac(+`4YzmR=5YT{@>1lNVbP^pk@E|NmJ4=D785pDS8-H?ausGOn{V z_V5Jqw5aUZ4TuVbdIh>=DGI&6DI)%PRich0G*;fY8T7A!R)x4hRnG%LIsQi0@hU&I zYDta*ApT*tYf4jj+DSvy!0w+qyhJ6F?~4&x4Qr#qMDP>um-e*GZ*!&-uQ30tg?CAn zE=X3go;mZe|O6wKO(t!Z@PE;t=e0@@fb60iZbz@D!cXhRJ>nbFctFK_YmHcD?Llj+Ix5< zn0Nxf?a@1qgdZV zY(IME`;D?G$k+E~8DI$k)rlOO7I4Q{hTQ`VuCc_cxQH4Pl5=B=_4_SJtoB>Ylch|r zqASb^c3JL`a0J5$$U-BE8AU-D0`992S4`k+HZi-CC&*MtvWFw2ze2(!*pTO2K7Dm@ znbCNR&L9sqO$9YCBG`!|TGUBPWO-o$rG>AXB!AkFb!JBpUxZ6Em~{R6NM=lEgx=@I?T6! zuq2viDUSu%zn-xQ_DL=*tR+;~`Qby1ZAya`%7stS2{pE-x1u^JUhsnGpFA>fb)Paz z>d#WJ^=AL6$iyP16-BXAl;y1%@o#B+8_C;cJl`0SgjQ&Vt8K?!yuiYHjnAa&gjP7h zFITwt>By?P4fz^kVi_{MR{;5q#_T1l$Hs#rZi{S*XSoIcEF&4J$v{*94D!zRVd}^j zKU7>EJx)F1Q!^sc@D}=qOk7cx(peNsLDO-~kdIR+vm*+Sagh7YPNnq6OA9$Egc$Xs z!*2Cn#}H*zI;<1g4+KAKK9(pa472-QU?Z;*ue>Xr-?~(HyXA|dGZqgf?MnIPxZwH? z59~NDw4{F@MbO;zbUl}CrdUmN`24pYM_iXat%V6<03Qx9yxi=R zLH(4L5?mG|UWv%7XB$?hm_kTXyc_3nt+o}Ej50}wh)U0wlMO}THou)`GYfxQO+-XJ zdPn7cbjJ>cJQ653`FclsOcDo6NJr`WrC|-=Enebvdk~)5G^Pz=AO?RAIg-ywNy`=l%G>1JVv-Zt(^|Wy-3z zv;Rc;;?bohWn=%LU4mYv%$G$8sqb!h4^jEFj zJjyJ73iP}XKKNORiA;L~ui$yi+F9R0uvd^vFjh2ffJRPIXTAJzRIdOsKYOlU!5p!Z zewtMy{*-?#OvI+wdfu#s+~sx0oNIWqWGjZf%3B`T8BbTXV9w$8zwyu#n0;E{u(E~L zzVYg2$vpyl4LDM~u+~&eoivNWJKf&RFUK0p6{eSGlGn^u2bCBcDx_>4?tkiqU;8zBZ%=3 z4)-b1_4pF$_&D}QQZl#))6S;>@6x+6C6Z^FsUo5yVy_t1-&*RANuG%z_k`b6B5t=# zU=JS}yKd($VZQoMnO>wEH{v?CnOG^Dt-+!0u`Ij7qJbJd;VRW;@Q&9k_Zj+t| z^b(H8?Js}l?Q_rpeS=Hlmx(+IIVfFq7D4?1gmujNjtzK0$`Ze%ZKp$D6U(!Vuqkp6 zu&FjTh01pv^=YALJVNX$3z+qZLXk5HiZLXS!bW`E3^R$$tYVRRdi|pz{muS>b<}XM zK=ddfVELe-U!eIUwS~Le$HGie2YXcvv z+nzy9V3z^nA*$Wu>TBBS5i8@^Vr6}jmCn`k^BjZlH#OJi6wn*Os>Nn1;{MRtCnQKeF1S?ces7aoSXHkf^vh@!YCYsy8HdJ?hLFaJd6Y=al$B5N z{_#Ndw;O2N)+~iQU~JMGGqGyPlGK!}wRA^u-PJjFyA-Pew!^_u>zOx-!6<876^)gU zJK*u1qakZpcHs6FA|22^=6xTtgLoAPd!CAXPc%Pc^bWsx$K>pNqX-lv&}#JSGQ4%2 zKYB+Z%%J&JH5ZQ9y(9nm?!bg$9G&gIg~kt0zM=ICdY7^!_9wW>abMZ`4$l z?VX)wQe2r+py5c$JjIjlA?4FM3=c@a-OlI_{ zDyj0Bgn!+N7XOS*lBLM^prbA%Cu)@`lu6tLW~lF?l#Z&0GnXIfW1)C;DSz9enhQ~o z5vkJ=d8q)tGa$o{9{*wp9(tW7W(0H9(T#am^rTiBA{4=~F;QtzCdfUbA`8WmesGMH z{eq-F94yP%7!jr^sF|~vy{z(z(Vp@AWxQLgqnF_vmd^{}^8l-ma2ww-HlDn<1ci4@ zn4Ok*f=S%IH`|3g*K`FuT#Pg!OGX+sSHZ#7dq+M~Z#y9fzWv1!YN1fg>`pTk27@Qj zAJL~s&CI!28PHieJu;S_>OkjbZ;|ODA0#GmG2=;HIh%e#_Z_pZ>TPk~Xh?06jeYOU z3^JPycU9hIYE|2yp%h0f>F|emI+hZy&pzWxm5w}fPvKLx6s%gVImGiT!uF@;sZi3M z!T&QP;rJeneeLH|Ycl@+@Q?0#s>z$rkK@DbZ#M#EcFgnDVO%k@w)m$9Mh=d)uj1OT zhR>c>Y72E!9jXyic^31ceSKHQ5Z@y99@MM>)ethJGiw6YhS8bxX-yn4+83q&Y<`Jo zPAgxf#I`lP%*QI%4w^>CTr(c7R2^@X`f*b>R_NZ1JB;w4cVHvSeAl zvpK*^Q@t827atnpoU8dN_4N&PjGo?{>2Ai6fk~nJbnNynJOG zG2HmbK(h&%1}K*k_2&{KEEWxb-^8;32f4WiMNuW()`V5pDZjLwJJW=PI=SY z7*5vA5&G-*r)HW_b5|&+U}OY!E1}MZlUfqL(fhi}k2T}=??^cazPMnk{Pn>2k}Mm& z9MZ*?$`UJxmKbOOs>b==l8?)cJ;-)3RX!CD&^1FjT*%bqhxgN=2 zxWL2=9P^s>E2GhITl9lnI2dzJ=N)~p zp=ktv6m!mloeoX$#HY-bavynF9&%@RTW=JDHr&R&%jwtp1{NI!FGm@6lK|@9GWx%C zzy<;i`?wJtu&HvjtkOKNOzb1MqgdSj3Wf>9MO4)_WDR_kZ9*jso=Ey;H2_XOnNoMX z9-$-z$yC?=e2p#5ky~ubc5E;2p7LrdMKqZGHFjV`E;YT^VFJ8DTq*sQ)6ZoA*m6k& z>_t=DRNX%y?{TX`MYYsQDYiemk65chrMo`IoEa%;UCN;AUaECK7+fvoZk@T3JZ@7|btIZgZa|7yLy<*8aWnd(hb&Lj zEQwyR36r(z7FQbaBz;&cd5={n&&ZpFP>}IAtjI4$d5US*Ng{r>FuH$F4lymx@Zk0a{#~? zH)fea5|a;GDmJe{PaSiD40<0Zs{Ej(_tq9Ww|5ReKR?!Cw6p-+M=0eH+FZ)+Oh7UW(XgHGndsIz(5q z3t+)sV}&JS2HIfShe}&>26g+YX{$&`V;05CU18`H7EUQsT=jjeC)nja8Wb zi@^753x=&bo)a-B$zBH}ZX5mok@uEyQLby>Fd#9cGj!Js-AH%C(A_BA4N?NqEgb_Q z($dl`lF|(lqS7TGseq`@HS50DyPv(*+WW)v_5Gs1nTwgZt}~DGIR0Uej?hX76M2G> zN-R+_?)}__!M|2jEywTJkoK$*kRQKObyJ*vA+iH#sI~ zs*KB)KJYpdEU>tBTL}za)`R<`C@iVfYpO7xHbSfgUA-l5yTpEcJRo>+aWs$g&5*_R{}W1m%U+b09?m_EejJ zqJH>EYx{?@EB>vMj;Ow$0z{OW>ePByUYG7IS2y5xkJ1|`*M^nwmn}`nmv9Uvk?$MT zHUnSG_r&Zg8L4PzYlG$WK)d?_i^o#EahOcAc90lLN<7)9o=lO~Kt83!@Ib=k6hPHa z4NiK5`n={b zM1doz*MyJ)u90Al!Id$kk0oWXg=X;Jn7-7Oc2^ZGLOoWU1jd|qFGx?`;AEp0(fIA$ zY+y%)^OzDZs0Ng#w!-*<(_Sg@o|~+44Ed8^}s=Dy?C_95JsO6vQ@a zHKFdHk|K>4NWhn_yarUM_O65<~zelH$4&k13|=skitA(%?2^6Gp?qBq(QIAUSdvxZ9F_gi88jS<9r8=tZf@c&H01fEJE7sn^Vtpecajy-_XyF7Sj9i=W69LQzhX`*);g5)v%u z)BN!$+3e6gQB848_jL}_ZC6=&{RgkM-I{$3zOigt{M7sWQC>NpZEyt!6*xbB<#B-C z=6^q$$ar~Y`xpx`EV0XHa_LNBxIa1wR^TPJH>U1+X7*IU|92|h4wnhnmumqmHLW6f zkB@}D^q+0S6<`(xM<^xEF1*RF9;l~T4}tq~XWO_@p0!G#ZrxW8*X?4nNP^qOy>{kf zj)wFyN~g;Y^cxG1mpxL)42{+dz}K7hX+$mNJD4Tnspl9R=2cglY4dElOwF1a+DKtQ z32kP`YqyG`Ei}du635HkiJv(*yZ4+aIoB!V7edfZMGF^RYND>7c{qz-iB`8gjKh>y z*-hx|(#NOB-lyN<&lhf0JQ{S^ci&W;BpSO0!?Jp$et+#*BK~sCLn^-quN2ZXL#Fm| zG2}1F8RJT-m1n-5v%XNe|2e`=ak_KmD39GxfJkAhHz`yn8-zlhbB==rDPKHeKZCqn zn-UAwkT8oVs(dduZOZk+9?oYwk`XMU7vVk%8tal!cXHIrW0P+M>la~jo8n2J*uya# zMayx>U{WZW^}Z2hx%3mf=-HwnYM^Xd0U4YFI~ z^5w7a(6N?@?Z1Cfh5el2e35%GmI6%{VWdn_SwSm}_a&N4ZHL%*3vr68U9vz^|Qc3EfYnXP#b9GWowF)vKX z2`<7({_xxwg-govf~?{O)^-)W;xh&bNr>&Nb;0f>Cr(|V2{!zBWz2$*fMELZL>b+L z_JakgcT0OphQ@R{2{w);eR6Z333bt=2XmS|;2jVsJ$i0Ql#PXV<~j-@VnR3w^O-b2 z_h6i+BS*KtW*|wEd(+Blhmam@d0(N-L&?8QWuU=8i1%`u7Ze;Nl(~*h#f<^Vbw$Zn z^w5Y-8gWGV>A2#|v+wj+-${|gdmldL8F!x`SOvXz--v#g_ov0nqs=z2Zjx#Nh38xd zE~0c--{h;;@Rv4Mylxt#So2`XHF+1jAuUk?nI)JQ{*`a?^a6PF>43!ah9Z`(oaY&< zxnJL4hotI^xaysZ!K(R(hevZ}`3Owo4XuZhf0J+_Q?NV`@;7(@Prh zm$?LgrDT58lrd8(ih198@sLBBYjvvj>DotID7k*$s7P_{v9(g}T z%MD#tY^(j%tJ0$(YmKr;S_Yr@?rZAiMpF01zZWLqWNABTO1%GhPQ?wMDqm6FHoOI~ zBe%oFcV!hmqJbIt;0?#2`W34rw5jYnL%C6R@z*Mu!cT=`;(NcjF zu@K1zr=>U4J?Z@S_EABD(-Og zj8CKD-<5iT6Per7R3>hDxf0mj_(1nWQBDJ?Sj@aG)i2p_mCR6r@)R8r7Osh5R$?-G zHO&tvLb(Sm08T< zS^QX!nCR-3R}vm%A5k7C;?ztYi-p5|cJUZuXI=5{KgW5ULe7XE?(|+*-5{vZF5%v- zl_PvM+mXI^g5W78%Nf)D-)fOvLTChRw)pdBpu>$% z*@qeH|JPXlU$5UtBk9}UzixEa z6*{X7D^Wv?7`-ZNEx6R}{`~5{|Nq1qT}b+A4@;*|$SMtle0qVt`HcAI$*JAyKPh|s zkIKmMV+evN-Gc1PoBnhgC_D6)wk`e>hr~ikMITW|uA2Vdsp>T4G ziMy$plxm;%f&cxU{^ywub3r`Js3v_wI`sPuS_HSr9ZYX}CSr+GuxG z5j3|-jz?NgnxIlKPt^6kijppmnSzudxZbWX6ljLoei^v$lkr!1mp^1y?jd8dC+p=k zO=|qjNOytX00ZN`9R2?M3Pj)ASOS+y1S`J;;O?kg(QO6LtY4}lJb3!Z|V#X zoyA7Y%q!n|m~p`)$p7H?KaHNFA_(IC>QNC@ z(?fUo2USp+UtA5Ten868_E^HL5eg&38=U0Z8MABzOI~$R?_hKl`VjwOn$BYe1c8N&C zl~pCf&w_da`x9X1;x{tHq{)1*Ugh_vx=~QZGBo8`x%Iw3_i>Rj%V(I%{YI(50t>}; zI=&Y}p_@V~SrEIIp3bOAc|J*;JkqOIR@QM%wQJz_JYu0@aSQ$Lv>YV#mX-<_?<+QK z_94_9KJz(}j7TF#osg_*^Bqj#_V6{W47$zn%^AxtHrLe78z{oaYHdOA`DpMTB=Ckr zizI?e4uJ6fI^@`8Rzku2x}WS*z109ooo6(z#lBojiCp+6B`Al+qUmnss{++pk+n_I zwhDn>QaY5@THDg^JMWrF#3B-)NS=Gm4pI&A=ZU{`@TE|5?YKU($fMledM>5yUVskh zA2WkuSxu=C-_o4lG$r=s3aG0heNT}bX_pPXU*|b*?~v@m7hi+-E`WcKG-8qhZtt|j zwG+QK88i z2e5n$oPHwp1E5o$B|hptvg3JD#$f^fcnq+GrF|}&D)?T7o*})l6h7VU3j2Ju7w)2{ z7aOU5c*RcA{NdBLRhHX)xkT`ZenR3_qt?Cx z2c9}MUy8~*H=}^2R(y(b^_IY7!R-%#Dkp3_%O`J0p*Pb_^_Q^QDr|?Fh6`dbj;78_ zNndu7N|8WXX64JpbeevspT=*lF!oy%{u>y#?s^@aAY=bL>>|(3aQk0{Oy9Jgtj?bf zr^TwmsJFhE5+H5?*3m_>f46*@42jIIxA?qFD?h)rz5NNGsS7>=Qkzi+DNpaVfVQ4T zQrJJ8CHoVXWqTM39%kxk7S+ee0#uW)!2r_Va|=*e*e^+G4Z+a3bHY^fLfncEoR0Jrh#UcE;)-0=om4|4x$>-a!~ zoEQbH#~;aapEG}%AD;*3F~K2r?v8AaLNFpeHm(L+ifQ37Rm23-#@7u%ud5`k5%2$? zk^_CR0KjvpCXP&}cn+=lXRINo7~!p)o7)3bRddiB!$u(zBo?Bp3F zZ_-mg4XCPb1+U+u= z0v;sZ)xqN{yB|0h;0Fz|%1%35d;xSM)-iZ0tasL8pM?S>rm85fcs29l#k4U2>Ep^@ zc}v~Q_|B3jQz`A+D7~Dq?M=Mb&WTjQ?)n^ofXIe57a)@%jI?Hw0H`T;BQ;IzD7{Fp zDg2JKm~2u9aWX-xe`;d@t_nmNiT8S_CZ5%(=M8sDXit@oZkD=1=G1ueQ9lhOH=6Z` z@xuqI$VJ&cO8&f@_jy=;+nDo4-ER{97)O~<3}O@B>kxTjGocO zYCk+r&sqEO@cVhM{(3~p!|@Z-U@}iL8BwN!IWd59Afl9IU`Y#TJ1lZ8@Y4b;f7(@YKsSQ=7J? zCBRmKO~%jcuE+GOv|QGZy=|$Fw)Ayn`q@2(FPl;Qd^JXg@&s6VrM{R<>pt;b=zlIA z7VjZHxPT>nE$fHe8OUTvh}4x8FOQjfjGjpBuxWYem-15+v><8n1d^9CyvcR8aaAtiVv>C}<5_~N zwnI*Q=y-^-yaYe!)84<^L_;g(GB|HWY>aPiq6nCWQwL)gTwW24Pkz4(EgKrw2$9xO z%{V`(q}-M8qW^t)8ogvC@|J$%=!aj>Z9G^HMH9U4x?JBkdp|5Qso*Mobi;3ax;iPw zYq{?O6DUZ*db_yXIr#MIm$KA3L{DNPm4|5XCnv`4zrNCnDd-CAb3VLbV09%%XiLtl3HnNOU7-f{rns zI>v;T{s#8sgrX+$B>qyAZKq`94o$mppp%^MlsnJ>Qgw8x3YkrmQAd=wo|s&g&-DpY z3q)A1p4Rr?V+K(RLdK9n(J%Z~om8={lfSx^_sfkzS-x3N=cA0_2MwZP8fT$n{zel2 zT4(Nr&FFl@pXiEWdoY+#;Jz{mf2a4LAi1Ho*Ru71jP3@#*4O;|Dty zqAU6_BreDecurnzDc^X}jz@7`|B18XHk8OUR7g5nC+1gjw#eOZhw zSOTlHoFc}9{e(_dYf>Vd^a&F_Ysv{xhmXDfk3Z}FaQ}YUizhiRxn&_;rS$>;3+x@x z?R6Vm6*gtJe*hG;slUsJ6I=kBzBX4RV|ZyI=h2(*pOEBqUN$8=B`yB`pp!z`7fD6a zU*5G;GpXZ4%K`cB?=*qiX`bZoS*319wfu+~{pM0u@JxK3>;oF{C+awwa)hOFF+W6J z1EGxf@v6?jrKf`qy2?q{*Pb5h5~Wk5ItvWBWM#FDJfB-o$p7IcIUBOYR8&yGOEabWq`%{x zU|mrg^`cn79OKKU6lD#>{vgX+^c071!&5>SG+sio$DV|fexaW5?MA@ZJmYbK41^U; zu@FQY&ZvzJyfAhEJI33F(`EUpp^suS8O`x>!Hpc>xN@nB8b@W++*+CQYl(~kLhYkg z=jsFG=4B+_3ZIT?Q=Y7C8Ow_gH2NOsj=sg^q42#bcSDo<4f%n-{k!X=v)Y+}a(?Fz zfuuV$k0;Sq=StG!KZ>iu+;-XlX`8QN>ZawT^Hd()4HmS%d(Jit-NTeKoF=g1tm3^w zEIwS}t1+XVIU^4PH&-id?cSX4hsYJG-^{(k&a!A+MZaCCOMV*=)>blWr@z`PW%Yg} zCwjYI;$>VYhk)XT$FIBO@eOT=rG&BdxT@)3rWe}~Vlu;pFxEftX+|$s6{e=9H+!Ts^|?c0>2M$!J{ySy)oLl*TKZ?E2)JsG@yc<(Xwef<||D1{eH zWYcnXgEe|w^^HnyD4Lv@e6=6J?l@rdFrh(1o5#M

hK$xv)1-Y~4%Q%myMu~BPR z2eG0Rze(r`gO;hCKaVCDiliXd?ZF8aDQvx_<5`0%Wt=ZE-{citP z2mR4oHD;h@g~r}#4wL*S=j~V#C&mo-GU4Zg;_?R+$tJlE8yhQB&dYN4i=vGe%a0fz zXMR3!pO*S@ZOa*XuJuIb2Axz^2~MoYzVJ zaV$O+STvG?R7&C6{PUiwpC}2924_h{v3ua+%BZwUaq2XVzn6B z`Hk0$J&}~8pYHx$R90ASrCQ6X#c5B~=VRJ?qUO*l# z(VcwoQ6G(dbniMwi!R8Pn$5eSeqdLfoIsVjsCRj7-!ek=?x3nJPK<>65?dsoao0}9 zr(u9V9EQmf7M0+10ZKVhAC)s_{9+YoQOg8kA+H8k8IAln1%UW#&vz{eP+vDEY=Fqa zJF3!*dj7vj^}jQ0D6ChXd$zS6q*Af%nl2DUtVu_ymw||m*UX!#up6sx11zbA> zdIXX$0|YD9L~RH2$O*)cc&{MuPc1AqPLm~x%c-@;qM7k16VSZ?x+=?X1gCNtn;@}X za_nG0wkVn7#SmHiPF%vgLh_c~Om*hL@jQgY9}QZl9z*fVTUc8}oN-pDCh-~2Vb zB5~vvL!50o{6`k-u9G{qnFpGE9I0ZuZ%Kfr*GwrM(@glV?nAEtlZk)Xq zw3M)^!KsX|db7}+@JBo<*T=;!^9PF*!SP~0ZV0*h53&G_H+zu%73m0P)=U*_^l*_= z#k72qpj=>w#JU+>2DfNKdkIwdV1WohYr*Mn5FpSRKG+z#`e{MFS$=qj8FWRatbb~w zx~vML4Y$h9{$RG!+!ZCL5G6mjPI*Wqqzw9CoGC|`et17+{Yq-z$EbQ<=BDo1xMMh* zzNrZ@E+zT#d|&J*fOKrm-{(9+UiVHeAfBePe=9B#>~~ucT=a&JZ6cTHNa*wR7!4nz z&SlX_mi?r-JwrrumjXsqCtJcVRJTK~y=pW2V|(3>3))+0gJ`c^)-Oy-UV?@?S^u-w z{A=SOB@NG3Atn7?No*m-T;zQm?Q81IM4{Fgn;kYf=m|@WIZ;rV;&c@4b_40o!#p_a zBi|mBKey+=x>+=)a;bw9x!}f3Xu~+ERF}%0f#RdTVb=PvR`gPm(I!1=7Z#*>-*zMC&zIgO=q^Fr^pp;rzw zglQ{-c}WDJFA>kOO}M^?3}V24Xy{VumRbg?RRxcKHBO7LI0Pu ztJPyUyglI1Sh9(RDz7wQy&AZYO!B7ahp&}2d)z@^H3M&pfvHh{N&0dOja5795@&@^ z!FXoTyg-K>Q^MK`Z7~y`{HvFjoBqGu>!_z@q=?xmmB%E1VbNUcB7EMz3+XjULKKoeT$?-lcmaP+%+$2V&+@#kYg zOl4Cd+JkA`2YgrVMrFzG9h|caRdl|M6kdQFHSB2_dloeAdZ)S)$rr4*zuAk*#e&_O znGlqERVa1hk0lrw;LsQCUXaQ`{vN{Ls+*a~Vi(>(}yKByXS5h%=y_@{)%2eX+(Zlap}xb0zRbSG62Ih`@WD^O@?=VEaS0v1f&444d5S zm7YZ*f`KD2LdmVQ4UV7m2*%v&0pYAy)7~OBjtufK%X#i-;CiZuM_h6o0V|4|cFTb9 zBmT6=7T)tn*J+(1B5e3o8W=k%$&LBRQA6y~yBg;Ro&&~H!KlY+QJ$cD)`ugs?l7l5 z^cS%%KCwT-W+QY8^z1h|zCPd~NXsQmkz^*upIm*jGiSo`yxSEa0-eIU@@4b3ZAyQ} zt7#v27Q8>a!hO{k<;5^iXZ+W-C=mxa0e*R@44|j2c`IB1GKuP~yQk5eaPJ2uxhDPm zGSp?M$b_bRXf*mp+EP(VoE78T_;=Wo(cAvaHgV-bugs_icg7?BaibhQ>nh9`G4+*$ zZ}30^+ZFK{bs4x01%b@{-bd&roxV=yAG|D`W)@#~DA z&>*FP%Tksd)g_UH;j?8-Z*oq%JSeOX?XDIWYnMjJc$m&fK=e_1?UG8^Csa++7O?!b-ojCqFFFwjyvVU2KGlSr4MuPgLrj1=z+CsGD1zvaC4OXkw{fQy?aL28@XmGlfwL6tTNfF{} z2f{l|byLCR3=!tWw+@*XHs7vpo^Y=7rtf`xS8USD0=FLRX1ypKj^C?-NDxQ2-CnAeotaGtUtwy|9m`$=es2RJEj|{p!CJF#FwWtXL6VT zldZ9N1SW*bkh?STX0(v$lWp?|xYx+8UQE5-_Ckep+#a1@Z4O+^Qgc6iU=>4%y`hQl zh8)JGj|p)yr#i&Pk`;L!o=3)8neQ_titcR~my_vn3Bx~L@cbReNVzJ$u1cf#9>JnA z%NvSq`EWhK9yczP+rM051(_N-go>Oh+qP+D7w&O{Gq}bUZN|`0D|Nq%F#*BvE52k_ z!JH(V2Yc;}P(`qPNCtr)EE*zD82?)J6M8%2WYl1ZmzD5NfsWi#wQ*0u#$nd6{q1Lt z)mh{4J2?l#MoKdKj`oko?#eLW(EMb=jp|=R8Et`sOBI`4YO#?M- z)^!&G3LG{2TnQ+8{0?{2zJm6$3}hR2VE&3 zYo?PVu7E~0{LnaV*lJ#X|0voj0dK}tX#@~Y(~GCe@N$U0gvi*Ag8CU*u<7i*)(rSK zVdPBr-o;mLiS$~N0!ZDBLTp(5GK8F`&o(S$L-p2}^qo0>5=+|>OK8YZVz@GCu}j5{ z82NKm^%Y~=v2rS0K1KfXf|?@J=hN>?rOuNaBnEz^)o|c$M1S$@zOTw9q&1tzG1YN> zkMUE8^?@Q!5M&fzcx~nvgdOL{yMPJpu7%a%_Cb6zYb(L!h1`tR#iGNy*6^DoSn|$nOE(aN_g87ao7-xp9KSf{H)yE&0jtGAab~%$%?11#Zayz8mLZjL zhc)(*6#Ai~>-*(TbP1&IPJ_p@c`3*ljXm2Go39P%J>Ly#P?rx*L!Z<0FB~;CI7K8x z{|J3Xh@$wx;Hdxfz&aQgsVO|d&vo8yiHT3`i)dcP(+1s*Kn+{G__`W)cP)5O3MFW$ z;sS_c?r9{LwD$G_=Hv%&BseSLmxX0@vcgNBdsz#ie&HG;Nc`ykS@QKv>j(Go@|?q( zrm5R?U!S|NM{#M*tOi1{Uf#d#>J%rFCGstYl5gXj^?L(>Ofc&&JuS&aswof3ZHg2< zS<{yF#i0Evf7T1O1`9xwR4}TwVuFz92KV79+UtBCb)7OZ`?r%tlFaugp1D^|fNbk+=(Fie)wc9s@PYK}TIWId^5tAN`Rw?eMNtAE6UJ8IljiemQ@JFvr$InAe;AwG{<*{GdFcVi8D7V{3wG=LwKCjhc)>g%t?M* zhQ2gN)?m+LD~{_2fww`Xw_I&&=ScT7mq4X~ zr6hWB&cDyW84FvE}04BLS#-1(O?8#cOd66Y0z72O9f%4;K(LQEQ0y zEDz}$CyVZq$MdB?VdV>;OYEd#QW6q;(+zMKJSOz+QLAe%L)HYa4O0QC0emq%$};-(qA6$deoORbc~CSu;UOc=IvJiIaLZMDsrW zw94}0@sS->8u}5lSWJF1`IrrS9JK0hl$5@9EdPvJpLMwbOwcBtSdXO|5>r^MG4-LJ z;V)-$ABIYI48=zodKnK zWRS9jO%Q>Rosj|E<{yC~v|#euVb<7t;ht2Ks( zq9w6dvh!6ay9~~7whKblQg5~#!+9C>-!d$9B-Yc~9$!|f^UOU?i^(itQyA^Kr`62@_|md}8mz&OyGN!+rx(-|n7lz{?M$$>d~55UBs?|+ecf5f$|x6m z+CkSMhtVaGT+#NoJ%|rF6DgEqhy=#Nf~9PKjXF&wq>g^YeruT*uXP?s9*@zumY zm9>ZEg6Pap%rC|$AW>!hrP$9V62rk4=Nc&2dkI!SHJuwWRNtHlNI7dR)yGh5DpHT*xll_ zb3T&W6jpn;v@8rG4J}S^5eScB??)sghIt6dpEI_l!P@4O(l?F8!UFonDd*ym*bP?f=3gI}I&xzW{ zTO@e+*4^P zpH009al1?xthe7#EUTbM$0cc+_@ai`x>66JD9&ixl~5HvKwk%-Q3=U;2GY8WfTG1K zE8aVieg!-WJxJm(f|4JEW{m|vru_|Zj^Ol*;GqxpG*tR=zP^^i ziEGY1J9Cy8V*e;x29b;Xw(@_EW@R7s z{+yR;eC6Pt+H@7MT9qSI^LC12d>RwUZf)4{~ah-x& zjm^=u(z?nA$r|};aeX53g**?`Z7*xnCot>t|9u^+4*>6Trw?6lQol9tfQJLA2Je5aDo!F>F?2@S+GQNO{eg0{*PKV5M zGS*Zk@V#>FkU%lI9-W=hhltEj^an7W8GDSsK5q4_w%?yx0LEbSU!ewr5!ZF|z>RS; zqRn~43q#L+use#%l?sRW85hBdSo|^D5cm>gRWWIUv8tLTPJ`%`r4pMJdd!G7rbFxS zh_QGCo1{2L+DkX9E0J!;yZ#4nURg3YlDwQK?}3s}EMDl={`*sFqrWFW<4UWc$^TLY zs)3+yZ;Vcq?Hqy{&lWIjj~uD5&=hdM{C-X4uAX3gd++NOZB>NJSp@vpl2|q>>YW0D z<}M2h&N)uaE^E*I_F`yHGe4($o2khG)_0HevkxEMNI$ORqfnPue+{?v^cds#2(XtP$axBAx zhJ8aG^DduHkiwCj4aSn|lA+m*rs4%7d5NCCJj1}vTZKq)HzdAABjHKtjg#=QC&Aks zesb-?ke=9>a7g;U^`$>ge_-^WnQ{-gzs}`93mv~6slh@3UFUzkK0^rq{Fm?hqj>el zmtQ+8$o}~&|9bb&`j%sS>DT}8t=j0FiT~vyaKHcglk~uD7J~o(bicl2|K(w13bEX7 zDgV6rzdaUHde6N-o9wSY4lD=%`C^#N{nbmszkT1IBMxYMtiOKWpMMO*t^fHVN;qxv zfy>{U{eM54t#V87A0MXczg$E?lXRS4`d69q&zt$*U*bx;r1{HR|7sTh|8fxpISgOR z{_`-?MmJ?V5dV)4bMjvF`w z=l}1c`~OD+xSsZiv156NJ9{tSQ{Wy0(s$xF(xk|8Gb*Jdx*%t73gTiqp-HLilO~0V z7xd8lb$VJmse>Hlpt;hB78=5Cib>+_-8Eq^tja4qH~ZRX`YYZ?ih&N-bC-nEFh*+i zd`mk&&_wpjaqpXd=&1S%u<_pgRTT|XcAlTFp5C(D96tTI^z`S2^lzW@MWp4@JM}SV z93NO4*-r{yZu#UF?e-Giwq7VVLDLgYyskK5c#s4I`7lSO@{y#+CV!xIlQ&eGxo`Cg zED{JJY*YRF2Kmv}nOOHmaL@KlaSgo=D}3)8&HmC9MCdNe0dolcr;zckH>7q?03?! zSMd0~$u%-b^xChbdIJRKg+479U&z~HS)>EM01m4FUgQxHV*2rB%D-ToZ6ouZWdHzk zLz6TV%NbMLi&UuymgOvq^#Jk`OP%jqxP5ZRc?$vv<2Rg%-N~of(T0$)PQYX-<>>Pc zx4((iX*_JdPgXxXn%Dkem4qQv2l`z+;1&f`s5{%-_g?rD_@KW0fkS3s|L@b|k0qi8 zJ^ZRj!8)+<{dNOr^cczSN1`@F)Ur7{=3ks-UOi9W&$Ga$CJSXRUoP#pPv7tWyr+=H zDSW~oT$_+~DA(tQ=zbeq)30sW(Oj8FrBB+oGBJ^o{}Tp`qX~{LJo~Ymku2V|Z>??k zKB4!XTj3jaJEC3ya$0P2XOPFmm_H}>5GDx z)D=1Sn*VxcKrS;wNxiq)jBwJbCEF@Oqk}gKrpZ+yKr-qyqleApi-Z(*@@BFIEk49g zI4YIOM;V0yQ*UG128C=cb6JT%iwXWj&a?=_E4^WU8!+M!{fNvDj7f52=07TTK$nd3 z8apR@dp#uMKP9^7+H2j-*dvvY{*a1&Anw%xeh00qoXBK;tkBib#_woM$;Ee~J74I8cGZ zABU2~3m9y&3of!e4Fybu@EqUXd})ynL*C1TBgQo)2uGS8B=OcFt>2~rXrSBQ55nYGPv+m4V$FcI1B!!`SfCQ8+X42>f^X3W1e<`~F=7Y$wqEnXL= zD^C+NMOiriXaUflucUvnT@q8$(`(9fWIsNbE8C9#4q)PY`4>BCrg%lP@1MQ-8j`}} zC(-jv_|rAg?F-2sXaI8q7IZwpz{Blq&H*(?e0R@C@wE*Cw2L5_7~|~e1?^moIMQ^J z$^uWrG!)}yuVEShTjVO==h3l~-4OQDfT|Hp#sOe@QH9uNE_5vpRE;`;HB9AG*V)bV z(*k9Wk`g8CNL&3FFL__NJlkY)UI&5xBU`6(70q`N@*)Ejsl+kMBd@h!xr zc4w(<>zI1C!Ytqwmwl|GSw)vg2&o(P+HEK3XK19Rs!~!XLqkWseXN%$Xc3IQ0SKo{ zc|UHshb4ft?chiowB$^jX|o7^8nq2zzomp9sl0-#cEpdiAr+P2>&ngcNY5y((DMvF zO%4XPuH#lMz|bLJBae}J9DDJ!et-Z;O<+Fq@7|(meAJ16aUHJtOov>S&^>DcphB*1 zuHIu&d&^;VAh1(> z#iSTYM*1zl#0!$|4~Bgip!>x;%5VpG8fACO86b|peu^UiOu7?^-?H|sL%NAgBB9R` zfW&c-%yhuJ2grKByy|iU%+;A2eUpzPfz201Sm8+_i24!8I-$ELrl9%2suo6@PFyLm zfvu$yNN_fyER`Noe$Fasa%~O_`QfC~{YqT=unge^GySj^cs4V^Wogvo@g)ooBI!k= zBA3IgV{Bvp9t}OF3=iZ;8?guxSNg_8XXuEcdd_-4EjEfUdHZJENIBUJ8s;l}!w!E2 z-$ zV1Y9>d%iRN8(8~pBp;RE0u&n-t1l#Ix^xXp+W>QeJ2HWpv%Pjc8JjK;SYSq&^S)zA zorVun`y`DeezEaivCBb`(g@owJQm^XZ7iamMn-G!45w@edT9bEC2+##2#4XG zyqVgdG(S{0X3&j1!vI`rG`E$LqahB<(JiL-=x(MivblfXu6?cVbhi#}<<^|zB)Q~NUr;XH=1yl6CU4ahR(CwpI@J*v`#9Yjrfrmeaka24g`1(_B_~2;d$!K zn~q96g!Wi^O7S843q${ zHLfJA)oe1LSfeuf@9(4U)+#Xag(B@jbKh!|J40={0b9*1f&0NxG;#G@a@K5L z8B@P$cuArCY@i#t*#U<@KO@`0-$Os9x1O_)_aDtQWKUU{pGW5e>tI9XdSw~xfn;nQ9d-PEDO@4ASfmd?6xtfUSTup%S$}&$ zhROji#9&)tz5$pJ6Yd1*6Sofm$4LCxNCL4qgop(Xd+1#hA6ic;I+QV!3JV|exIpCY zbf9t|@MXw12yaRf2g?d^$=w;Sdc3Cv55L%VrfptKJ7d^DT4#YULp=5Y1!=1&6QTG6 z3R{-&jVStccYRVkBpZqykQ691NIs%&IAcj*yquu}92@~_#hl2dJx6(P<`Ks%at7Gv z+a4t{ymRqVHWyWf4J}#>ep{25ivT1LE9lh=zq~h^wgNrIKC3Fr zhpLC}oUHUg!y-s;wuN;4V5NM0!tRQIW>&OMsOpovvjjJ?6*^UUX;$INEVGU#3t)sS zSdYjzJr1Y0sgr`A=C8>eAT9StUvECdi^ikKWR}b@VG5L2>4n+En0h~0hRBOkZy@3Y z_(y&0<8w8**&kn?Ev4m7W4;JcPtd2XC~{$i1J+KatZvJzp1ZjYs;bAReJpTH$SqJ( z>-GEK77JfeBHVklK#6{|^kmPGFnvrVDI@J^!Z07T1|bKwk$jj=$NV?@%B0 zNcW6_xQjNuYc-gG98LVVnJBAt7!62Dx_qk#$)su;Cn@{{HarH@n_3fG2Fr@L63oGM z3Wt2F5_Fd_*7Ju^E}SZi5%D(61RWyWwkm072F}e0!ngI6C2kS*kM13-Pa>IS=&tRy zqv=TQnW|htrS&9|xM+>0n(Av1t?$g4jln)935ega*fuf-Td!e&<@GE@foiH{9A@QX zrhul@rvUjUVVb7FsC$#da+syJFd8_95RXU_a%^!azuXw-uT(2iM9+>yOVY*E6Cj|cQ>G}! zk~KPG?bJWQjCei<(kplMO_)?jWPZ>+LEqSH{2tkzw}1=ln97=$tPHBAEIti0u4CdP zC$n{3fCRzPzOI$|G^LgL4?zif1=@@sCYdVgiX!JRv;265cilx0~SMy0k^%F#78Hw^-QHLe}z=DomHTGIPPgk0pL?3pTyL;(?Bn zi#J+WQkK&MKTbDL3S%=BNz2F8=1-HpTF|rWcI`BPwWM!{CtHGLx)S>NUK(c{W~g)L zxkwTD1U1E(14g)HF^wTVL|mhLt$1CqUyKY)B*bKMnx3iP4?Vme%RDk7wDV!J0d+7m zcEd;lkw62FN4gt#Zbk`~k1WqwlWsU?Awh-MaA_GRbD|$Fiy0T|ZapSRf=$CW%56@< z8w2M)?P>nEt~l`~w2`NvEE7O6^d+fW7iM%44N?3GHz81OOW2*m7g;uxUrx1456l#q z?QiCm)N94i_@WWrmmCu&A2U_e-nlqMLn~@6dm(fX1v49Rn*GBQpi8>MkGj+3>%fW}M!3hXE`ai{W$*@6O(1*|8E z9hyO3I}0Mm|oz1c1L@HwbYn~m}o$+oV=ZtP#W!&)Uo-wk_kQQ6ENDm$;Ixc*DunH6t zV#ETQbf)jo=rv(yV}HRQ-P)MQ=KNbJhOxd5%Im>I_youA7E#oi@}p`VY)fv*uzuHN`!z~p&!&`X?D zf6+&rO~DYBJ4rd)Z({XeJXr{Z!9H%vU+O*a!NSh#tP%SX>4M_^D3#d@w=u?KUB;+Q%Rv=W~P~hb8>z(~^ z_v{?(V&8HKm0tPaT80u!#P_wHD%wCOT$)YD(fSu^$V=Aq25Eb>e+cqe(0^%-`RkA} z52?;o=On~xR+VenGyQCmMX6P!k2h)LF}>Cmmlu70X(kcVygT+;sn`8`p`vbej3NIQQOnh^_A|Zb)L8vlEXjf^3HqKus-y( ze!u`H1x zLPCsnG|HB!lr@qHA)@-d=f0o&`9064@AH?xFw6D6uIoI`<9MC!RxtI1j=Ym!ELJYC8=HRhpMa{@(*fhM?`Bo$JX`DaIlUt~>m;37 ziazkPbu3$E!}cIU5@K4XqSBa#huDscak6MZy7$cWxD`aElx6+`S8xKQb!2)de^&Fy zyI>E2SPe`<+Ybpj+rhQWz78%Po*OXDD>>UpsIKZ2y5euodS{&2Q5moVi7K9>eln~N z@hn_s+jsC&H5I1NR?B*neniSzJiRROPg8{rzXazZOvT3-k<~BXKz@C1Omb+`(@n*v zGlx83;r?QJb4XgC9?Y*lo)U_iV^905i}jg~{(D>63Q))6Zy*S0%G{7MGr!dH$$P6* z5@A=cGob&Xc~06WCMWjO9R7qQ6?`yJ$`Y`XDvmf84Mg{~kMKIa8zZZM>T{ksF98iw zF(mMmS)m8)?&h$3Jm0Tz3m2dKDmUyyW%w;jc~lfIr&vsz2hkEP^0v=bkIL#29z)ipBmd;q76b2G;C>n>~YXX}vTx*@I z|J{hBHVGEQqC2&A?Xn`c5C23(tmZ#2+5u9jf+h)XMGja5Y!QPFvtTt*m1qs zi3o^E6%i}4>5c|s&)!_}J&asgwBq>-X9GMS4g2=;)jLZGN|FPPd4U&B^L&__2b5dp z#!bmApy*i#Y{PvuCh2uNZ38w#J5no>Ousj61{xO@*Pu-jy`s>pBFT;Zv#cvkto>AdMrDn{>GBKoI)oyj}_kW;F62}2+ zH^0s4iYKu{7E2OFmmMLzNDXdC$?2@bfoxg26^I9?cwsL+WbycH_=RJ!u~*HZr#@W| zLE4wpHw_`+89IOr&t<9fw=R0g225kZH+rlu3d!p6gUiu?m!fM4U5pJ_B zo~%uXvdQe`$Ksm-W~{}>x7x{=)ao*R6E8t+V^(MWRQA1EQ|8UBevUg&Q|E=t4+dng z!)i@B+MP)vp_h8}azJ@;#)Ea;GTPs|RN@UBfXT^qT&5NMlon8*Y(kO5HHQC6-Udeg z7#=+55WM#77u@nS4=i8|{3p2VyYP=#+iAbvAS)%aqT1#)!xF#K;)s^J^n!YP9*gVB z7tg#@;v;Y6E)TNT0U%?x;KBafFmz-`uY2Du{=OCbKMPr6^uvy5aWFK+UvB+VOzj*5 z=b_GU!;vAq|INZn+ZD(#=Ptf7&;0*+H3Oc#j6^Bqlx4jtRyro<8l?~B;Wyg6auK8n zE1LB)L%!cKljhBvBWUb=_q*<9Ap+|>HZ4tCDJ0JVVZp7SD^68rGL=@@8U*`p-aOAN z?)j39*!0d5bf}ioylsZFruc&v)kx=aH(U-lubw9_r&-j+^PjmQatlz;OVTUfH(<{r zJSfv+saP|FbB{TXy0X_PHnyrwe{*6R8 zbkS;27moB?UQ72yvIVuhV^D#nW4@J$7r3hH9(g|wXey+3Ng&4#rR|*c1h2is*Jc9x zCH7$%r*^vd$bhsUVe`Mn|8&>pVNTOS96W=FiCe&X!aN;6kwLu4@swls`%-3L32q@| zRRhLKUNR2mqt|`P{8IjC;J{13fB7ZwAaS|xG^ICA)_#8S@eF#sRR>RtOFn7UCYCEjh!TA0Vs>o#eJXu_DE3ojMDb^<9c&FqpSm`Wau+jr&f~1-0Hx=A&{gw*Q*aac8^W1j)FQOGL>};fiQR)g{_+0q(d=ou z!P41b7$vb8((_nGUA3pOj%bzL8re52AiS6P?p3M^eMOZ3TFa{?^6=LEB%$|PxF!mH zv>f~T92-4cEj_zX{qTpG(M2~Gaj6nUX_(o~$Of}lm`l7dWDjh0deXQb;sAB{@fdA9a*=Zr!tm6>@`0xC0i*H zOIyX1?#Z~tp(}U>XH%BILJR_`;XPpN9&zO&KLnPq(w%3?H8uOIBLo)S;F14!0hx#8H|} zi)5IDD56N&D})UnyNQn$lr5QuaH(?`&7Fu&hAjx|5VxLEf^#T3o0T}XrCEUdVh@AswLx}6oMH5n8;6ZtG>WmPFo^`|9)+jZ6Gb_o&nKe`I8y zts7Y0=SLb!6;F=)!ZSqy{@J8LeL&qeLe4Sy6Qj6Z*rz0m(cFr#JY?^`U;BVYisa32q?Z0X$LjPX;}+G7vz zdw9nsz!Vp<|7f4CJ_BwF4}IoCy>FdMpIvNhbOzrAFr7}l(3gFc^QHa6`{8SD=vpuJ zUscYwCuRTTyQg)-_k?MG{}0{U-{NoY2zXzoi^zjqtKZvpdqMpuXdT(V_ZVC;S;_C& zbY1F6!7e;2JpJW|gUGDYc<1AH(5FXVpMKbIYDZApAW_*VKYymaJbo7X_-{Gj3_TxP z+@Ef5cU2M5g?h!)bUp7S*PgvwRb)DS%uNr*ou+$m!gJZewVr^$wIKaxe}H?yVHk0~IqEQG-%6as_2)Y`yJ1?Ot|EhH z;S6zb!-03A23_@fhiGqk`e8P=fSe2Q%YR%Y4ZeUqt1t$e^Y(0v_#}r|A);i3gGA`7 zNE+3$xMVIVrs3(6At%00$urf^rIYjA^*sPZZ3#G}W6x^}DWyGY7xqU)@6zKG_}@>- z5CodMQmN#jC$ZpHxBo4yebxn9vGQBcHed2XsU4fcKPv?Ryc!KA%dQ5nw}JMT;_PvO zIrX1|7+899t$D2$_`u1HYVgm{jjDW>ap(RiNu;wUw*{vVNyDGz^g=lJhDLtT-!={T z6SV(wVm(Lkm6^Q&tvqrf1<9j2@+-Kzx9qr?YNT=^fo!e)8 z>KFL0AJE10{Ce&BKY#dtuT8f&$ba!e^bz*m3~H)|TAjV_3W7g?FNsiYIRUTS1vq76 zr-boKSp(4%4AFFPD?y(fQizdsiu%D;)>-GWbNu)xykl~jlYN$T&^Hj-Z~mQEt^oc2 zpHxn$v`v1Z^)J-Ujhk*Ftp#SeVfI91z85ASXC;#bO&fU7n9g8JAIhf=5TJ zrMY>y#9%_K+%eMv`uQgPDd*RJKLKh-0bxui!VX%r0=92uoblb?i9=;CbzlhdY7s-?f-_ z$c8T8{{4>{j6m{RW~wt1)ibuWNR2yyAB`5X10o#&yXo3lwX0i>;cN<_YE+Bobo5fh z>I(6zfRpC@AHb7v8z^UI0k+O;$UDrtx1EBDyvBpK8h4Ues?|E9B5G=Ic zIs2&|8%huZ%RI&@#z(3%Vl%hQU!8(Go|3^v6DmY05@PXuSW^u}(-~byvd!^N0nB%3 z%=DZ@dM19{PHZ1i$;{DY&VJjdbsu>3hr{Q8H&vE@{sWK$#idUwqdc;=4)6Wp>V*Xp zMB#2#-+jh`Qz{bV5O)^sbK~Xof)FmVG%O`uuV3KNcGA>hJgfGyGwO{1B4=IpOfBIU z_(VPcRVjd4y7+sE?^%Ujd?Dln3o8luP4QOBD(h8-EV$4|Pe%>hNz+!|8-=MBKPe+E zs7Uh9cWMdN6Kk)57A^1Wo;i+G+$fXc{>1q2XXBwn-k`D14uqI4>Kd?yDyw}ui!V;Ce(x7SsLHs8Ci-FO zO|Ugqk1A9w@tvkuS)7cwlZMA9)^hoa6eS_I_%cP!ggWINvH~{8i#*vKSV?wfjcyyU zC)xGiEu1cWGQc!nC_mg!mr?vkW7?ScvRfZ{dyslWzos%!GgXohu z12Io3UgXNkt?wp(i~4q#X4F{LB9qKBdEZryWX{p7g2gv`4%Hq7WpwYqVJU~`-s!`} zO6)bwE?!-^LL75ESlQmgEv9eDpDwS{$;EoNJ^c6bmuysrTq@p}4Hq0S-21I_oH(B(`n^i-EY%r#%J@YrmOiqcwH?SedVxgQ7Nri&(i-hD?0+0nKI_ux;$q zjD0@-2Ry8Y12$d*>YjnEzV~HwRXbsfpAwWRnb66XtY2{Z|Ke54cVEl~tx&(z&3{pKRegJS9Nv^uhz1&BB+FnD`V6Y|#2g9`Z}> z`B@n*I^hiMtdn@Rq2ipn+8hoLe9$_5UCn5WTWbe5t>PIvI{;^xra4%$04JF!n^tNW zH($Qd#3T(YIpAo>CUTV?cZDbznIA6UrjFJyiY@4nez^zOq}(5cs{#`#CJmYXAonb< zEO9DDF0$C@Y#Tr3jLf+6b=SuDxA@pR%@wgZz|HrM~`^MNU@1GlaYspHV*d&qo>d$~5kn*14ozt+8y7DHvmITq{7wCTt-Fe^=Zn;QteZlxSurJ8bEqk z2TXK__Z4r;q8T_}Ch6@~iv3zQyYuX5o;I#ZIvt(}?#edba6SS!sb865E}939PP5$% zGop_<=obTc1~;AA1>o}6dXA^cf2)3TGE1MC^Dg(d2>6Dg@jQ!FUg}J3(M-$$kJfqn zq?O;-O;-BQdnI?jKA~#GFS^lg3k4OT3lHkk#I`s=yor zcub1d8ai0Y`qkw;Ju7aqJIxiIL9@e*yqIx9*Ei z-@IaQV_tFp%nos&aTf*;fsbw#a3-ej1ptma(E#9i%q*wn+Lhl0_OpEpsnM2ot)M`u zh$FxXmW`heeXGFYEsS3yr)fMCu~?^>{b1nNH9$x;;a`90SkCmk!Xc5@tSOdZqo28g&P?FYBYN6PJJ8N~06h9* z*?fPu8H&DT5{VRHKRPTrMc(F;qn%fKq*>45tS{Hu0)A!dOB1OyR`dQI8-j=}Ka(h=3F3f}(8EdcCxCxAnd216rj0SFC)@(U@*R$9ktpWH zd79mpLw7n=xS*og)8=NX17j&*4zpl?0+hrPx3(FUZ$?ZNbC#X8oYt=J?rt9T$kde1 zFFm6PSdJ(_h)pTzUM!%B{j5QxjDu_GzoAxsco~_nA9Vu?&$D!(uhY7&@{4T2UCDYc9#Rd@g~bFccvfNQvu|z{v*n_&d-lF z&>K)Ja!G#yxci`8~V`wj$z#w z+Ml-L7#(V9w*OZF)CYCp<-pLaqxS`=e{S%tiVw)P0bfE!)Sn2VoV2Xl#I2l6@9Jqn zc89Xb!gf|n*Aj?lgJj;xvpNFJ+Ks912?ZKsOuT*GuOMmo{eKq$+;NX_y^f=;f>!MJ zkiP4Z)l)lS{W&&z&h|2l1mV%+v8q*<1A;vw;@QSC;KBoK0FE?CxxC3^>hDeO^LO8D zIhye~{Gshkz;4zLdW& zuM=1#pDY+f1qzOI0cQ8bwhBXh-wHK)w&v_r*gu7ntaaI~4yJ8LV)}>HS&J^iy zI7dPWTu*@^e4f9~nY;U~`+|SI;??UcMw@4ZYq@kk!n3x~XJTfL+?dB5RPEjx4hy;q zp)LiEO2gN=%6>t9Jp1$Ol^JOHk~TC2Mz;`bF{CTr&}X3gqH07r{%|}=&yh~*)D!*W znW^aIvX+>T|yP=w8gi0M11UCQ|$-%kfAC9=r`tr zq%Ve~S>v~UgKBwQ^%_1cL}*9dj!U(_{A{lCQ@PP=7MDP8qL0PuO>= z&l<3#?r<)HO8#ONr1risbIap+1A-T(qmZ~9m<*D;r=R<9RcH9Z@rkb&y$I2z?}6el z8TnUEu(F=|X^!Rrew#c>^%p=UwG#XyST~7QJEMNpV)1xBJ#B z^KJcfv7SkTtG;Q_06WXf^h3Xj#v4etw!$z8|7++kAO8IM@XDv5R796&mGhO@w?>U_ z6ANbfvn!uyksSKu1^9In_f5XPEO;sXffXkx!zc?F!zp)Ad58!!pC7$YSM3}gbxhMl=ZE4keKr@M3H$H`Om?mZ%$jT4B63d&?@mfOJ$@z(&Jq& zCV9qz%1x%wUp8Fm&776Yo~zBC2%K#_+egT`Oo#+hEx>edeT zFejHt4eW~!%>3nh-r>^V(=vU(H?|%#zS5_-@?6=!#j!7%4}ma#nR5dvvWC_k19-N2 zQf)Zchn^&icoV)>dVdPm&5_vzcC%=T72$RKV2(b00^T)?&%J|FWBg~@jNkZ-UAXT0 zn#7$>8-Co1&L8`+Zhx zKP-R;W1VM>XS`FLGoUh%-J)*hh&k~?z_nBZ88U{3g~Zi5hYEAq?)E|QoDl~e$pTPb z{@Ze(3~0DtfVG!OXJOHVcYc?AZYiX4N>Q`V^R2m#3rT@>`l=7Q5Vp;zZ#g)eFYKV< zODsfCtEgx}X@28(bIY;$@88nFn7!4r#G{lAxH6-JceXQouei5NBseuNpWw!Y%!kK~ zV{3loa%FjOdW_c7o1#FC|3Fl{z9oo048x9a$FqU?nTXYE z2|kWDxk>jgeLV3zxA;?xK3cGe)o7i$g=#HN^fEJ;G9^lsIQKs(dTNIn)VqD>E9$QH zOH<{cAU`5`@y3t&J2ELL`(XHdHrY31t@+yDu_reLXrmUBzY{0g{&98lEtr6vHF6{| zQfN?s4Ld!8i)@!M5Ac6(Q-rN+Dq+3OI`+miu6)JfAEa4I9XE^%ibA!0Vv$8$uaOz? zkn0ggJ-Ij;bq3nxY2XK!JiFijw5~6m--JzNQDU~>{ZqAsM>+J0pIZIe; zwp2oJ>HIs1y%Wb(Wv`?*WzPkx(&yJ*0-NkJH_N(t3v&`HrM%_tUakf4Fjclw1_hTT zPVKes4xqD%?yRP#%*M!gF#WJtOxM#}Z2qIR`GQ2~tVE&SM#+*2XI9HyZDURefzLY@ z=^AX#X{nY-B2VRp4KbgBP1g_Adc z7$@8^=x}GXjQvgSQ05%go#-XqH5a94h(fW#G#@jdsV?+-4mc7RE?P{~a1r)m&NOpw zpuZN66r&4qq%SBFu^Gp3hig88NI?r#5u@@mQR492*ra^eweB#Q`U*I*M{D@-sU50lJ8Ai@};y@`m z46-RXV~r zJV&#OWE~|zSQspdHlIY)19kDV7uTr1Q_?3nHZkuYtl6GSz7)2#Ig|k- zc#U-_xUv_L6%^7lLVY&PtV{%Qlwa7F zhq=9U-5p#ulRn1tUe&&{OKL@G_eXm&{e>2E{!TV~Diq|3a{rM;4SdhHdTPxVhe& zQ&1+D6&H=5u@20-;6-2l+h)p6s&_`PGOVDB86iz`(-dw;x z)=1^d)H|zItOa3*PV9gfdP|)vmt@Kqz%1^aQK3MM9WLnkg2h*gPc<`;4~h&=Ddh7Y##^xuGZ*~uF2(+Q7zOXy8cuEb zj+p`-vWQrjhM~%HMpflPle2LqXX&@n#~R{ccPiI5<~$`_;*riwNs$L#1Gkr8H;J+g zM@QveFDerLUN>B{i3`&`TN0V>v#nn|=g{6>t;93Ma}st_!zVw<3>_pwmNyI@g1w)& zmRxg$<3-;Fh~HGrRfnRLHHC_-3YF&>di{=<223KZ_1@-saP5;+G|cAwuaHQ#(j>ta zgr^Z4OW|JjT>47yU~64Bo)MXjG9|ObmE3U~`w=?=@~AyAgBRYw$S$$yOY8VRF!w!m zIdw6 zYTh$tJb%%TGixpg?@t|4)04Cdzl{C>z9s}Q4Qnnh;k-;zb{-E-J!ykIwaAaHab_bN zBqNLO|J{CNNu$LutM#1-t!6v<=>c-OMZe>;QgWs(wM3BjQ}Odiht=Dyrnn(TgesDo zv}g_zP#ui+s|u3jW!Ey|yJtH4ZKF-p%~T`*3uHtJ;>o`znS%BlC)$;_oy&V(NLcpr zB?<*}R9;o)v}~Q^pra$I4TNx|vI>-X4OQe3Cb-n^UqdcmO|wGdvpUTK$BC9T&s@*| zXg~(Dd=j(J{7Q;Zb_c8W=*wB4C)Bgl0`&}BqF30#T9kIVdhq6?6Y|M);S~Sgm+kpM zFLb`6=wZITk8V_=JIF)Q?#kC8$@#9V8p(ZTuv2my&H_Jm*rcCDXguY4)$mb!RAb+w z(SyIm8DDG+Ip_%-P?BVkAua3x^OzEg==|W%dflT(4<1Fl2!DCx zWK|waePhYfe&Gjr`f+V=Y!Kog!c_vL7RMUowj^EW*BZoWugWuARb12VX}3Yz^~1Rd zl;+S)T=S4C%V}r00nc<}%CY*#e{_=~Q;s5@fCYiV_6U_T<-eBr=o9rNY(d=r#}DOC z5dLOP=GRg>oD!4^zr)Mi2=ZP|KVd&C1EiH6e~vo_WYh0JxXM7$LN~?9a+W$uBAcmj z#_dx6nOx3J*UxGC=2ri(ja+k72gI=fbD}g=%&X{I%%7vOhM+HxUk|~Bz&h_IV-~(J zHAVOlj0fb4s*_sE0K$UntUI&sVn@)jBhyGrdDQ8HCs`-Ioliu_yz z1pe0BLoQ8Pn9MyP*wgdY{vxmcxuo{@o}mCf6_4R+6YHDd$oPYOct+cV3z)O93~XUf zMM7sK*w?u{@JS9E0+KpT-}%AP}tAruB9Ic8VGr@?jJM>MltULif^O$5168anJ#9+Wv#C&K6o|n zBF1Kn2=};@>0#lreSG}=<&Ru&9s;4^RKv*u!77LpNPf6xGnxC{!UpezcD%nN!{WNV zoM>^7iV&SUseAF11Wiaw4f$255dE*OpSBqK6MGLtb`o4%U*wd?g+u09ws9JR9tzP3(TtN*cl<`wjwC5LYuoR&BzUY+)sPI z_7ivmw!cga*Hd7aUqT-yu>wtx-m^j8Yi;7VU4Y!;n+Nre-jJGk;iU0f&uH0wQK|tU z(eF|=TR`V0(&3M=E4PF)Vej@c{M+aA>%8)vhx#pQ2mCzInyUZ20lNc(mYC(NPUXO& zu@l>3i&xuHx#hUOvTZXsBYYA5nuk$8=7*7({uLn}4sYLlh{mTu6BLKkAop8EtaQsD zwKLO&dpJ|%>APFcDI3nA-b@#Lo0fTW=4!_MCauIZT;5@fKeH+e&n!LE+X*VJ&V4JC zd4a8&m(mb6Zos73QPKra>xI*ZR@|9(l5YS|!~9o+`OEk3ybChRXI^K9sss9cSE*o9 zkj?qMA*nDe|7lym8E8}W1aL3jBj|~$cUr-#1Td9b3M(0!TOlv%J^dT|P90po@cEx- z(E~b}OlEO7*lo~kJNjgHlD9Rs zBo4r7s~z*<%fbmFQ5d)Z z{cZDgRaRb=cEwF!u-z>CWbx416k7ZI2WWt4A8N!uFAVjSNsJ-4n4Y-yqM_#LAn#@Q zr<=_OD21j0P9j6#u6*E$VaK1dj!w2g;CSabk*i8{b_lmrSfM9Q)adggaA;N8nnfcQ zR6;UEdZ4!}|RsSr=RvqiMfw zV1rM;d!4({uf9ogP`Omp0`^|r3eHY+xif#;KSmsUBw_eDxLT`F^Ssrnxz}i%2H=6; z3GZ7xmEF=R880G*<|Ebd`(3{6c#mk2R=4}YR8&rMb*@&=*5KvDAKIjx>6Y<0CQ;%^ z$ZnUvoGDe>e&5PBeLcaR7=gGlSd%AfT;z-ueI|0pRO1Bl`?;C|QvuDUH)-HNC%Q2ep1V8Bk=~ zI1+q0XvF*UBI)G0nht0}kU}KGQKeDi=6iv5=D-Ug*gBv(yv;n^z{cVycXniQjMMTB zh26yl^^#prD>P7bXK?@%0l@G5#*?+Key;i=-RO;d^dL@x0$_=fgwhl{eae^?t*d~C zn{T1OHKMLe|3~~)#U}9cu^zX1-o-kBvIn0vMZ&un;i*@eEnsv@kcWie%P`m<<^jh{ z?WO}vWvwb%*$zw^#DwgnsP+UU1fj-;gkMU=J@8&#YDq`J`yRw>56p11YtQ|&a;c@{ z6_g^LSy94#nt{E&$Yk>p#30sl;1b}sSB>7BZNhxI4q50j8BDf$ovmX+Wre2a$hs_U zIeFx_dkm$=`5){*nO*3=KV@^i^t1q}_U+00J^e`?-WL7s!o~QMz$N>$+YHy3>L~-> zJ4Gr3g`eIF7eDDr|R`GNwYg?kDj7N>XwD3@`ih1H&@#olK< z0!Zk)BIiT?1mq;qziQgx5nSlltbstztXFLO5t&GZf-R`=K31A7e|Y(j{0wnnTu zrMo!hR4WA#mP8hycOkJ}IJ3Ev?8W=;Dd3A#*oz;i>M)vR$8QEDdo+!gz&+!lY-`ob z+YC~T@H%2E`_s#(-1Cx6c%2#g=rK<1SMrnd=Vq$RiE3kDf8J8PMQ;-a&BR+B_ou$r z1wpfP`JPBfZ8)yGKsE)h_G7^&p^^!WJR?1)`Wx8836;wy3N?`O1Tc`hJolENiuf39 zdy_w|!<~4^a4RYQEuK})2s3}1&{=$@cX#5Xju zLvXcZm~ab;n0nXA)2e@B=>&a}kml*JoDM7ai-FuZ&{VT<5hN9-cg4DTq<9BhKJGD` zdHa2{si{X_1Me`oq38eFcK>@5MkeTAe54j+C1}*-g+PXR8aH9opE%qqtM>O%KssWy zx#S$uVO&!!;`q<4{Rld_dCU->FWXf>A zLiTjJFrq8MJpK^n{y;u^+wDz5F@F%6*FZi>*i@xByM-5-my8uSr$KQMjBYtWXNtZe zrWLhxddw*F49-o4yg7g%bJIn(`=ks8zSRBEdN(mJJsW8v9cMcLWF;7Orx`gDf{N8P ztRU{@Qpd9?9<5u&Ui1xvpy-yD^dx5nX&DG^ni8zk;-lbzDdiB2Xa-JBk4IfQqh}i$ z=}AKZJ9t0iR+O)EQFTX=5PraT%iv!8%sr(2$|dq5v2KQ|!pG_6JS+3UxFqSE$;F#( zg(kyr7c2RV)DY4cX|O6e`1*X)pn%bB#Svw5QxAu%tXil4>pJ{jn`AG{bb&44>3T5I z`5Z8Ic0)?J6ycxj)7n5r&d#%SJ;YshCEEM%lyT_$zo6; zaE_JT4r15<8E>d+O20WvKVaN`e}cTN#m@oX!(O#{EIvN{6cy60m^N>aSn-NiO-vft zqk$oZIoos@TuCa4_X=MzWiM2`R6eUt*QpV5@9X$wIAK12=yGruZA@MkVbAZX4F?=H z6^ni-kg|G-B_zVBSy4f?Q2C^ud&@y4-YDP3?8PCfFeA2y8QQA&r_&%=r=Z!GSQ^s4 zaKORGgtB|x^5pvqZEa0|ZSw1<-6KtJWB&XKk6#XN5_O7C9^1Xj-hGRH;MU$>wTvwY zCCD)3ZQIQ!wBQWe5z|C*@KEz@)Lq)7`3QTIcmfDnLeN$d2t~7jz za*#73J!Pfiegr>e7>`3?#wsX>*`N${JPb!6QVMxI3~r|KKQBb9pdwpc`;iNfG}&W~ z-tgACFis3c7E}AO^T1!OWf42)w5`td^6dS@em%-SV&KKZwdIdjTiiL*WTTSZ?t8bv zTe97CQu5u&uiFV`Dukc(CM<(4>)7-<`d+# z$8S9BpQ^Eb;5$N}=}}pee6!8WcH+y*IW4=K;il6G=@r{X#++8BX%|b4=zaZXvQ=;a zxE)rP_Od5QEI9#JOjNTANAV&zS9#Xz%oq&-Ut6nn(!EIr;%V`L*>A_#Hd&I-pazB- zdh1PRdLyqGeVVMXd{`fFmC(fwrNeGunP*3L0q)cLK?#S5F+UaahX2Qzqn9e;939Ka z90yZLu1;s{8IZ zyKh$%Q%dxxrj6qypT~YJ0x~uvwqhaoA+gq1In*RE2Q~qtgsxn!ZUVPN^dx?5en0Ob4@w_#xaqd@VNzq6 z$2Fys@hw$7-jlk)6)o>T3A#D6WW$@GMe*FR^9_BIZ4e4t_F+)PEPJB797j8-|T^mapPmWyQo%&hZ|a zW32t@fqs~j-NLa{tJx}XHNTPrw^^7_QlhA86E3_(Z2lvwO&8ZsfDHAxhTyd1zsO=09BrH$fp^HfzUl$H#O^NZxPrBcD#_9*m_?(l69r`_4 z_XN8oDnEb(8h$hw1Qiz#ETyx-A1uqQ{qp$o)ES@bp7i<$q}NzAXylh z=@a0Z9VuPiS>V2-*?R9hXN}kO(9=>cQ2jb?VY_BdkpEMc`R{{hC7~BwU`{x0jp6%| zlCzn7DT|5AxXI>l$PaV30bL83TC>g}A%N8XKz&#;SJfU!Z@qYKgdg)@+sN(uI{!Dg z_Et7}iEUz2`Tk=GXvW7#jvsG09VKOz{K0D4PkzLeCaWnJ-?U!yn@H?skX$exeUdws z=cLUcGj5pj9~R5jb*QqZ6es0Li5ZIoWRk!qfyHCbZ-aB{m;V4$0k{?!5&kf7A84Obz~_flyv)4}GcDKTO#WID|E+D^ zqd~@R*u+k*{=OZ1w6d0}5zy zS{A(zI2AjAbH~>o5F(s`$lHhVQoTcXeazu%0ic1QX7XLG*U>yspq-!G@*q#&rV53m zaW3#1fZcZele2f}W6CqYOGUHH_{rJsJ_#wOR=B4w0Y)I-6yjKwov&>TZnCEIUfUjr(F@HIT&ya#*$C-t)D-VS_F;;az8LW zbs>S{Jpj!5DN7JGFG2^>Y`7G zG}^rn6i|aa7qdSAbAER56D?w$HV#pUx>;1;N1)g}0=`DweZH$96Wr`HJYwBP+biL# zc|ZUz)SXj#&|9@Xuh&U>Gx?*QlZlRJPD zPX5{P4sYU#1QIyE-)>p_En%lL@S?BdkBi<@m|7+9vpHenloVFYVG>mgkM$<@o03wL?P1)Z;351cX zfmVP?QjWD2vPXgWBIEct;@I=k$?BiTH*?Un6MP4`HUF7#M;L;_j zwFgOSMt%HH)Vd42h9eq`jfR-4QFfq7UVuU8xxBJ-{~_RV3t7%M=Q=e%eP_MMBDS>i z|Fwv71n7;NB=IIsO;&-gxd#{{O0cth_+cl%ovOK30(`Z({w!_zvwLn?I33~SPYxPB zuds1k(d4FvnzEtXVrSIEt6a-*0rm}BlXIDNW~~=G+3w`sLW2z|*v57kyi0G-Jo(c0 zmeqf3-0T$SN>Q(iX-)s;)MN|EqDsAW|ILjSUxlIhiF=bF0O@d@&&5ZAYCSKidI&6a zmnbapi;kt@lNHq|RR1e8(eMF@x4>Ljz$^M~Ou3(RW)hf)BlH$kt2`-y;V4%7e=TVh z+LET{qqJ3DEmGk506-{p-91NTYX%B#KHt|3UrASa5jdRcj3CZwqR;SfG%W5i!FkA- zDWPT$BE9kEf)DAZ%A{DimPwh9-A#nk)<>9_{GDl71N*Rxl5 zzk9?BTD#~Y`DH2^{?}3V-=!SoWSqoV{1pJePq~GW1d}e6Yg3xF|5* z>Q$U{`2|+z8?6^wJMizvQ-7YpsdHOLbdG(*a=~w;=;vyu6I16pxYx&tKlL((UziH{ zuWX+Y)izAT>;C}eVE&{HxZy;q6Fn8EnjeWCR3e;vUfHV#DQVFdrD#X}m-_d=v_Ln6 zzT3DIq555l&Ybp(;Fr30dg+*8!+N{pM!<^>Xf@SWk}1GItBQ3#IN~B^2uV=FhLGO= ztOrNPxpN8Ba~~qVzdV1w1ohM0EJfMI>WhJzD3a^m=6T@JW| zFMRx_2B!g|0o=3eOSY7V7GZa)YJZ^CT?hH^IYP$ zzy7%N#C6W6pC3JSjodbAt-=-ibWE>)K07j;Id|dWRW%!BWTKRr@jqpIM_(}#jG&$4O$*9p^^en!zT7GP$2mUPf6zF}Of~zB7W|b`G#DeS5MdYP5ry zn*(+f+Q?+%bg(usmrX}*P#Ity`rp*tZ0z4DE1In))a~M;+CVCn>9JO-qJ#0Dekr}5 zNbsmsiuz+>z1S&XE{NoVKd_bVklSwe++k zyK()`|Gan>)POdEd<5Ae-~2hn>kY^y;!`&tsp9?|P1`0BZhcmxM=pCUtUTVj#|b5J ze<5ti3BT065S)_!OWcCLPQ2E@ubCqKKmvR3{%igZw_jiFwl(GHbxNaIuQjH-_01lW zt8fus@xaal`~Ud*>#(TWH+~caB!+IJy9FG&yBh?hkq$u`2?6PD6ow9A5J4IVrCUH6 zLAs=sl!o)nckg|!bDiJo{?}^+W@fGR-1jHtU-LZqR`WrUQjP5!SN9@P8Rp9P><&^q!4DF$*486;IQJ3+8 z5^>*;u8jwEkxLaCEak*X+k+q|HoAKzq#`3D+2OU z9-`*P9&zsash_yK^xda&rmd&*(81;<9ha(%1gfmm5B+ivekKc|p`GtE9ibj1;%9T{ zOAh!_C4M5422r$A%37kEB%6`NS62iG2VozT>%J&u`k3>Gj(>H!PeTAAftrRr68?F~ ze-AX6CmF^5_0hQmM9a!u1`y#a*kL&xF`M^#oK$0UQX<#Hsq^D-Kl$z?qR$l$)o(&X z6Yu<@l$yO-WuRwH0)1jAFIblNx(C2#Y@dx20ti0OPYp9xbZLatfQA0OcmP%s+i)ZN z=%iSwpJfN*aTcF&qArnnugyjG`SD1QuUa9ZaMIEK^Pp^GtV~& ztDe_bGooSJO%v|7>pat;sh&h_7d%$yZkPf!<`m)_vne;PPy6h$Uq*for=h??b*TE$ zeAtEkRAUOI_7n^i`_CTk8k;W?k=DhZgJKt|&|;;?O47knoI;FfOH>F(^6lSwd?|-6 z#8GZA2wMc%hfBA=efmpCpBZ*?i&}ZzV|%}X&~v3ITF)d)=wPTggmhgjIMcPh=VrJ%@qc(f`faN7I+ zLhG`_ZM~)MVLq%qtxxZxYB5p@L8<}A^IEA=2fQ!qi}Msp1J_-Zy}hKb;y z3vZ9lb{916+Ca-SK(q`*UaCoS;6Nft>Wba3Wf$#ghRBTSuPRhAUJ%z3W~XAl9671M zpHP}+!;TseFLC!kc`lmAKr{&kie)dt4pB|NE0-uzW^)k;z4OI%iX`#;LU7qs?23wP zUV?}jlpG7ME*k#nRHYeN*{7ACLRR?{2 zs}h_|kGdrJL{xtq`G7Vp=6m1;Y-~k-$jwm7LZw`15v?@l5rzpbG?6xhNTWCie`H>5 zq+&VVhdsrp(PTRAA$a+&DXOe_uV39Ji_s}BXN`#;R~sDI$0VVikFX zI_(di3T%37UwHZaz9S#95jypOvV_Ui6#5UPF^86;^HqN+!Tx< zO#3_4<|)>q0!Vt9a%olnfqNqMr?2RQFxXo|M()j-6&ACM%*WLoD1-BolKPyLimVN( z8mlYs-$x5MMO(1d)eZBhCw7AUW)3bl_C$t3$pUaA^=`GXZJx?89Mp{yHm-_!^l?i- z{Ga&*>quoP?s3}%Ag=*hL+mqa37l5eP_F=$q_n35&!*BemO0WD&ls{?%UBdA^dWUt zu73$LYYCv^4=N_Y3HNQ9$EFBJM#&k*61%_>Msr5k4F;R_;xw%+|J^dC_y=@PQ=39E zpYl1|iQ8kYM1@uU;|<2YLS=<3FxnY;r3P@ca(3*NSGh=shF@OoRh@fKpsentS*ZWw zP18-ow-kECr!I5AWYxIj?_{k6E57#nK#;u3b|740)b&6aHd4W%Kzpq=t0{xJ(Dfdx z0J!*G&|Qs1?Thi2G|%_SabVTAacc&1Bxu&fC~DJdpg3q=h`{TcrfgfUgD7&wb}*R? zu?yMq-44It#y^%!woD72)50bk;@m)+w&Nv6gO`nQ6-+C?RfNp`T>m8VcSqq7C+gZ8 zPp%gK*BUB(M0kF5|HZ58H;^>j}4Y`s5+sJOyY%xvdM=8g5x z=}eVIFB<+b5#&s1Y`&N1m@xm**GpkD3lHRQN;z7Z`=|oAw+@A5sTZ}x-SHB&7n%aaJRn6r)!x|oPJAUSEshWqD-&BCt@@FB5$+MUb&k& zrXz-fbHsR?RjELI>m5<*T>U_{VlX64&`wYQ9_jhR%x8xm=eahzjwaC-=wN+9$MCC= zJ9?p?M0FPhIhr<#%^@{YP~+%PPG4yRUE`O4kN^5!Jy6kaxl)-2J=RH-nJ2Oeh3lHU zdre-y6gwP9A~i5|2Ua^1n-oRnOE9O_ePR(_aC=bkVedWZwv|p0&t4uld0!qKzA#VH zTLeLic!06J{FRNr3@5@TOBe?3xRA7s93DB>(E-5AZrirR3g(Tf&6>C#w$`v{2c4m3 zI)GSezag%=;Z${=op-!oFDgD@%i%5I>7bJ>tdnLO;td^hVwUt0QdfS&@Il-c1v@01 z3{J~t0(}x}k;CETb%yH3m|18Z$Iq=4yNxrF`cO;5k9XvF=y6i_f&QB@X6wTFS?QI` zTPFRP=WhbF%M;2|Y;J&IPy8)`NFK*JPbm_LYUn>+{>6<0-x zTntKh{6;2{Z8%apSWUz~HN8Wjm@RUY1P`UIS%J69zs7g0;?pA#U^)*(Ln^c`RL<41 z`}W*cT-#M&ev2v9GHimnmgxKoO22N~470yrxW(!x)JKo)CLEKGmmdBYKg+0qjbgL* zgrxlOKRzNsl(4pOrTW42w7#Kq-0@o5O(dVW+3J;-wZF|7ILgv9iRpi#XMSp2`pJs- z=eH&)eTQN$4+XV%0>V@X7Dg6DJJni^zQ4%;$Or<1c`qBQ^bb8=^}cSw+N3aMJ=jE0 zFJ%caAEEfgO4#B+T8SRt1;`w30q{y8@B3bnEW(6m{^2rN{2w38Q4w+1>5dDNwvG47 z>evYBl1F0ZL+oS7m5C=c8n#6uXD*T>hB^5$``?mXO1yq&IINV~>9!X9w3dKkgv88_ zissS|RaQzKT`~73L5ry4=l??S!AJ|GpdWjRw#^?8h9a8fKpxaoIb|Gf$#iC_tZ!Vb zj5%Yd5|YfWD1Wsmsy?O+3uBM-*{Heh0#ND2C@SyU1og$%uD(}90WPxwgu=CAVsk;x zR<(oCH0(>ivgbWDQdGU=^A+dnU)q4<@#zywib>;V3SM@1+wg3bt{9DqRjQBG$yf=4#jf2tb8q>)c&x4FcLnA4| z!t3e_(>VffdSZ$hWV+(JuPl}-&|dQCLqV9A!GBNh@h+%Wn(o(+QdDBnM9&A3ZHM^m z1EbT;!jwI#=hA2?FJg0hBt5Wz6h5wr+Q+(6*>pw3+huah?+$`L4pmI*C|*xW_v?Ckg> zCT&faEJR~R>`WH*@#$DWYe3DyKE6fIQLNz%O(upD(!JBi|N5&D%dQF+@2MZs z#yrAXt|#jdXg^-8{YiCHpF}5qtX%U@?;Zfwn1wRIb2pv$pt^$+KKGVr_|-RoRZQkt zP^Wd*1@8;0a7Pb00f{OUM~E*$)WwC%A6v%HG*e8>u4?^V4dCCV)SaV^wC z;q}KqKx2 z;dN|=LN;n8N~08d#En>raPs23P4sH(*asVDYb7OO#Hi?`2AKbteACcFQVeak`Hzbi z1{*iG=A^A=OAinE);Z~(5JEL%kueHaVG{&pA_^kIF`iP_%G-9L@V9JZ#xIC*c^|z9 zGm8+*&PlS0d_d0NoNkCmA$&18g`rbX_TLwng&_fX3`?X`h)Qj)d7Gxv@0Z}0LHYCu zS73Y!<#;wsF(}AEFDT$5b7mFf?M!`Kb_S}}MM9}#(@$V37rEywHI71(bgWjGGvt3v ztycV%e{TN(q27o-_Q21Y7iWw7>1oj zRKv6)PvNJ*g~!=*u$HyhN8csSO-~W=5)^i@PEHPX8?$a6{1%@_Ze@PGEW+){#STJ& zv1lodsnOq!x_aCU?U-LPG7`Ld<}L#hB*o4Y)aSYYL{e2oH=x49M&Lj149xv@%Svf|S3-EHE&=kYrKw={3pCE`s6TEESIw1>a~(f5 zYW)Y;V4=VOND(kCsfVS_HuhXR#cf(VvTXhw!uZIe>@t{jP#9NlH%~5%!LrrzB69oL zE7=Xz63qxhl%AcbgfcG4)xdZ#hKm7BD+V@z^*^^Y$7*3RwNbEsaOf?$km=x})RT7- z@@H=gwoMS6feH-$#D4s>z~%(NQEs50?6!wZiZG0Ef<3jS#e-u1K{u6?Y9T6q+S3zPC_Uwci8q^z#s=WEohdTNc3#IKjh}c|H0e-Uz8|B!D2v- zneT<@Dh#dwavD#vcN0UiCP$2M@ZoXO7lJ5>mdD91SOnV_+Puf%)|qbmu++o% zOz+v=>lRPMqFjW%;N=6BGccn)w+|QGwPX~5%UFNGNPN2XMB;}%LqIr3TTQTvi2g4y z(j!`*?A969VQS0(sHP1yG`3RE)27RgguH}j+5v-e7@4`RWsa}+1h9$#xgvsgIz=RQ zB=%l{3KDlvLF6qx1$Yxm%Q^^mTx8`x&Pj4z#O12+_mL%8wz-<~2fM-_=F`ESN(3yp z%L{BTh|X+|ch-}P@(SKPfY@UZhd5%%3M;(8jm&MS(I|Op2(j)%k!qPn!C|ZJeI$AL z7TT1-0;SNWt3k~&Fk;MufiWec+%G|^s9Qq;lVLZ#pf^!USC|bmyv&~8PIb){K(FB9 zsyFc%yBJ<`1r1mr!3f_x=yDT-*0;dqD|R{1)LSNs{?hqv`Kqe*Z|x-WE5_hVOOyKO ze>_4r{zuI*s1~Lvs-Lw^Zz2L7ye2iz5kWIg6K0G#*ar@tVw`f|GE=<)6)_u!z%;oG zfPMfDdU6BI0dR2mmrcy&Y%^-Kv4n_1FOcS;g1~gl910@__}SUZb^MNcA1+!Smp{MV z1y+X~AQs;r_XS&QS{4cg3L%yL*Q4bN=Ol-7cxqGRE!_z zpSP#4r#T&Q24)3-Ost>8!pvv^OyMgp&(Ixa4W2|81ja7OmLp-hpm0mNI_VN{61)Qg zAeTcMNsr6_jz3imH-IaHBQ;V-jl0h!1=LLWXh|tX&eTP$*nNgt*i&I}gERaSXK zGdRTm7=+<*FE90b9|x8yz+ zN{XiuPcNbjkzr>XfISU7d`4-D-4gjZ$Ty!ym5&n7zdB*-Sz)=s4@jr+{_J@TpoiAl zPjdg9cIbBn=!}a1-14n7B#DCG4&Z_hDKoxGTolFe#8Oq%1XuN2%36Q+{8RyxRnjFre78CueT3Nfk6$HHdr#@k@fMP= z=JFlQ;c|R-?4e)(i7>wajzHHyQ%8FaVNbAIV|_Tv@>EzCuY=C{>(?S%(gYV%EK97y z^KayNue*8aocTe?WZQeSF8e%c_Z~ukU*LQKpTuvFHNt55+J45^>C>;sGIB!1aeoIW z-lQ#rt!on?U5_3&#MtaIIjASwNZTeVD2T|ThM~UAhfpEHWC~M_aw27BSjHTiV{FI& zMWdyJbrk4bA)994H&)Po612Q=V;iMXf8s7qeSz|az{zLgmP(RMe$7TFsZDd;)CtyTJHRH7l?X-<-j2I$!~E1kyasxlC-XBz5c48Au6hqir0_OK9B z55!4?GL+-frzJ_Ff30t|1mv}5%zs(;!2mQ^wAr2al<+4m*RhRP?Iy9dL`}&zA^&(P zVJ#t^%c$7;d{`h|UX~>cC%=sOfH#y*Pam|TrgZ93>vkQAOmkWGM`5H8bu&k*f+Mx@ zE8y)3e>WJp_mWba4?L-Rl~qyGM;R?*&2YTk)L;8W7b>V-^fYp%nGadKhxW~lyVYN> zVh_R^+Q0=?J4`*U^<>u}%dD=R7@C~P%Ks+|pgcvN;E3RvSC;S)`;}oXb5xyL{3saH z@;bi3`)mD09xWGzR`X;M7)j_qLzCFw?TZ!y8 z_9v@7V}-AI{+DtS?6Muva)_gS!b35thm|*7j!m)+T#Ckt+;Qo#ukqOB?gH*Ex1PIo zqS-l8z-d=Q-Y@N@hMSrn#4z~R>0)-)O626?@V}Fwe!j_q=j$8Av0ElWKM+sB-zT@A zw*n1Vjd%!xjYO@=hUhg2Ekut+ru?>d>JU|G7zO^CI9gu?)%pIe4R&elVMIsrI~C`bhgqaGgW)Cp zv>PcuZmmA|P_-|9U^Jl>f3*)f{dflJOta#Z%mSHF#Z|wi$dFfy``#Dx2WH?imu$Mt1TVOR^#uRIh)0-&a z@iE3=y55VqFG<~3uo6Y3!0`O&!@2QVMm&m7)5=yMZYy#E_*B6*R?I?Y4A4xqGd5~@ zg>-rTUX=_lA9>p{T{z9^e+F^jB~=w%@54NK0VbT(StmoiX!Sgq9JgXwfoVzy_xWF$5`^ zBA3lob$aFG19xW}OwTulmR;`mE`eZavksY}pub|Y> zb)qH_+M_AVB>@t)K*FS`(@`4IJMh-+*{q_&gCUxalt9~0seizjQv%&PeeELkIGKP% zHvi+*?Qyd`Zz|zo%mu|5=-f_XO;LWoim~+q*np6XwFF$HNVQMSlTTLw+vW*UEYhmNTPgV1 z#=20AMlXu7{5tqlm|6A8P;Ean3P|zl=|A~dP?YQ5N3qKUcfJt?{BMYzP@Z04^o+w! z<}Pi*8!=4pfXdXVvFZQARQ7)XO@z@M?W#L$3+t66U%#j(rI8Lrjr%)G!p+sm5~7a? zRL8qxzP8oBjizB zvZ+M~+Jb}iJp|5@#m}*(pgS7PA(_2{$xlmn`v3BNlNTMBC}tF3ry_-FsSR2!wkx=C zkQE;fr#+(@_8Uc@cn^j-~IMeLRpr}SJl2H2M*4~JQc0Bp~8Z6sp-u4&4F z+KQefNmZCE96pg^#{3b#?3o1->sYwELqmgAC%10TnDCPCy5#u+Kj>I(dh`N3kdxft z!^S^7z?xe5kjlLJFVbW9I0;$+Tvzxb2kpjrt(>8?h zPtxKH@FWHbvtVJ}*cwj={K5c4XxU>IF0^yt)k%_n4TV~36y>WcHWk~KNa}z{pq$nB zc0^13;H?;%_bae!BU@pb(k)Y34oD4>e{K3=yEyaOy((18gtzjuvFbvi5oaX)NyfNx z1@}V&<7z^9wjGRPGajfb*Jeyy19wU?J}frQcmRGfjY5j#h`GUInO+VX1^Xdkrq#EKhOWUD)C;9N19;aB2)BgJJJ;DVXjJ`@WebOF z^B0q@-=1lv%9y_RzTWe)^{L#sed)zNo`HvqWA7%;cRE4ff1)%Hj<0H=^i*aTY7;h*!gNek`>5w_sNxrjsQ3^ zC~ue_bmb_nydbw`M`p(qc9>^T_j}F=Y>S{HEe_ryi$3{Y^`ejXwc%QlWxUPKcd5T; z?(g;DM?iBUVYLnkVPt}QZxL{}m$ndwG(FVNP@DyiUnt1C?E&6vyDUfehjYVdv{bo) z7Vx3vhb_g2JJ2!;sPj)#SmdL}8T~g5g@cW9ukrOYm-H+V=2>wkLux&n!5_;@X$s@e zmU@XmlMUdhIS4B@vdka$G6LUAVrOFK zFZW*RunwKx)N6qop4>MH(KtZ5s)BuO@T5M{sBSoH{*?tlNe@$9IP$4XQogW;rj#XV)LIF@QT)L)+BR=ndx zN+jie>yP?#BHV-n`6a)~H;j4$xPzin67lF+eEl4)Kph-v&7iOlZBRo@aYLema|!R? zsp=(07Zj-R?d8HdtAI)Ay(I0PsZbGa>^l^Stx;lqyxFOsXD^Z1+lVCBvk|9+dnJ^H5jgI!{L*ik0MgkdhOiql8Y7?p>E} zN|R#up}_;mxFxSv&LijyMc!u)R+LJMk!ux8%0K)zpe?4x0;Yz+D#NDxKN zWjym`mnJ-op29B`FT5-~%+&#lt-ejKX8Rw=$p7#;B2d94<_MKD=T_I2lVV1$qh7CI z+0xFGsUomo80I1w#IfCZiB(A0H+2HV40u7o3#yJOEtR5sgRqbZqaUEE*a>-&tTRYu z?_|5S78Y2=e6VT0ND5Oa35&P%=QDcrJ0-@w&cW&e(m;CScA5^6W+49U^|)M_ey@)b zJNon@P^j)`nKPUshQ=$0O@?>-%&`#mk2_UG1~HX~s`4z>?rm(zlQP#CG0p`&Q*~wF zF7vfH?dgg!xFTSlceW8LC$~%6AWoltkLg3HTngsO4Dbkn+I?4c(LOw@eNY9fBlqGB|uoU@^bt`x()c_9>O)^$jCe-Xt%SqRv$ z`z5>A@Al{5$zqhoIg`=VZ_MUSyx8&s&f7! zkm!J&-oY_<#_HoFJ<~yT5finUSJ7|JLPEehOBO*%eAi<0HJ!to78jzeMK!Jt6IeJZ`=R#DFCQI@K9$vVcf({hRFIO@?YsfzmYg&OtJ_RU5^42% z&tV-;$Q3`)GUFIWOya$UXbqw%Ct>e!zNBN^WXq!`NT>jjv+m4xxaGxng z3cUkoF9LS~>Od1lxE1a)asrkUED|wshV)hNq7{q>g+Yfe06xU}?9jAHFzj|E=76r7%*?ux)jOjdN8(Ct;G zprWpmkTPDUWn%AOc%LvT9yI|aGGPLBzPRLI=Nhve%D6pjJyquCk(!SGu|Ec6TuV{G zpkr|+RAYm}GM!ocsy@1v?`msV<;nO!-I(Ky4pUlMPe~;CrpwV$M%kO6oCpd!5-+53 zsK@DJ^H|eDlYVnsf#B*F+UCPezHmI*g7!y0k)k)(Nb7>`_(;DaQdJPye6Kqzhj_c* zRi5mM29IE7enVR{uGz85f?iJwI&o%L8{eo--45cDO6%Aw#-Ya&q(gD;9Qbf0QK2JN zyHVdVR~v|jO+@sxL`$G2*Wwtw4hj`CxdVsPbd=pxa(||;#oJnes8PWo@*n^)kWMld+gV`TCR}q24VRsTW0#Ag# zb3M^wn{(`&FJj7Eaa&V+JR1%T9xcNA2rTJ;07t-Xu7g7_{h=(Ff>hwmX{4-q5(wSC zhhEpaawhH&>*mB`QSu9fagkdUb?@IKLX@YX4Y5l{y`xfg3O3?Nv4z9muZq-Cus#{K z^r~$yk)@QT>be1asQaC!kFod;R0=1%1|8$A3qR09jnpsTp-&>OgcK*-UE})XzD@}1 zm-PoF*Q=Qbz)va9PXD)2LF*x2qzcdV4nP3Wm)(!$rv9ex6WRdoN7%XmEOP|%_-Q6L zSp$|)gFLss39?6OqR60q6>}c?wT$S)7Psf1tnc<-(b|%f+P2_M-%f901^`&FUGE>V(Bf7s^QQHa}SZumMUGf^BZ2B)d@q>Wvh@T$1 zx_+|VvpuTiI`qJ^z-N#p1ECH^n9IO!=sF;!eKi@I-y>j^?88}m2pVh%x0vWIqlX~y zbR(Dk`J7=yivZ+*7D3>F(`5ZYoe_%Y9!Rk-ZZA<5d~S-H28hl>P$>Y=DDTod|J|HJ zHzb-3y?iVZ`j)$^J-UFyPQbOy3i<&VEJ6orT9nJ6Gac<19sFwy=@RUTt$^~jXaILf z`<~e!N=M&g~?Btj7WjHuM8gE0*&HVnf`*@w(bF*|x(sG1Iz2*$3B;-R5M2q=<~3|HLPhJAD7 z5=fyGOA9OlP^YnhGd0~ywgW(^HvlGFEc5-l26DE?qwT-4oPeJ1Gw3CFlxZ7lTdx$s z)`l1Gco~q>-vjys0nzr{_TAF}aoE}mDW=Gz1Rr! zAhFy$sgZ3{$B(|ZH<|nIu-NYaLhlbrEy3Fg+NR_2SMLs6e-R%Tn6}-<)Knxr;EO=S zFWd}zz_>P?y+l29*A5W=#2-;PnecQI7t!Y_l3f)Di4G1XN>wvaj4MJEEtrPLfN@L+Au*q zBK#lE`#djQq)ulLQiqKw)7W9qIIs15f_K8i7b5!1W~9hJ=+_V?WR+gblA#(XE-!SO z1{mWu&VAan7=^;kHU0G-1G&L!6dlhC7NT3=M^sm~NjI`Ha{AODv(ckooH!0{Hly~E z4#pQzPdP7I{{)Vn7kN>I>S7n}IgC24lja6t}* z3@mUq4_Lc8q(P(YjZNpc$XIDQhjKX8RZcl#Tb9M3QGmU!7h3ev)1y?);W1fEm`BKA z>7FQMju8E?Op)%}oT26HR!bT(u_?+q6(L@i5x_QZHbVfFYmj^BRM~8o$Tg;q3LHv? zEY1Yup)ozjzZ2ZR}i&EuBsBES^6xADd( z1{skgtX@VqJ(TT>)|AVqqo_r)+d{UC@7t zuV}m4ykHG78{Kn>L_!JR&${OG$L0Nt`*IO$zwzXuM& zHR;9?MpIORi(Q(a{FMy@N$r7c8o~S-MAA84`eVUv=7)fkuB+u(jm|u~-CVQau2)pO zs=f!`Q2zZ0HPEgE)vUv&5hh#{2M!{poMoyRJ_Iugq(h+>z;pr+XSBzY>c7cmbHXK6 zvn>TPPH4x)ea_dJ6}N*)(_quQz6VEFKb8SR3bIl&8C@!Mn$vN~C|Y=bue}>eFMI^2 z9!BH2boHi@GKZ57%()mwLxSAzDDSX3T8_;#`w6u@j5XC z@DHBcQe6UjDFox!h5sGXjm+tK_6;%y^;nR_QGkOPC7DwYvdNG8z7w>ab~2Q=b{4S) zj{{x;q59dR&VZePu_Md90qWbTms=mhJyGR4x1^H(AkoLE&|lpI&w{-C+tY{p6Y7s? zsC$0vTh7D)0a&NjC8<+CaukBBZxMZP&7T_9C65wwR6E3jopu5sTTraVjlm+{x-9b< zd)z|}yGOC3=+ehwzxa>q;`sNd;0_uQLBf=)_QrfK9kpqZi|lCy1Q&3HFG@OJR*dy6 z;?k4N!+ z-m~~BvTR{JD;SJH-}6_oxD(wu=$ctxf7*Eidkb(AcH2fh)3Pvj2uYjT3Sd2VFqVH3idOK!@XcC07NgpDE~C} znO6!h=J1PxQNecUYtb3zmr{&e8q_~{k0Tx9ncSi;%K$&s{~>p?P6B)z2=Z zu>BZ|* zHe%HEeT{X0i<- zAqHU<%Pkf)AMS4veu$arV{eo<^&HcWmNk}vP5|fSK@IV{^!Iizh(s=5pe}X8iDHri zv%weFUj0;NOf>Z3PIsrY?!ytO0`)@2ns{nF$t_qIB?9~nX_qYD^q(*LEAsXOpLk%^ zGJzlVxLBJ>Zi3SbD7bunUNZxF@DApzlx5Cpg(h!Y=kNDMLjr1-FH2wTC6gZ7kUrlIE6S`5qf_m zsyaMKTs0}`!_+A{DJl3}a87I#q@twLHR{N3x_X%QV1WQ7tJ^=b_&%aY`XqBy<-{#O z07sE`;xK6*+0+#Xk(skKMNqs*M5-3*RwJo?lK$Bn>=jx~9^@s0|9U@?c#}?}SGfiG z;fa9_QB$mDlreBwe?8N?fR97Ay@sI?_}!3{cNe~3V3qIaEQWj2Z@8$C@J>41Xp*ZR z#WI}1f1|NrE0&G=`Y9T0$ni}c@qp-`JqsYjWQm;2KU6p@><^LoCY)iLFVY_29HCS2 zw&`Q8XtYg*c*le&HJ3=rsYGYZpJ}pzXZO7Vz0tD=bTtALudD9%+VA$bUH!1>a!CL3 zh|cU{KRd-u^;XqVaH#sMHnw6S&JdVT6tUh;d12=3ZNQi_FZa6PHWPA9?L!)YpTGZS6s2WS-I6m8ip$By6PdAf@dju9IUp|b z(Clz;Z>NmXWY?m4=!Ztg31#l!y(f3OT1_6_A!IhH(~%1>`qMjhIy%0ij zn&qko{m>6a8RQwwrsEz)?tZnaBavul#y@Dv;#~QJ#(U6hbE=)lt}Bgnz1)Wz%_?1B zix2B7zdtNwL3e7G_N#2EU}Nkv%1e&3`x@(3`T2H^j#FwGR-3-m641A*B*8l@*G9B2 z%Dl3@;fRJBTt>&K9r+_MD_TBX93`A1MBuIU$w%zC-<+uu-wfa%fJ1CAN8jPP{8pOV zD##Pxm88|+&!3CG={U;8x;uC+dn>;J95hxe@Gb`jD_ERg7=F5bE*8Y>ULei3{?0Qs zJ@OEHHp!T5e?GQoSuA>>$1)Yv^*!b*Km16&LXTBE-?w+0ug01lJBYNG_?Kff?x9YQ zW)W!&PRPm+=B}h6YKK-MabiCWp;xBb-~G+vzoX-MI`(UUbZpdTA+)yTXfub z&+hUymi$!q_42RYzG=L``0_^Q`s*Z`Sp{z*eOQ@-Q(W7$nZ^xCr+wNHV1Ue5W8yzy zh@e9GNg1h62V9Xw?Fn7|exHvbxnS#I|LmdU4gOXD_wAA*m9*EFU#Vy=#Gh3*#ZcA8y+r`DS|0q88y(0`>ksh{pvc_2|MI>{2d;%gk236(gxB5Ev{%(zy zk0f<>)#zwRXBY2#cg+Fi403dS-aU#RxRgb%k8<+e`IIpx3GOcZ=`5*K-2!67n~d03 zvq#1r3$(P&5!e0^xw(n8PJp)McKseWc_W36X|Ma`v)R&mq_`83Edxr>N(}+UE+xgX z(-8=@Rq(?{Q~}d`5k^sA14$p&s!m@9u=&eD_E* z>AP`dcMEq&8`+=s#>NxAAf+pJ?F@~c^I z=J<=I!-R_Y(TH(41zSPv2CJomp}_=?&0O2dkWatUym4=T>UckS?9*KdiUD>%Cba5k!q$P`LHn1BkGbq zX_BIe@pE;6)|p3C>2EuTF9^IkO1AI7hBV)sfvgh zeba}F;lk))kjRAQDc#ijj%H}2d?lI5mVu$8^OoL!dCsV2ltp9wtZr1+olR&xh+2d# z7d>8N`|mRwQac*7uCbFjLJrcz%Y7p6Y(fTj zf_rifw9sR}M}2-qjn4OmT2onwP2$Z(K5M{ewF2>XOl>|t!E2bQcQ64Fy1Mvaoqv~X zoY^0uaQa_4HGAYKeYGae77pI`ah^J3?ghjYIV&Jdwa+@lNbd;hWYmSmP+Cu{LD;Vt z7CU$PUAqR7P&_&9SCSdK8BNhr9KE>pT{b*EPDxAuhh)K}7(qsz{sl9kXb-Hj-!Jcp zt|yF!tpithVP#SwUe<07XR4OO#Uz)%(zP>$%I=86nDKDptR!-f&k~#0p&U;*;DMMW zQK(U6i~kAPq%FTe_UzObh1dQlH4>rBvB&p9Z-tWQ=9-uOI<6dwPs0DlPnD!|5 zB-{nRKV%f|S$pEpd1Z^jxCP_Ky{OEDGg?!4-IPXu=fHUR5ZB=DhSN>vliV;{E*9OG z)b)2^`Q?2r9EBhiw+TV%t{}opK9Z&?bOT!Z$Er?*ZVQCXp`r_9XRSNbhu$62B7Kw; z3Eln=TW-dlj9aIZ>(lrM1-=a!KgloJnt4r~I@D16w&m4>aG2c=HOCE388mOC8luR{ zMwAb{o>qer`5TiqyN#!(v@`FzYvAik=+Dkq}944SSl=z>PpCO-6v-NEQ~K z7U>1muO|6Xcds#SWFEU2c!4OJY5^44Y)Sme}ef^D3 z=Q*J;j^oa{kles!u=sL7tto0ihlviVzz>Y5NSJnsQB+@4##Yu`#isv|oK{%h##bX< zk*c3S8m2|A385B3dxZb>{9si6xo42%lZ{diHqowC3k(!3<^oacz{WM-`sB3E2(7ZI z%9HRu;+S9~49L%o+(TOY{w$UOk?Gu4R0+EwitoIKzZ{M=b&P)cU{KV5*!fk!Y@@+l z7c&l(VtXYJ=FnOO9W+1~Z3`7GX4ROx4m)3&x%nL{u3;$ucv|>{Med>PvSr)-@#W~| z(%bTw=ic>hIG#K1ww=oT6?rUyfBLd!HpZJJQoOjWJI7j@X`8vOYR?6$a=!jBe{Gb~ z-KH6+oxD-^$t9?BY@{SXaPLaha=D_4$4jKm`Y+H6$v2){km08ZKF4WOzFm^zchFr% z{wchPf16sS_2yLM)!!oGf_tGu<&z=u$tnr~x!+rnewIbeyUOHk zRff<1ipau!Tw|rnu6hreR?jgCs5z9@E)miH%HRCRnO3heSnJ1Ixz`~Ju4u}g_>bk& zRjs<7=dB3mL}~U32S_3H8{a9vaLlpbkv}=5W1~`3qNa?R4;vZa-@ZY{UdX}uPR1uC zwPJ;p@19V$kRfYc7tW#4?&$28xJ5|CZ-;|(P{8`DlkAd3bHbIcov;@LRT+!TSknJS z!@YY{>R!xx=8xQ&p45Dp-L@q~d1VXdwL=0MmnUX_6mMxr9lI?AB^}yJKjbff@;nK? zJ>?xJbCHym{}p(K1f*n$+UVS@*z!lko$wg9vpNl_#9wg*1ts%t8m~$lhf9(K2W!yN zw!Bph307K~!`r$aU0OVueW!m+^NT*9JP~4ZDrBFPE%QDMad0jD#6fQ56M<*lYXzV9Fy^O;*%N+%O%cO3Gxqe0mXXVUD!8*~%`#RYncyTKS49FUqlWwX(oMnH!s`SDC{`jU=y9DW1T{rjZ0IM61B! z(#A2M$wl8{eEDP6Mwqe@suy17T`6HGJZP9A_%SNO)zo%qI_9|r!wo4uF>5G#zp*k$ zlsSH)^!EskfwNGG4&o^k2Gf>Wc$pxVH>A>I%8Sgz0Oe2~O7NpyYx1k1CSQdMnz=gjcQQ)AHo{OunZM?nc|9vdRrX|rt2|g< z$$Xam;O=*oyHiPD3S`1H;HP<4_Pk#i<$^Xrd&l#bX9Wh)j+bOa1NG&=iZGFt?Xqq`Z z)0|DD4O8NlY73q{rS;MUwk67T$_Y?yZl{ojaoaA<<_|~no%y}%wvlzAU>|IYY1^kc zYWe1;$o2shP4Sj4Yoy&<`xSzU9D|fv&du^Ou@XF+YK2@?(LgZAmUA^iRko~xTEt0B zkOTL92SYr;?8Ta^dZ%K0L_SdED&4Y;rpilTNM~t!6xG^r*N^DbD$Nzu7{jAY_P1UjM^v@!S@=?AspDaGNRnKziXyC=Y{YewT?J1l*{d8=Knzj$H559XI+OGsJ1J`M(W!;kA~YZP zqZeof&^sMIQ>Y5RF3v8ILZOKE7TaOur;}hFf;k@8-|K=zGQE=*YOfZKuFe;Yfr3-er*S~!E)n673=!i zeVNg<6Ig=FCIhpju?31Y1Jw~ucro2RXtnbaCH88LtgI7PWzqm6u3h?(lYp=3cK)_jygQo9OEs`V4i&qp}C-GG5|f! zLdL&8_(!v%!C>Ey=A`cSbT~zbWS(d&zEN5jGlxSb&2&0sQsUjds$A6&Cl*n~hcWog z%yfEudOGkm5MEyNz|yQp2@U+m!ADnxaJ6JU^z3lbGXm-OY8v-kYb=+H0o@8?U2|-< zI*x4@>$*RXQ4?dH<%L#900T~Xj;e(OpA_n zfkG#F;c}txt{mQ>2;Uzw@_15Z&l6s%Y~V zA|$YsBP#A_uBeuu;|2oQBj5dqPBG8hZ?I*1<4!@9jRcrZ>869Y3l0O@VBi>W-pc9E zG#Vx*J+7;Ig`?xFT)Mc80hq?tSp^nY>*(<;jE7X=>mBzaSt$ZwieQTD!r%a-&BMa3 zD@}J=Vd#RneXwYD6Gsr<665|WH;h#~3XU1xPUe~5oHfk_pgHu!SN(U|PgU$I(~Pp=c$ z^32-&m@6~{hQg&1_+^5pP#4H2)fY^djYUsiSO{#TI4oHRjP2gv7z5&LCIq$+*xs0e zn4_xuVzf1s#Ug{CVZyG~-k7%rl0Nc;5@a3NJ<)8TJvBz4N7ab(+_}Lwk@lq|q(qN~ zNXceSPMcc3A;@mA&+Zf~QXU-hk7PCEmC@x|*;^bll?aOowT)o}4*ljmbUHQGeVM^v z8f==fmupe8gxO|jT34|~qjy)HUt^Cmb}3A7PogS9z`Xc9D|RV9~9UI3T`5sJ+$IOzDQFkcY6 zfu&^d<~uK|6-{k6IvhYtR9PwMG_iU$WD{TqplFh{NdmJxdpBjz!EKvqds%PKcx#35 z$H@@+lfqgdSF{sFkaC$~OM$TE^>)QnPPq-YiC9cnbVOInbFY8>pE$K0-=s_fEBv(F zTB;FF@1%gu{;ZrAiaY@Wpgg37kS*f$u+^IZvCURZ&>S3_MSpG zZvg8`4R_1X@2lywa|L7Z-o4GAHiVnj0HmsiQ5fu!=XqmLS92uAspxI zWrf|<616&lKlmbG8E(tohQOx~z76qCQms=8XJ9PS4uM*ihx8+uaD=mMayk-_AN}Y@ zw&|K|SFI>4y=QZGV+YBou=kdL+g5Z%dbFKa_Ei*G?0x6LmN$)Fi1FOULQ-X;z#j^^ z$F44;(CONygc4PlIGW&%2XkIHIAi#ffGv82U3U|QsR=}O5)HM>QNi2k;%4bp)SDFQ zh(09IQ|ZxCn|@Mc?TPED9ko*w9D62UVeG2}Rkp<*A})DS^e=OUv7*aFJ3@ezH*8oj z07TdZ9Uk4G2p!!ei?8?lOWkg9fmp$Mb z+eS(-(_^3NgkbZ;VLdo)ToNe{@2!!BsMTMz*>l^;IdAB3;((D1X|{1q156xJazo+4Z0Dyrc5fbP z-vb+><`Z!sF7eQ|&riwt28DS|iB8MfrfA=FCFtB|T zCe)aQu_Lzu?;+A1YK?YtDcMdh4S3!|~BIr>iK^Dn473C=*r4}7h?gZY~E5?8;?1v8@Iw6aLDxKxLDyoi89&jx+ zf0{hq^HRt+vpp-05Q;S8D9XK)_bk(Je0=P(QUn;KhuZm0Q`;E=c8TTz>A|O8pNfDd z(IL}<+jE-R<8#fSklKuLUM=g8XaqrH($KX)!0SX8l=ZSl$wOpzV%ikfpFy=o7+tli zoGO!{9^{NQrI?F}TDI`z9z9AR-O|d`~rU`qVd>4M*K0E0;nDPT`cgUa=R`N0)NH@2)m${YU(>%JY1&a%XdJSC~p zgp6_#G>vBGUJG#>2AWM!MjAb;mYNkk`vi&m zy2_jh)1bnvz2PAxw6#OdfqVDv!5HwC1Jk1AK!X=VCG}r`O^2w@86DzW}BdT~# zVWFT;AJI=d<{z!lZ;Va?HX@mJc|IFlRY-vFn+_^xzA% zSrf=;xqAuCx!?duReI5O%~2@zRqfl-aGy`-q} zp)}I~@&-(Ps^wu-)=F!lUNG9_#NlfiFekaKEglwIV)Kmst@|Ugse)hA8i6$eHxPkm zQ|b*^+H1lyZ`SbGH}}cS7mCdV<`f(Ngk%y?L{NpsueK>>st|i^#Uu_u+^X-`;8bIa z;GEEI0LJW}184*!!VKj!Yq~d&TaTHxV92c#&I9&ObOKZU=+Pr5E>fmBKje<7n9+Qb zTYkLH(M;!=`8|b;eEUT+BYdB3A{a7tvSn-TtV_1wSeATfs_mm3Mpd;bsmbT%P2cb% z=oLXs?-2U`iRrhtIV`bN=uFw}&K-R_=n{7hv^ih}-8&6Vb|@m^0HBCadvtUpIAQ`U zmoOu=0wz!7c31e$S7cSC9HB~1$`;3V2Vc>O<*Efz(T+hb5%+RWJnhetvc}jsktL#c zMl%8!#sN=DxdC98rlqO!yL6l5)_B)F)<}HgXPmL)Cn7og*i_*IXzavu>T8wFlN~bppaB~a^0u6DFeevssPesm?bV0 zxqUXzF)+Pp>x{5B;&#X`%V>7=O*yscw|}xVZ-1l8-pEsIWI6-zwy+SShfX<+Toq;< zMcSO(g%=*v9|aUHghAM_L~}-WJLwfMQU8w<*U_C3ldcgrHB=?CjaX@-*o_hDtAL31hOS_OGih@ zuG0K>)bMH<^2{1Mn&z}=I}1ZYZ~n~##&h~GxxhA%+%;JKl67;M zDtQNK4rE2nx?$$TNVy}ENQ!{gDay?}0?UxQLTfIjs{vL#o8k#zmNSv+O{#QypG{4R z2u8;)8Q$DtX2+e+klh) zRyt3U*@-MZ3{-GI6*9 za9)~ZqlxKv3SL2D+Ag?#cQK!cDyy>3C|o5tvHObKcL9oCbD4}@+!nH|yL#{~pVP0m zRb2p+!tp{|Ejn0qcN3;O3o~!6RW`(CG&>B7N;76NASRuBAn45-a4Sa`%%-_cS(v?M z&z9ox@iB}vXLzr^X>$wXj;3~jjc+NBU&C<}%jv1=Y_5uLa@q*CE9w&fi=t0Pv%*_2 zv;>@8%LWm&W1fYkiliiCYZ1??{zeiMWKo6fy3lU$I&m5LPvLn&Dx} ziTLTMV!i=bhA9E=l#5WTYt&NfJ4wr}-3(fl&1l-tY}uCs+n=eW6BEbg0|srMhi|#6 z*u@Qwckh&QNk_{aioE-!4b$=l;N?mFv+iAun-K%p$|{EubA`HWTLAOV7E?QA~94l z)L)?Ac~yvL@TG?4=KwreG=3b0FyqH8yn0Z4@HK>KFhwFU+7ek-mE$oA-sMN-t30;_Udac2=z5ysF$02n-lNUb&|?iHm@ zdHndXX@skgo(lv#mkdr|Iz7nn_`J2ZLnu1f*n;B(hA|;1_7oF0n9Y7_%}BP726nv( z<1qx_kAs#WXQ&&}hP_T<6yJb1$r$Qv@f=_B9Pua5@p-GSg~b)KJ-VU}>Nkln^xHZ+ z`5;Wce&Yi$;%FGY#Y~S{30{tWHkq~Dao~H>J;ewbr0{K3sqL7m6$sJL7Ehv9OCsRO zO0!6kb>{itgAa7X#`e~3;jNjOs5xjFxTi=h!n{XA+Q@2!>tMBXx{;?-75>J>(_dDm zN-@o&GbJg{jRaW))`Nuw6X6U=557gv(+$IC#qJQPGHrt4z&GmpKrbe+F00BxH`(&^ z(;!er1w#Ki93oiMKpGHg-Ngf=9b@E@Dz@Zm;YT4DPcUI}sYWS!5S$w^r8K*bCP>6w z0l=beRRw@4k7jo#DCR)0dOtyYOoG`` z9H{_|8h#0e?9X%Zvxkwa2FhJYm?%i41cwg>IV7fHULh1Ey@}JhtOzEJKdL0oV44B& z6|GHM(M`YjwupX3ch)KAUV%cGVEy3_`y$S(C2kL3mG|t_78){|X3R#u_Ez=oyYCu% zXsN2KD6#@iu(qHvr--0Ajy#9m-kRjcJ@*}|Br8sD@AMjN_7WBXe6gjr2DV}f7dJ2| z^jLYs%>RmBel(r3n|=V=6k9|D&;@kw4;mZvLq6F4S2B#cE(ER>iU>IlK`_L|(bHDb zzzxihigp)@jFz`NDZ1H#%q3OT(vxz01X!J}bpU2&MJrd!nR&oCCt@BmE$Bg^0JD|> zSoJK6@O;R5$c>rpWa2o3&x-GBL_R}VcWq=l?UP0a8AHpfO0$qE%acod;^;N45m+N| zV-eWwIc_DH7t9@!nIghx!)Sc|Y$ub=bYLb`J4}mKlOv`t+ucAkL;j4M1EiDLe0u zF4jdHerjy*7MPPDG39pZ{=$O(YD1rrkZI$9&9`sDRXGtkOxiTq`Pr$^!*oGlCI1oW zM@XIeaN2(h9X&!}heyAnbyov1bpZ{udCjFrmd}VHpq8H&8!KY%)l3+*5>-q9Q)tWR zLIehYRDAue76(!Hk;SpashO>RHYby90-$xHAIhfBJ)JkBQ z;J{E=d|_na1Rw$f?;z@RnZahIbU1VysI%;$ee$X*B|Evc&|n1m>R1=*I=V^jFK1VW*0@zVIMY|2pnw^mS5KkNnUH19Jln93fjb>I5a*5FblJz*2 ze<@e-izh*}RLqkqZ}&EQOnu==(S7g)`5f(#&sCg~T2^#1*~Sc6=86C(72(~wP6PK6 z^w_wlqQqv%rCSYQ#aQ(?xXkhcgl?TwF-kKjw(R~L>1~dEJj|{!d2({%2NZ?}0G;_j z6(Sm6QI#7X+R!wMuH%OG|cFC^6>=05ZNf&HjTp-dBP%EG>8-AqzK48Uq*$8 zOO)6uLE0JXXo6HJ&fT0)_-ZDYDyq~q9#TDvhKa*l z8&JQ%qSn#T$KG9c5`_123^2FG_L=|^zFbuV6B4^TnYIMr9ztzCY=N8q(@IftnUw=z z1l1-rUU`lj0{crzoGg;;0tg*EDZT7zrNI~Do@LLJn|CoR^78zY!>3u!I$0fu_E4yz zAq>JY;WaqI5BmI35;Lmd)T9TWU9|`)6J+Di3{xD`qBW}20cJFG;3S1FbO?#ye7=-s zEj7-o>im#Sx%HYkvOZ7;$bXz6&`-C2Jpp0T*!sI^DI%8)bzp}2fC7x#rV%ux91^|; zCZ&iEhTyQVAqP~~CtTP+KOD-sGN8r<@jjlJOq_L?1-9#sTOXVnfTyOh-Y_~u6Q zgI^g~s&K++I9JgnkAR8YOsGXBc3%ng-bgJ~Jju6;lz0RL)4KsuPKl-I)#0`m<8f7`a(MBB4g$a-br{%8aYC9!#g4q#y&Eu_} zcXi#*`U#%iIXXI`f9X$6mRqT<9rUj6 zIwUx!VHT-~c8yKY=S482x46A%GQWF8)BMXisph`sDaAI9q;Skoa>v4)H+urZ@Hw~9 zP8xEPrt5a#kiqZ~RM||ZQp}l@r$MeL6mlC57)fLCQ!*l{s8fvN1*p^btRpCe$3E$li!I1>Vn|xPD4%p)fac3UiSi1SP$bs3GN#0O-E$mdE z#asoZKZOzW5UI9y!4!D}zSsh&Qq(S9sT~>z#$m<=qVF$MSsk^Dt5zDd5-X6Z(nDwd z(~O^f94QPKCK@LIyO<1(Giou@ph|+oadfw9x7TFX)t*;d*1tXh{mpNFV>lfh9T|;z zWxz9m_8yP8ruk&uVo#erNzqG16y;&ns8RXDtmgr{s3memEU{sz2~*L7bDs-`;S7z3O&qCd=NOE zRsb|WBW(=fSA~)M9LTFm-niCO{6*1|Y!;9U>)iRa8lKX>wJmEe9YXfz1od^u$E#LA5p6 z)NO}tZ>QEvHZRu9V2(_*J%0Sy^^o16D8b{{<%01tm_NXrO1u zByZ*f;D5@!I~8{KL2bm12fsaWqv126LX=l%A{--2AtnzIak7FbamHtTLg6ns7&FE& zg}zJj^|ebPNFre4s-C5a+N#u2o9VHY6%LHJs)-E$;T16HCt#W%$U_>L#YZt$89i6^>BGx^Qt)!C%)^Hd>5rp}t%i~Xh<>VSrdro< z&TsK83eJQU2urz%9cE&}bH&C9@sB_L*yP_e!PBQtE%Y`3IGz!;8O@&>&MO-5$25*% zgBgtSkcwzsj{wG0MAV77K{k%6bjod?h`^hC@QVWu5i~x66k4J`wdeuQcw0)LbH;7B z=aC_*tjh=r4YoLxNFG)9M?@sHBp9?D%o+IYx8L@5q#QG^a?*xMANjqfD`Nwuu=G&S%ruU~AL95#L<}E+Uo#;Yidm!39FXIyMMLIu;;Iyb z(UZ3^v+nw%TJv@qGyxH<3Sl(ngWX`n|EAQXGKQz9dyW2Y4 zyLWHiH7#q+VtxeJhIRNuEY%Kk2(u6R`1shX=H^FushP1!N?sAa`LxMYk!<@F7%x|G zT7gED*UOlayYSYMitMISZX;-$<(%vVDTS1)1x6?$TDQQOD;-NzjW!rM>EVM^i^}oU zO8FCgHXl#I%Q=%O2N;H|ta5^w8B%oAw_)N5_q_DbW!7dGyE@y2M8d=PY}S>f76(Rcgxu!Lb`aqY@;OV`dr=7E zPYI-LM;3F*LFW=eQagZ(Gz0L~V92QVLVoNDrtN_Na@g5e>YF3mAXg-gi_XbZS!)H{ zVNWo}A`Olr-s92i5$Jwo?AlTFH!s|_0PIbk9cOsfE$uE!C=pdfTQM7DB59K|>gLhy zhNG;vSo+9#)?aqXdP+m~>c^ze+j6^Mugbj*qsmc}Z$2()g)R(e0sau5Ty+eBjeLpH6`EB%(QSCxwz)x!)J~iwyYgn*G^M6E&E`LriCqaaf1uq}s#* zxZ%O^_@Je#BIt(?iQk4E$90k-!kk=fYL|FZpBoB|a@FW(5f01-04Z+^UE(AzCyuYN zW&@v0do-M9ST_gSG)M=L6 z^wXw_tOZUjwfG%=`i!3{r}so4IpZPCB}II>9!anworvh#l4h8Sn>L&_ka)#7>0}gA zLg3R+8`{dK_|7};bgAuVYTbYE;DHys1a*g6Z;!PXc)ZloVJQ+gj2elxyFfrQFq*k1SG7ews|r&NT%#0;C?X734{YHYSnm{>F2aG~B^rV?Q7}gq zQWcn_R_e1FMO)~kASE0cB8z_< zRnC)UN#Hg#i92crww_rI)6g=UQMHaqW?okS))>ntOkOpugGsbmNLpF_$T*=?MF!86K4zS$Al{bMG$bQoP{$*rao`T&YH zgxe*x9VW?2d1W<4u(CX9Fvmj|0sSVTJ0N<_COy>_-$B%?DgdT=?!d+z%~WAV#L-P& zed{Sdv|+^?s=in`s~m@jq>miZ{2;>HY_I((_mPEtUcYlmP>Ud=HHSqU#R6Cx3wv$f zD&F7`$--0|J@~{!hKb`P4GZTvE`&~lTR2B&J})bkqLR$!FU1>V4IcyvzG;vG97H(sHz}fGCFqaM1ZGHpXv)m z(^0$a#PMmU(~fGktYkLKdJnumz@=IKq60 z_R2k&UQjebz$9v6@-T1@ERL7zKD-CcF&uGr`(wxQv7*d#&oT@3J0f^v-6cdSirYSIh9CojvsLJ_GB=Lih$`B=@9c_wsR4B;! zFlYx4Ie;ZHA3Sc>%6aWff8tM`JfU0w^BI#8wNe@sF>4D|h-%Rx(~pK%6<=PObG3UM zwM9Ua-2wn~Qqucm^_o?nF*zLylTIAnmy_v<24Gs^q(qPA$ng;{9$wKP;))+2{}BKZ ze&UT2#W1QU$y9HI) zLae#?z?g|EitTs~hL*_6yZCc0RY#;V@i=JDl+6shjgwoU;w%qF#7Cf(0|0-m#t-Fq zo)r$jG<~rJ>Dd%*`v)zRz-j_*2%MDXIgWUGfg!w3-BkI0uuJ(aSb;q(__+$tN*A?= ztTUQ68~`-;Fi8=|L`bUS_&9PY7$QFnC*K))QgA{7n6%a6cs=F;r%+YRN9WH>xh4ve zWSY6b5R7*}xp%z%(K@=NS*sLD?? zj-uQcMsOf}d?O0cPpFIp*ZPY31(Ga_#fLUUKHDJKW}v-xs=Wb5CxS~;nO+?Bk?*%)I{RDCD)fak-9x4A3MC6bX+It&k8+gh7 zV-)TXIiP{0&^BlG@kt3nVPQg|+%*t!$oSEjLzB{|Y7sd3gO89;^A$_fQY9%8gu#*H z*lQwK^dU?-x4_TX)7Xu{H#k|r&j(u&O~C1!aKzcfF%Lz2AHEvxEZW*43}Zbg_xo@# z)_aMHMs&G3PXwcyu z43Jth>(O$sF;@|QxYlI|z}Zz!%E7?cMrdjA7Y)~T_LQWE6G8NMmuw%w90PDV5>Ntw z>Cxak-|UQ?-v~1I{`>FSYO76^96%G?g831?$;5}sJ~HJ&z^jF%MA3g~ zLjuPz?ip9fC&6vqN*~G2VFq6AOZS{j{OiA-VkW(&9~=74d=PgOXQwZr=l;vO^`2>?vPl4hTl z5K)yN<$eGqTyb(+l|&mnS%h4I*{Xl`Qxr5m4r6=Bh=1iG@Vub`q#f6TYR>_DNGkB~#QS!&VCH(SK;_ihBDBfF zC$&u4<EkSyd(~Rk*1f&aTMLSC?^n(Wv_&`6OZIKZ~ zYqxt5i6aI7yh+neUPU#C^=SFAm?50)@%ul(H5^=fkf-MM`rvRQspJ@>9)_`Dc065Zn z?p54)B&6gyI;a5`7Sym=NDcu$Ris@DufKUXoqBKjk#G|$YI+u5pX;&3b`6VBn@ z;Q90;bG#}oWo?l?Q>0)XK78md%ta5{zScD85AG1@FhI&rBYSnJm(BJDSX~LD*8J?F zE%S`gt{2p51t#T&ho%8T8YgNIrg2{-Fw|7(B1x!$Ju>1Sm@~c{@nd82%63s z>rvGT`U$!S&MWsL{>BimYMPwRiqXY{6h43CXoOsq986AxtCFIWAyYI5TtsqGC#r?? zXPzC~a5UMSsWn1pNqW;?u<~^kabs185WYaMOf)1o>=0`%!e`n~nR13IS1LGsuoRjG z88Xs5R&uTa6v;3=!O7uTMB&Eu?P~+{v<0&J0!%@(08wbT0)#Q%5U%9R)d>12;jO$s zf~LK0tBLokt46XS&wSf75l6Qjx$q?J}>qjW0B*)0W@ct$UykTr+wmw=Es8yQ}s^AT( z8mP&Nq}0Yq4;T(UrW}(+oQ<*5^?G#raUYDtJ#&awd%gt9l1)E8PGEF|D}D;8GQycC z@=G7G<@^uB8HZkYvWhv6rtSFn*qIsAL?P@rmU2Ie@ylOJfPrC(7G4{Mlamv1pu}Si zPn=iq>1QsBrY4&}N)$P!uT6|$%uf02&m`m60s5p)oGCG|eQZ)IRN)ZMLq&R`IiOg= zl+eHvpQ>0%RfPZJ8OFn|A=K72W_#hKf#ye#9*MSvPpv7-UdT_a(WE@ia@}DOF-h9f zD*7H-mzgfex?^e!&6bIgT}*pm7eKrB zxihbF!{BETfS`wzU9?$1HJF~{=1p@}n;4Un zdFM*}q}4G{9#M{ltefKSQ~J1 zTb^1uTI+0jT(Y=ShVgrKf2iE#ZqiqUW6B2OnA~-{zwcn9Pu6Q&DF-YqJ^-G`+`D(r zx(j1#(p!rUKKNiRBk0^okd|{fEApCYp|$CEU{-W@!z&IXXxT%1p=Y0D39*pOy#Ron zy6qrm(7DY$KSryY1iKw5gqS}H8PjMAW%pj7h} zxh?p!qar1G*^+yuFCYLmC6X$5AE)#l8#p}_)_fvpFgY3^549RHCZr@I_Yp3w#yVzk=OQf;gbWX?O6U3m#_3VT%D9S1Hh|94j{9GmS}}1o*1d-Lfgoq}6l7p?npcJPh{wmr zxg>h-fBQbUra`Kb8%x~C6LXhZOm-18^M{AFkH85gH8epSIZp9WB_mT@Q#Qg zDTfn(c-RlA(4Jp63q?t-Z&fgTsA3YaA!V6`56 z+R#bSFP_|UWa67J`hAda=|^*^f$BFkSy5|-q);N&v!;r8CqN2eOs&Qq!aGyJR!d@GtK)J+pvGs83!MiPhl2#BEn7Sv(h9O9(mN(F!eMEs6b`) z&^a6uUs}Ry7mlai1f>;R)lBP-I|9lHc7d z;qR$A2@^-l)e`Zif-0ij6qr1DtH)hz8TL$6E9ahPrzX=0@8ObX#rFtK>F_u-VDi}y zI(qzxr%J9?a{BUTEVyB8yY)*@$LWHilxBR_Hx^})G;*xtsq%mbPHv=`PDOM@*E$5z zo_fQTKWLvjS^n}+@&5h$<|wVOWg-sgph!PjW4XaS!yBK-gfRzdAbbee$)qD`v03||24&@ff;UzOJ7 z6+T5#KtdQKjG{?7QM*CF zbnkCZg2@>UeCbPH;&9_ftwgiD78NTwE8EHqA1wul8HYkl3BK=++12~Z;Yf;}S#Frb zj2}B9U`*{rcy=nfOSM~Y#{r{X2(l6k;ZDSa3CXCU;JAo%9Z!juh`^erww)(Tb`3m@ z#g}mK>pmI;AG0DW;Sn%a2f?9mhNXKee94d|;uvC2PJ|vx?A~2m*6(VMkB@a+ms6CG z0+Up^1$GgZ6}KVQuK+Ol0OqOe{vF5NqW4%E@3#$S5gy+#?(f3FZ`@kb8i6$eHxhwo z^WcrxdMP7j1ha$OgzH#nZkjperR4t`$OKBEsfQI)Pcu;*Mbgn`B1E)x1+0xJ!7Qlq zrpu!1bv-ZoIru`z(&d)r{E|?QF0@B*xY>YF>mXSip zagPT2Eij0f&O)|sL~;*d9*;gRC(D_v^E`8oZ5H*;@VQ++zoK`3X#WadL;UXG-FM$* z#Zo5@m~#6M$2Lsj_`tf-?%C4PIxSq7?ulGm6hMOZWbmY{HwDl&4G*QZ7o^nEZ>1zv zZ_T8R z&1?+(&;RIUuSOA+NZyKY%#sX4n>P=N;#qCZ)E5ADX`1g84ao|fNf`;Y9kYSh+S$!0 zwAXic!zBvSsi!yzS|5p_HtH!ayWlA|$W`enzbEGydROmh*(uk$o?817p7YRQk6`-c zOgj%425Kp1f;l3P#TKDy@t3yH_~kVjosx=qLC^yQSrsBHHlU`C(ZC8Hyyvu?N?Zd+ zgM&~%$Zv$lKbw2?n!^6D3tFO$a&CmzUT+j@nisqP?&%y0!7w3nPi|xFIdVAUxkMau zl48nh-S9DsNf8UfDCIDYJvNcF!3Z)wS4~i8%rJaF#K|Znm;iKAA}3gLR4Haf&Y6^_ z&Bo3p9dOPV0|;}D>jOurm<)-$9lS9{iV_3TkM(78}#-9 zLY@Omb4;u+45Uom4FF>;5Fed&8i>z8n6963e zEn!}PaD%P;_wTzdG$GTe7JfWY1fUJFhTv3$K@$O>mXt-%?HDWg@N!dxufRMkX(l2L z&DiiERUuSKq~rolRzl0G-7@~4$IwQCEFTir9p1y&vkbKy5M<7>$R!v#)8@l(ed}A))<)uq@D@Iwb*|cqIHetqB0jYe_h`nE(D;g! z#Fx7_v>ZSJFo$sWVL9*w;V{&nnNWpxS>`B9i4&>S5EvZ{0f0vMct@3@$Dbgkgo73( zdot#w zmk@?Zg1|67;Ob8-kpfdy|5BS^-d2=L1i{qi$+)2yp9XZ(9q`8=e+*18B#xUnU$Qi+ zA;T-S`O!vT09(i6Uo>)9qq^i(BnwITd+)s`0yq*5z8#$b%%Vm~TG!wK4uhaJFhIpT zsfa^Q!xD!S$Mg<8V0~AD-y{$iMlXoDXI~ZTF11Z%7BzIXV4~#|Dwfoin5YNwsm0`m z+0RHw5T79Z&KLGvW^sK#pK{|(uvw-X4G!r#(nBYuRW{Q}X zH<}&_o3om9%sz_cs#pQTS<*8-;iDyUoE*|?L$B-cg{wsl(*$Rsf}zbf(q1TGI`Nub zlf_TmO6(5XWwR=G6YZ6OrI3OU&5&>WeAejeUz;lWDaq|rITAd5`qcbumm_~c++6c} zwfN}KBS#tqVC9a~n#Y_ zK3Y;mxyun>n0q534x%LN_;d~|@G>8r-dX7>gGvMp2^dmj5lgq3@Tv_!7j$L-pk%3Y zNvooV1~1k1vT>`6S*fLvVx_0b^{)YQbaZ4BpqZrXl7YnpEkuft)0~_i4xM%G_(OAT z*@-_@4UaT?SX_TN&v+^s^3j%;mV9dyToAl=(_j4eF&78M#Imty;^7Aai8)on$E+)H z@|^RAw;-qzTS4&s7eCb|6^yEMDng23rDL_oSh>F#dnlul7X5G4G@bDi`31N+*0?X}i^*1hlN{$cm)K}tj`kmve{ zCglyV9zNb5@uxf{jRw@OsAR@9oGAa#Q_78H){_T^~PKL|c!IB#bf9AT9H%H&` zM+mDAy7^fi%wCkvyZD# zGeiA#gFMy`V%=P>>wQZqbGf12hOtm;b+RyI({4BzpCfBQRYQAZ-9oY4T5TWM7|JrC zy~|OTshnEJ^Q#xI;?miMM=d{&po@Ua@` zJ~dMa9x-(t#SZiSh>%-?h(^X1vE-v&DvcO{T0_yHp;IhVwrKRIAu4g$=v(q-B70>< z)pxy(k(1!p)Qxova6AwPmZra`*lX@nuWG()7Lj@^02S7XaP(W^S1e7uWZmiiDFI^xC zBuMVsiK$J)Avsnn8Q!+>>kAf5I%$|}gaksT#$m$AQmELOz%EK?d{icIcaqtP6xEQM z6*#6il%)B$y_#E+!>BoRZlS+Sh7OiwEM_$Kk}CNp8k?SH*1FqETIXyEe9Uz^WNKo%-NXkO`l8CM)TU~8od4Hrt~Fjx7Y$6Hp@dE)5)*{H$@uOM5b;uC0(Q`YsG zq5qXbJM-~VF&~d-kOLcGva;~wWC##`^*WtwZlpu2^@*&_6k=XP_kGx1W1je(&+|4XJpxQz;6|mEfp8b zR{~YTa|~2PVa+SM`FqKS(&^?zQjbS%`X_6!Jdq?W)7raYBmS&?r-K3Pl*-C|1UOz=Z?UQ1W63v<)FGjX`@?+By3WTm z$VbPQA4tYN^CzNA$Tvb(yc`a+9-${=sE<~qPE_iM3I%4$YKU0YR}#xdm1b@92pv;J zGLmlTJ~g#R!)j-=UO}+{s528Jn#+zriTbIyuq=~PC_y|d4=4%+$*V^Is@W6G5WCQ* zUX;h}A&jcL#U7K`Bf2Ph4l%v3`F&Q3^2yhpjuh&ids9o={6=1dS(?1g4JDa=L2cIe z4?lZ;+7j3lwA+33bE}bP0CHT;HWV!$S153w^RC0;zJgh#msb5{(COhxeSZx_3Zb?= zRrk4vkVuAi`%Ez@na?D3TCw)Z=YDzExsE!1|H;h2@}>NVC8$YTg!8pP{vTi~D8m6L za3dul9){e)2d&*Qve+zib+q31!(g!@Ju;oVnhUQ0%;YEzZXieRSN%`C^70%@6(g2E6pjIm`-cj+F}_!^|uUk3)wm zpp;gqb9Zbk)?|8b&xS&MaT@&t@n}HAkdGZu%H-YvWWRxuSISbwd9b&>YRCIVF)O+4 z^#ELhh{!0^1=<>ZJmNf-WYfl|mHWOsh3ildz86F}1cbkY4*eP+Ef=Tqc4GN+P^o%G zHs;8@^llkwY5>UlsaZSGUv>5ZSLij-3l1TT&i0%qVvafj9)3@A{$LUkTapd^W6>_3 z7o8j*W-K*)kF}crPK2BV!A}??Ej;aaCsdspSjznI5?eq6O7au%Y*+N$Ebm?m;Y*G> zzUYSwt00rUqU5+^ z(K!FQ=;^-KSD`TJ$Y6Kp#KrpJuFuxN3F)@-pFCzMtHgH;MI{iNe5ATWImr#mNtW|4 z0#6d5G@k5+iT*)P*klYTsi1vk-(W;>7|vD3?42y%wIm{b>b+A#>*1LeVk-v2NF0Y_ z8R#T(QOmzHToSWh`gfOKQ8g?Zk^Z4qByN4}Szi}-xgfoJW$>6;gpGPBStmPmIrkrX9Bup^owcPGQI*!xBsJQFpTx`~|1&0jM zGp)}B8kh^m=8VdsmKH>XG zX;If{toq)F>I_AWiSv(oZ?X*Of3zpg0tD&C5IuqMsv0eo*UIO`>IvxKGp_Zd<19lL zWD{5-6>qz>MP0icJr?OWu2m#W|FjgP!MK4!P z8Za?e2iIQ)x{smw z(nB%HU2_vah4+NdvDW)#I=#=HMH4Y6HSNyxIAR;w5CMAiK6%ZFY@UjN!{ThE@!`)p z3T6y;{r_|@){3M39nnJl&3U827c8~*^JEj(B>N@EL0HK9-gI7Ps$%lbjsfOfu;q!M zY@ZM=I>HsyLT2)tz!%-E`@gv(Z(nRCN#w!pMXI4Gu#Y+vJ6gj)MJ$(#$Cj3 zSW?claMM`cIa>*)BF@qFdZiHww#Q){}J4oz)3EB!=71LiI~Z&I+NM zJfvwzYOX%qpk0=8?e~vO@0$8S!v97ix^EMfd4vCW*O=2&Gd`yzDgZ)_YP)oQO|`_B z{hk|cFp$~UuJ2*}A(GL)clgnWrxxUQ&UEpeY<$&sYnRYQ*+LQbSqu_SQWPiE7)WBZZo9o@<_hyU9t=(4>^e>5>Ll%sOr~)b6Lp>f-;FK8G zOL8q@9w8to^n1v;*W+)YXGMno(XpAR#*LX+*Q5u+DAG;K0Lt^U2SFSmm*}nQEI+K3 z$;FbagpgX7S}2~_H{p`b;=7uGyPt<*M-Lx>IW}l6&~NR5dnoTX1b-$aD5SucLHkVrS>g8PK;#h3RU&kT(oSKJ86va|gi@^7fGVDD)*KbD>4NhUULr5%C(dcZI?x?@uu{%XsK~HKZxApD*&y%caQm zXzB{TWORiPE0)-&j#%G@q<$0pQ zRL9*EkN~Cu_VE0Edk0>p=Dd~ou^gnjjrH2sJ+W$7s$DOoBkiu`#q+n;+V*G6vspFVInT!{N9-0F{1LXY26WS&%K3`8iF+u_=BN#q2gtodh4K)Ai#;F z?!1&jkXoueWh-{+Dz}AtQSX~eDOrr8+J~9AmKFBeFKz}d3S7@E$CFM0iM?mEp*L~+ zaSzJ?)uLFqn`*`%uBW)tW;=8Fj$76auG{UB#TUyHqI;D79VE5dr0p9@*U<(sVX;8( z4w=A{RuM)mH4NX*%Jb zCMa+j|ibJ#A)qn}`H#0GSzPqp6bs;6#xCxx4BWiKm+8^)?k@mG+GdCDmd)z~Wq8yJ5a zxM?z#+kc_@)44QCNY>30Dn#pvms+*AVitRSdPE)ik~H*21lM50h?P6tpz4c?Z{`ZC zsW(?r=Sl0@|Gaq5qJ>xMKWL^=tf1GQ=<^^Vl63e}sC~=Id`pE+8D?27YP)?k2p{@I%Q*Vpnvxp5s}~TT{+? z8nvCfcxt$x%e8CE7VG%bJj=@vE{DN*mqfIaK7kVRDExnTQSVtj=qLUbS?mO8z9;mQNMhJu0&(jUn#yOoF8AjL9aqAp5cjSRP9MT?3nc5o5O zq~?k9C4)K9hz^S)3OT`n@^qj7FAIQ>n*gn>a;}Z|tq9sp6R-JnBjRm|!o9mXb-ow6 zp`6gs?4$eGw)un-s+16$#db2xTk>zbV3OjFPfZkTa#8OC@baW(%i>n< zj+~nZ%~5l%mB^X@i@g00*75qE6WcBWJ6{&V5o{axpY#ljVYym5$&57?nI5bzAICJ~-TCfH zDs6?WC%NVp;*k4`JuzA?VI35jWxj)#eyt{v`XyqZ-HwuT2(n`}^Oo&KlZ6~XVNOdzpu<}h>q3}3O zGyy(^458 z zWzrpht_j_Q7)lJKyu?&j?Bm=Wuw`$-vRKp~Iqd5@f+H@}Ot`xKSrMp1MFi1iL|EMI z9{4t#aAQkK7E7AqD#@pD&rC6XEB20WlFzg1mRm)bJ(~Qq7a_8*i+VY@(CBFC=A0!g zyj6IaC^Th2O4I~{f4U^Q)QglFLL$|_9pe_iCj#%HsS>CbGro#(GSEoJ{m7hD^iH@`AhM^z+{5&b-%> z7LxCO-#fc1i`4XN_>JKKklE;bFE~nD<@p**kK8kbzvLOK&|hb@F1u>YB3bH8m_vQB z6SpmRQfRwK0vW-fFn(ZLavk6+XiEN;H$`6ct~h}OFy(-A;3&s*p>}elf)0138@9Oq zXCPhzc8-Bs)lBZgPuV7tF~;H`MlN>uqU>xI==L}+A7Xbqi!m&ew#&a7N?cJ385K3Q zH!nyJAzYm+7o8%C$GrU5ke$nJL2hRv?|y+W*>*Ez2WwyXF@3pHRV5%XjaWxGEeOD@ zOFkp0x2W(?3B&6|k7J<6jrg7j4apfU*XlPpq9dXj^I*{mYLno+w6jBs9jZ9+M9W!o zV9lQweU@h~-wWeva;PjmgV;y1fyXh~m|0o@AIqD4&@ZiG$V3T8ScbyLvG7UM7mp-= zx8jkXnzQn%gCO0B7e+e{&7TV21lImK)e{BLr0e$ zJn4U2DiV%bK1Jeh33z^0a@y;icXR=e;Zwut_lqtB!x#Z6VvLD1psq6RCaRe zkWskE?h=ip-j7Cw1U2c#1Jx`%5pm>sOVe`RH<}MNm*lo>PZ1Y5U#21))Xs(X^?#$( zJiTr2%3UUJvx6}6Eb>kz>&s96s2s*5!fAqqg65%c?7U2Ncz1~cUi}0ED{v@^zXV(iih}(yH~72@ z`silF9n7(#+msKw1{q~pk^sR(2Y3mnQAjAwI*8&-6Dxq~&GhK{3j5+S_)&Bc%7{6C zx@Y)(8=HNM%YVh}2Nwi?#h-fbOL@}Ly}QDPm6V90-xIB6d=gz9HfS0wa)izN_;eW= zKM=`pJP^P0sVyK+3anp3mNt4uDlbvOcw;jlmtma?396%Br=zH^58?4ELCPkMwYX=Oh&D2~ioN~J2cN0W3>}1K3Dx3Atbc3KJ zQ4j)2^&Q#HO857Ax{`xcoTBPMOsZQ1y0dXsL7!zwTU_Nul$ex{$l)+2PvSZ0M$;9L z6Jf)ceb?8xocWN7^GAdCc7WXtw+>h@l?n;0f}L@?`FPqhrwPZsZIp=_|9jVuMg#i_ z_<}l$|Mm`p$1%6+GA;gLYahh>5bMMOiKRxy*8kc=63CHs(Eq@ae(Jqw$uY*dmCtu0 zQjjpmg7@nSc}G&K)C7++@1C!bp7|6GQLX)qyUy3Ln(sspfR22BF~;$Yzvw>c7Lo4h zjYc4})y5Xk?g|uZn>i8KOq8tCxZmksk-0ky)HRkzF%XQ8QNRSleP05EXFigjU?{+e zjlK8DE}JMYL580>6$V#wM&9*?g=h9KNE%II`L78u&P1RtWhiMz$*9Q}2UtlmkNMXN zMg0E_|1-$2C!p1Gxdr@uj>O#Vr)$kxJnm8;LDlZfCv648a2E@J@tn!RI;!C^l1P|O zu%wD)m%aOz#JGR)Q|R*7$^5tH+ZK!GtE`^eraH&%f3Rk$rxUGjcf%q`h!oEhDc0oL z!;#eAdxM@oi~mOK9TKl+ao11LjU>4LW25VH#rA&t`~3UgHAr&*4b!)$A8##Q2qcO9 zL)hv<{)b_kQ5O_>W3cG$;K%rdVIm{Ze%Vm4_&Vp-D^<=Z<-`7|ObtFMy?B%}~iecDpptJgA2o>+&X zo}UVU1^|YlOc4QM*dyfces<4nS!iwkHzPW5eV6?EV)BR7pH~J3PnGTM%cu2AYh^kk z{u;u!l1N(G;{`I1e~2&GGke0+&c;Sgyv~2h8BM1t;SYxM?z@&@x^a5iao6baO1Qe?f8xItdZ9PlgNGhXm5dScB7D6)6`uFCwT@-Jad z{+lSDltI+xxs$T|q~K#Y`hfWlTaB0L?g${~^na$W@82O=z(0|*whOcSNaClFG&c#= zr`Z&_7gpbM!WcuuE(b~IA67=bO1>Yry6n0B_!HLJ_BEr{C#z#4lBZ@?2pGLfkCRKD zwNDIg{Pci)ei@y==J(?UZpzm^(vAXDLRQflfBvvZ-H#z}m+RL7lC#NWei_%&dBEfD z{bBu=9!0dkANSE^c8&j;9G?jPQ(k`|lWg-$(;mQ{<=1gM^jhka!R+<}5><<9aTn&h zxkvZAb)SWA6~>s!31jaXR|)4+Hei&@++txkPR} zayMERG%liCo?zs3Pa2D(I!D5IUxxijQe2*V16TqXA}gS{UQn>SHLF9W0KVydu&Zt{ zLCuLg-FMNWT$^UWRljYU6B}VjZcw^|G4>uw@5m%QID%9_Qbmglg;zT>Hf~Kv1+h|| zg5w+OgSuc@J06`63BCh)HNk+`I-cV6@lGws5UkcEkhIz$thAYR0Nm(E%6|o@q&7p7 zkYq_Qz#U|9b-iqE>EaQq!4ls_EkYbsDKtj#r79g2LsBY-`2lv%wqd=TG7N7X7h!9jvX*LTdBd-f%b5!Dq|jWe-RY=s9xF>8aUPVf!eQw4?Td`S3PZ zdeo~)EwLuc998E-f+1OZ#AO-d<6MJic2Iwi`RlZk;k2Q?I?=rggYK)0tj>KFpKn*0 zEw)?+bEGUo{U$wo``DqQs3Q;W9Pj(J6l_$*E$iuAqs%Ws)})!igf?mx*ZP>}<^_ z#I6UFix{&TMGWSqqR3Qu{mE_U69Q!u;fZF!Y38v*GBDZYsQe?6gy-EJ%{K?(&xY;m zn?K0Ia{u@iAqwzx!kJNpG>s=^^d^1o+9=pMAh|6n1nHu`D#=4Z1Z;8@%-|Sa=~;|j zmwud!pU49n5@@rB#7_HH7h@2y<^EgA|C~CsVisq{%V8c!=CDInG}Y?(SpL&@djh$C zctNq7-qa01`7&fEM|~5t-jH|0J)={4->Ck_j^Ca|k(8<*Vh8k;Yotxxrvn>X-n;v6 zv*vdiAGt;SxywB|3!)AEx4dCaOdfr-kE6~#+0sSogr#xD%RXKp<24_L!w^h)oI!D0 zB~Fo5!*Rbns3P{>hrq9CWy-|V0iBgN45e%P16p4=8##rj+=MKI*TTMf!@g&CHzeGLujuPg>O70C{-p|OQl9_z|i)o>Cz@<*d% zMN{&Ce{r9T?$emT9j#5!&jM*NyX79bADC=0DPl&goYbtd?>p-XbR z1bT^`&+O0f`QMy{iFB+^c{u#rRnyOKMakg2P855u`2vlwaOfTbHdW|SLl|1O#z-=^ zp|3oHm{x~m8AJ?~CQ4wh3bD&-c`8r&;3*WCzN*8v?W9;zS{Egny)XJiv$9@nj5Pt% zDj|w$pBXIsWteu;cgX%oWu8zkw+=JW-;aX(n3pUED(XOYy2dZ?Np=v!TVI);2!poK zNzvQ-@ZGkX$9AibDDr>hS2k-9d$94kH!>Np>dgpm6rR6#jea>CBiG$w@*m39-EV+K zjGnRW^Jgn@SYr(j$_SBF9GRTH8b*>3{Dt^5WzO)hK{d{rtbaDU0Cl&qLIng8Z^XZL z`Qp!-hKTgtLyvg!D)-q0wm}HFIHvu$wJ>4D_QI&W+Z-3MO z>3`dXT_j3|%1AV-@Ar>1HmL|JB^TIYka)h^BSRnIud5Q!DtqZIg{nR+M3XA77csmu zh}P%6)t^95TFO}gK?7!FpM1hdQqhs*l-m|uUQnY%OES(4Dutg_zE~)X4L&T zVt(9!sLpTk^%~>H$nWVpz8$qN_w=7CUm_(_2!k)eo6W_5{Z=I|V-n9_a(6Rcsc%aQ zJ9rwhr4s^UqwfH-z<#w`h+cvlVu&NHu{l*E5u}Q<{Akk0{^JKWHcg zJAvFbRi`l+e`6g2Oy1)>?2K-=W&g3+VE~?W>cae6NYNEz_TaA+8VNabBFtz^WozEzc7REc~b?4wZ$u-$~Z9x~U9+H=9v*!L_hnijwXxNGk zA5J|by|OLe>);CL=uneKvOS1GB?hglE)dg;z?St8$kExM!{&|iEbl&tAu-{n8lFXhq z_Z_WTKZ!Nhx;f^cQ~Ym7(yuR^xwAlbE#?u4{@wxm@|S{dj8kY`#ZPsT`;Q0-_!~(N zoj?+Ziyi611ib`o+2$C9UYU9b;X9)3`Tk$Gar1H90y=>5@2d?XTr}OVz;kB{u0!-D zrhIDx_Zg1#D4Na*b^SJ(Li>%XfW~g^^bg3GvEokxZf5q_)xWihnhxx9`Ws&yR5Cw0GWAQZwYbw0 z{%3f)B-8||sJ{Nc5%;(09z;@nVTYrFSv=ZghSwOzQZt@LeIZlv4_Gzsxr(r zBa{T$!8@P{uIXiC*cnM}h-Bg;T_7`(8P(I_N>l}^gi%|8t({bMqeH?|SOv&hj&NzP zvR*qigoQ`h_303C3W&LtXOO~o8t8N=mH}YwE}<`m9~%#g5ZFyX1$YHRSj~=;XG{jy zuJbES++QruekChkhH+aEnR@>jyC{?#^M46B5>(MD?A?WtTq;xfC7$bn(4-O0i!aD& zFrD~hMTWT??KwE5;=v9km83Et=}~6_ zG-FR`HT9~rBfK3;{sv9V`O}n;qsF*O7u5>GGW|iuy)L@B;mb*ly!DQ^KYxgp{UCCh z)Roj0e+7}&oyOlDwDgs^WY(NMk|U3;kd3w=(^NS9lt@^F5tjg=0)(QDOy+LTts7g& z-Tv4#@mtKi%5)=82la9JqK;pqvY|%Et7QOiG%#V3?ATI}38i@ZQKkN>A$yr39JI+TiaKhE;N5xaC3hQg?YLqYZIGsM#4up^L)7yT%+ zRmgY2wL4#^=xOQ1CmFwg zS+SqImO>_4q{>gPOhE7%k~verKvbK&fzpt1BCR`;{cxYv^Z2@j0^IRuebN{;A>$3Th*fUp#z7T)}4n?DBc?(d--sMJexn+auXO60-ogfzM->~*bY$J+fplZrWE`%o(h&Nf1%{@%CVJ!WQiRfMU*w`^isqs zixL%#=5TMt0%S1`ve5w%$Yox^ws_nAEY@vN4uI@!;FDu)DlG1as9CZ6e5PA5pa_$1 zf0imwLH)Cx2^w2}W&}l+M!{SG`L#+PX=rF* z!i?q%Bc9ys^uJ$uqErVaaV;F|;Vg>%pi;ocM5{P{TN)a;u7Gkbe4qUn;N;R`qys_8 zUH7jgQzVt<97d17n+*MGKBv#j&~IJNWmcBcqy1d>_w>i0khIFXIctM?pU-{6-lJ;{ z0OE^XiFYB+rQ0wXYt6TTMzQHAV+^AUjbTM3iPz2+4{HFw4|vo9Disb945ponxLXcxKR3drMNt)g+ZUmQ9Z_lK z^S_8Tlh|cVE(IEW^ORl|;dFWVG#HM1RNXLr6-k=KF3|o_>%WmfIA?%TmDKI$LRczAQ33m zhuUDtn=SBkx_>!BuSKgD=6-bXzD#_`N=5+oD(`pa5kFwQ`J1W^yx-jA2U8KXEU}eU zpM-r(z(z{B9QF=$!l#qE1?6@VPD z%5bKa?1>@wMfnz%VFvyrm&Lv6869i`owxa)LS9|tIhr!KToXJufLj&AEk<;5W7!|< z)o(2I?(rJqLl8OHKhT=V)*?VbP3z}IoD|XXfR$!+*trsKtCPC5@o*PX^l3I{lhYMB z2-m=%vieQqj^e#i`JOB;AiMA*92UH^mAATuU?z*N&#C@w%K4c^VYzl-{0!ridz>oG zM4P%0=njWLK;})RaNLcy9l`v2b;av^1j_Bi7l$Ck{!?x=|F#DwkF!3#cEQtgtufa$ zWRITG=xtoHSxVMc6Te{_!vphRhoIH2v|YbP(@tG>o2%z!rjBZ4e(s5sLvhSWTxl&k zRi-2^8)jc}Gnx?ek(rL2F!;ugpcN<0GiIPxo|2X-!O0ZvpN2nGduI~&$>N8y2fZ}SHs-P>&A)5;s z9J{+u%lTXPe@LR(YvB0@U^v#f`jqK5C9vSrK^&Cg&Pj1OrYzq?*giI)HPkCnnR)81 zEhWpIAddY!GueH$uae|pCQv<9iWQ>_Ws^67O9_hR_vfUn;|Jd8vop4BUi~p|bm|BW+2)-=da@0L8p zB_Q^Z2D@WM`*%7&^QIt$M^>x+NCnax4;W56%L=@E1(f>XnSAXY)jFcJooQ7|+vqCJ z6(F-nY-eRGXvW&4@Yzlbc3~rDJGw`S$CE#(G>b?9hv*7rI;6hu7~dEqDjZKoAVXL5 zdm~CxO{{}Nz&&r1d;>pwUF$&Qa+z7F-Fp7S@E)1n07C#99nzlR=&yj+vk#KBtnoNi z1gMNB>?7{;#PfXWA#Xwh>Ezo~wh~D>VubAZ@;|7f4>Kgv<+?Lb}3Y6b9cgIN;VUf3Jtc^+uxQ;ILFYhW> zX}<=kiEvrQ6E+lVIyPiwSDe?5b)Tf5zY-U2+`H-QcOoR^zb?O@sbMT8j6m^Em{3N`;d&k-C=2tI;>vUI<*$VEuQC; zF3_uj?OvEox_W^sr@N@t2ExEHmigX{zQshu#Mdx(EzQUm_Ojmg`#^a?^@_NQ%sReU z?ic%6^CY8R$Ft4HDY#7hJy3hCGsD5K#Z#pWw=ZTbcg=x~=5rd*)ifXtUBx_^y1Ssx zQB7hy%g0*#={?xQ+lO~D1+Q1f!8~eQ?!Q(}e{pL{3)Os#!lX9y7@QB5rW{&opZ~E; zV#j8oSim`96y<=_<(tCaj-znr`{o6q)Hs1zpZt}{0lkpJ!x!RyaGJ2-ke=Nt+gl(8b6Y9X3w@Qr?lm~M*a1F&) ztro)4=p7;st;3C31Qjr>l+&Q_S}}#T*t(!6bTtRv!3t$2N^{J@*=+k3-rgkzO7lk$ zW(WG}`^u26zUe}ln{%V~yo>r7L!MqQQO|q3kJ<%THG^$CA*xB1?X0?uQ-9Ge&4Pn!l&x5<#g*cgjH zGc)kQMhm{Qea}pg5hY;)y3aI-T}|?pv<81U&XdONe)x2jmAQl+_Q0G%+7LNYn0f-h zs{UBE9>25P)8MH%;N1iLszD6PDnK1m8Is-c#aZ%b!j|uyG-(^mlqb$_N4$JIU^|0m z&)yW|I#6z*k+mYN`0pDTV0s=|-u$Pp(Ya9{T|zh09QeQk^N|rs>vUC&` zg|;JI~ikuSX0Jh+?ieE)plT=)%@tg&YvD;z+6r5lUthDekQxlvHM<{ zV;n=!o(MYUvHw{7IwU)z;6<9*w7%qzY(VFqKKmBh5Y6)Y@0e;?YCpr7K9Yl;;01PcgrJ?Pl6Yydk>~3qef-qrcIvM%6-BpBPi$`b zN!D77`8e|>0Fd?VjA;8VX%i$rbU!wOm^ul9B~3N?L#M0HYE~?SHRPwd_7BZbe47HHR*F9mkWMu6}>`Dui6@4miU( z^(&%=d2P_T+2gZ-cR-=TbAu0*$!j=mLUE0&Iut8DxjuU=Md>n=0p`bq7_$?Vh}a%3id*- zY3DdD#1X((Tx2j|gDw3F+l0b0A=9RX+7W-}{2BasVF6#XB|~AFr6lQ5+&T$PE(v}` zltww<1nbnJ?~@bqkctlEkA-v&so`NcD@OORTG0aF`|BDTioXvbW-~gMC*xg4+X*3= z!TTtMoP!F=P#Qw+B`^c(Zoy3lei^&~gZDQ$>6c<`St@TFxZDD4c3E>-y;C@cm{a_~ z9&}2qKF3?BPBSyj)ETa^QW0V0GHJ-s@AG1{OZ#&j3F>nz`1{d$QOF9{nu8Hm`Y-Ld zgYf#i=2vj%2{y-=&z~G9SIUVTY@(KEUyw+4YviDF)VZ(V(I_|Uc#c~Vx-1ezpkvlY zr5J4R_r2dB0pNUgUgkBxBJ0onO|*WRt+0m*l=cw~&780%_IeTsr_XVSPiFfoOngV} zJdp@pL0k|g^57UI51Qoxd(7v&&DsgHP^|-;3$8; z;sH!8jz5tfqZ|;bMH~EQ9k+qAtk)R-pW`&yUsPoxLy}MhPmgj8V7Ng_v%}A(HDq}!MmZAG z)_!l(@{?g$LY^;>vIv`bGo40w8tk7(hJ{rr-sq8!*Gw6ZTyzZzGN>(s4Zac$*KN3i0bqWnu6()u-utXV=2}J|}e>-UaP+I84QC+QgbVDH%vI%COHPY5kowR&Q_t6Eb32_P^Dd z!~bLIEW@ITy1z{`^w2eQmvlG700RR^3=LAk&?!>D?u&cjg88!k!=C$V@GG9Cryl!CPR z>x%3C)O9dnJD?j^;s@$K1j~7xh`p)UJlhXcBd;AU;SoZFzZ+RKH;2D5oxQBXvzS;7 z_&U-ku)TKiC9Kr6z&Y1stKIdT4ywBDdGB1fi$lf_6Mh<+?HbpA?r{zY;696?p6Te9 z!KtKQdtIC1S7m%-Ui3bg3CHlc>jRMgJR zNg03`f&B8^{B726qE(2Enk8B-iTcyk%yujTxxjPphGM6%{kp2$K5or6)4Zh0oO@Ic zp*f6ThJwz|rRDpBZyd9^sO}NJOR~gFbX@s@-<^Gk&gf&?O9*R)aFWd`Wq2O!$+`%( zoqj(QF*Aj{+&&evB2O_86fhdy?ip^aA8*~MOijoM=C8^3v6UA^_%FqZ*nhR>SkW@! zx~55lS-!Lw=UPTx)oxOgj}p%Nz*{-$`qi~W3K0GeGGdJ#oH>l|?&oypRH-Z4>jt{i~H+kHs-9?dX^5OyyH_1%ygBJzSweK?R4~vCaIZINGhc1e-{7 zi05FgcZyl*Yq^KSPWSR2g_9;E^hhbIY<}hcv5aFPIHKS!e*O13*;(;K6RR8jsFXq>I zd3oQF)$95_b4Aqkq*up4q-^w`%w38DkjGjI7I6*@mU6?T4;5HLN7QH5_oAvT$5dTL zm0N!#=W8DYk<={`2GknaZ>XMIZSikouB%}JgEJ<%x;a|$ z41Uojw&f}4`3jKGW!?Il7&6c&ingLGO2CF?T$9a81%fh4J-qozGl5Z2;f)rxG`S(! z#pTs^Vx+@7jSkwI^-HYkzmpcZV#5_oGxpW(*@|%M4hgTO=A2&B+ID|iY-{?uqhe}s zvYoth0$Q%{7b}RQUb^1owPs0^|Kq-BR=$m zklOe0m5hJzRWvi?I@U~N`2p}Aoct3=_qeMNok&|pCq^whkSUH-2pXgj@sjlDiU0z6gyr>DS>BIx>96c% zC3dQ&Gv)wvg(!U|4QVuMU&Qk-6t?iVQFAQ3n!J+Z2Hx46?D6ds=};4Bo%vqaPVERg zoy^t1-Gcwy50K-yRQ{e$I|}?z*S+$2sz|Ex55DRtfxfmb0khazpiZ?rokH5IMn;5{ z5PkQb;k(g)4~675cOnJSmYiRP5~TLW=$q5qk0Gl|UheL!lhWzXf^hn;d{98O_ajz} zMS4I>QZ;|j-F-Bro)%squ_bf!_}qaq{D`I$$9;q%wo@@jCJ5DNaoxM&8st%E%a=+( zS^XYe*s%3bO@|yD1v^WIYLH)BP-O&r8j0eUUey)XNzTY$DEt<{st(o$F_DTw{P_XG zd8s#6!+%qcv-ljp&Y-KxtyR3to+p$$KHWc?*SLXpw_csQ5jJ68gTL^|SKG#0!|LMW z!R5E*{G=uNueewQ6L)|tE)d}n*waB1qt z-%JMw>b zXfe3pp2@L_RB_tAx{l$#skiP9t_zp*B6xZ?AIF}_BxHX;hZ*ho1YlHubTUTCGY*$9r8&XfU%7kDZuvH+3K zzoB!{2`?J3kFdtO_?^vUfIYmflZ##Jq_WUZzP;^Tzzb%3H896sg+T|cjHCBWtrj~B zKE2`#%_QpQzV}?Q`u+aF1KwN|-u*fan@-US%%bF7Zb#yBc8xGtyVSg{YEauy>xA3KX zmy@+gy;~ye0OtH%4f({8MiqW~@|j*1*hul>h-RIv%vW+QYU5~ zr-B|XmT}`Tjy=CT_#{qbsH)}Re@g3(|G9LnaWtAm+0iiJyPUu-8vjMXUiM=f3Q;{MBcswsw-|3|TkEX+^79l-HK|ZXoHqLX*1Kha#j6t$2x{em232lErKXGrY z3gu6#I)TQr(r3xu&$jY7~I~h z-_IjQtA2Q@_>}~zsTB9#>a?Ymy`pt|$}Dy`GQSuZ)z|d4?3C-@^z+Lq%OAHX{vXYq zZt1Ls&c^#Km}T#G>8zi*oVKikkvg7|#Rat+2pt+?+%V0P88UNEo`GYuvskx*AB&^#1@bnfB)s*r+Nn>v zU^U$TH{rJcxuCXtL+hed;i$^ChEjm+;b)R|wUc(dGJy__=P@t3thKA0)t+$ezB6K^ z&T$&}OiNEej{pMji3sZgieD$_1U~B{fJ~3c&ic8Z@1UAaY*Xl@K`|IJ>H+(`X_!+p zAG=oOv||a`yN3p5xZrwrq3v6b0?u0Yu2`4=M07XOAlaMx9%Iy0;UieI#2bJC!s_M1 zr1F5J8{PvVN`_hPUb77d-&X^X1H+ifF9;K$DANO0P+Pqc%q6jt56>oK`({9Atku1x zc$?^RC9t0Dk_#f6^41_~S$EKr25Xq}UA*ePsyC%MU8+cI)hu&tw9IFM?OPTZtmu9f@Qb77g#p`VE?+Y$=iAs_9B*EqGg@ zLgj{x**1dB_mBkY(n>%!Wl?j}W6XvN1GO9WfZSV*joMP{V%VKRxB{>xb6rL#+7=lMJJ2MHU4Es1-ZBvQLQy&pf(DJ}eE zfN!{Gz=qI<*+D3Xr3eS7K!r3EWMC?+M1>s%HPa_n;Gl z=j0z&S<(c#Gxp06Oj`tS$oUOl?MC9Ag$Kk*Nn|RIAJ#eqGhjB!i!yR)7cmG5cAydM z-U52cNpx41sI9(lCs=Fyl>i)=l#F8=3sheC*?r5RMjd8JWm#15kR)YYm1)bL5}N0M zfYrd41V6srVB;WDo$$*8sx7oy6az@!vVZ**o75{n{iD83SWR3suMvvHS(~MxgxSsd zUhCfHtO`VJ9{jTlC#QOzt2Dlf*VMfx3a{oxyi2x4zRfN3be$SjS4Ik;aZHkJs|ZEA+1m`kGNSmX^g$l4 z-*(pJ6JCj_66u{ohf3_14mh~8uD%4`1oA&~4^Cs5FWedwDTKU)867N~;Uq}YngkQT z0<2A@?R4Oj!9ZOf*YGl0=z3Uk+Euhb-O#Ix~^T zsWgBaGe$Geglp}iBR!l{LZB77Yu_q@yEGW|E%@FLmia6t+?Ml(Gb@|~5AvJjn;37p zKd_JF{)x9&{W&@=Cwcl|)*zAwbDKJmL@k;e3o|45{Di?@WA{#Y9cC5#1i7FmJuOc4 ze_R0ax>x1Guc~HPv<%s-Qo=bfBUp^cr$`1e+$}_I0hI71d7L4ff+zGS9ANbX<2W*P zw6y{fB1RB_y`9QN!1m4kho0OqCu4CvLbHbSbyd8xy~FTabz!wnGi;@Ir9o%a{$cOw zq)*>OOJoszf~$f$sD{wm_B_RK@%>VYmv^hFLE!%0AyuCCdC5*X)Mtp8A2SI`M1oXE zdE3dL1k-(@L`+Tx^@pSn=b(``YBI&bj!$ZVQIoW0$by&h=`Gaj>+AkeI z_@ITZ%Ol|ZT8Ux|&Qd@I4~bTEbVe}z;p|A=z;&dSvhD3Z95tig!e?qG0%T9YW2Iba z$NT^``T^TH7*%g=5#`v8Ei>{{sg8dE_=OjbEenw!XD5!XyQ0alroM?LxwP21HHuPC zra$0odMvU!C#c8K!Z^ITJnZB%v}>5a(s&(?#!rY>3KmpQ6d&{D^bu3x{Z!t6 zVbL6y*j_So1NiKt28jSo1Ejtn89|vCPU19cdTaS851(@`Cl(fcR74W%2VaiMC5;dV zF0fw1JVTE&n=>JfzvU)it$G<<@=QXogDbI5F18KiFn~)Lo!oYB3?Em5}|O zBJ9UvOz6~jQ;=9ouP|FUA$mfXhy>)qF8FP91qD8<#C6sul)2-ESiSQY;?e5lkLxv? zTN=+{wMtHLFPht?`uO_aUo=@T$F8Ah4XFx%jCpA7xO;jA8`$4d-4cOiL=s;c49nKO z#kQJqmVH6G>rE`w9socxley|~#0h;;INXK}mc}Pr|J_m{MK-dxQaG7Vk^725=YI(U z^T15GUalF$>fa0O{>m^mhp=536c1;a`g!Q6!-^JZGD;2=ZDIvM1#cYX5>4iRUZ$We z{UzDM2#qfjmN<|gtcbmnm~o?LF$@woVB5|i=cwhG3X@Y6XSA9r7KF)@@CU=06v1l$ z$TrLS981VF6ZV~&1k~1+LXbBu6Q(QB6vCR}0BmH!@96O}=OW1sqp8gYEa6)LZ9 z%0m4VuXvF5^f$;GvjVs5{sWIY<{N#bQzUHktr4yFV8ZYt)HfO|HF+7b$%r+~oUG<& zVmLTg6-P$-ts8GsI%3y3{EQ?iD34N=W+Z8XdS9dVwZF%q1FiFqK734Cu<95KYt)hwlK1tK8neW3n z)8KUtTn2B|v2!=cIQDU86t^y#z=?^(zBaNoOx)$Q=BwUQ!>!N0#<`@=6Gab#{HV~y z*wiPb`#RB&j$8|SIx?a^tn@p77sm{cXL@~<$O?}*%=hH$UcK|hNU2%U-K16duH$*V zvsF9GJjtlv=eD6LZ~3ol>cn3RB0Td#T6Rot))*>D(%;I@q3krLmz`u0$}?TsytLU2 z(i@;p7ig|@^=-Vz#$~BnO730U=i*V1j!8`NPwl;T^tz8{qJ(K$`_k~G+v%7AaffUd zd{V@35>KggRcszh_p5JmyA`{2z)&va0Y0pPrepOx3zSh}ve)~CwO)#Gu z|I_vMBg50{8>an{kyuL*e$U_Cm)~G88l7ZJ*mce>u>>fgSgoJXyVK(FY=Y>;1;gk? zvJ)Wf*pd~;jEsF9EG4@>!6tpC&^lq0?`_{>DIYW*-Qk70cg>;BSD zh1_ZLmvP_{OBkAh5Li&s68nX*kCyg4&Eba7KW zlHjyu8haE0W4yH@^4?$v$ zvsxCx6QV(4a4dwDKAG-V1Y~JK>gs7*10;i%JhJ30V=D()KN)tSyG0<6o99ZURQraF z=j&5G5y=i3)843e-WWcD|>vszEj5z&F23$OOlWV?X}VK#jnBlV#VQk8lGP8 z&=Y89(bA?uM3i^n-he6{|B(Fzv?3-;;;G(be*}k%vj2_%F$iCdn<_f!ncnB7n1fXH z71b`{4c5bSCyLaSIY-Xx?e}Zd1Q~SFBJXL^k~kRKK;aoe5SwvKhEzL>EW~rt?R4TP zbL{QfEiHFRYx`e@vFv-_b6l-^eBl%WtJ5Zb^kkImb7M1J;MlcFz*zl*5-SPQ=;){Y zOQo2oqI+)tye%=(GO1*o2Zpw;l7OtySEbZDRmQZ=e%a~30zyob186-gKZfe15!h4t zJXR|NfkHviw85&yvf<&jujivZ*b*rt*TuKevNwO}%OUN@a3EN% zf}XQ|T%lNTnMYWd5b0By8i1>%Gv(~F+#DAo86uP6RwOp$T7f_u1W}LOV{?qzfPEex z`yJ1)9fOM;kBfET@k*!~ojYrfzKsNTnHb-j7$c3v`tuhlz21{45gX{x%6!ZwM__WQ zn3nLeousG`eh5norXk#GoVR_lVeZa}3J3ZQBiuBj`A_g^BK2IYP)O9(-Nm)?F^0 zowNizdVDYH^5;MG{0skl5)##swqx2f7asyZifU#Cs)`bQI>_E8kKnH5-$p<}^A+M? zP5c)31>tL2c_zjL0P47xj`IU=!&OAI_eW4^8XJks1>WS`8FfBUDg-G9@;nyof4tUY z&kf6u>en`eDN&0$#0GEk6XX>rNW`=GK@y4b+!nb-En|Cc*6ku9a6|VI7rp%>539(CoOVPmNu$Wi9iU^P|XW4bq_Z2GwJuccK=g zZjy;aH&N2tpxC$!n8kyUL-tN0@xq3TbahE(yw@erTld%shj&~%D%_IS?oNW1v}EQ8 ziw&$IQ5nr~!3@u|zw^NIUr1jtgk_fv7r&esyMGpPFC*!937xM-n)R#xe>aC>cJC%- z^b~Zpu=GqC zTqCdmmJK~3c}R2XsqtoHowAS44u&Z8ZyabiBg|~%FIZhtc#eTM-wY>-Hkr7*!v`Nn zI~@6xmi?A3QOh2GDXaH&J`0A&;1&rbTg@?OffZU#2pTn~A`nH?Wwg}^K9d)#K9j;? zFv@rjuY-S?*vXT%#^F@J#=5vxcHnz86NNp<+Jz=}MSNWj5Tm{sbiGp**3NLc{2kJ;P4U~1(G+Th;q znx0>$tVhw&ZH2Uztm+Dv?O8%ArB?pG0%D7{iBC+4Cg8_~s%oVlyuK-cC7?W-8nrVj zNut+@tWWgGhKDSL-jNv{e^QnaRXfx=vV9e^az>g+KUjWG4z;}3*Qli z-)AjMX)0i6IUI!E{s~ofxb(l4`lZLc%u+%S$;*}(niMBrGVbGKX5}tp61a~u3#X=Z zqX&hb3CW59^2IlUFB4;{{R^d0;f#3giFj<5n+vME*ZE#ELDPO1G+CR1;1Iw?pH7`g zH%mY;Q7R{QBupaikY8g^u%SJK{Hsut+pIyiWdv!=xSb3sm9Z={1pHGiW4#WWN-e}Z znzgv9Zv}Tr&ZUH_ic7H29yH8JzS*t1+x`3YB*W~MkHy;L+Nw5m*^XubcQ59%C0TC| z9^0x9)JF%4omJ`kQ5*Tm0jsK0B-ux`CFEDGxWYK!aV|modz&SNe~&+owZqm{|77gp z1SAe47s(y>`Ukdol(?~cAJbkwwE^dk(fKlIR}Yt+`XU-->j{58eZ=V4Bi%v`p3#;9 zgW@p+f_Uc>#YZ~Dg=tjmwO8GKBhSElZf;lO0g2o}6X@2|Xg zJg=BSZ+y`y6I@!HT_iH=*Sxl7X|nXoI$KGgaeXzvOdCVX0c)(2L=Zfvl8^_|#$+e5 zNUlHdy)qUvXyE(O6Sx}&dark8`KwE}&?D(I#wzjcI)ON-N$eS8hajP`Fxk$_GQd!S zy6l@lHnOQS$_kv8#>x@_wtJ4^B@@mZ3^i(oex#KEG31b3aYk!ivD2CQU|-~_`1W;o zFwDMu&+|=Z)1!2}NTL)LWzn{?2X~wmEo!zPyOWXuZ`*I=@-k%mtDc6hyhp@)YwiZ- zxuyJERv!09Eff26#0R>N*!KYi{}G@34=Ejx)W=qcb%u_MWC81yG8FZy$^`1lDjurdee>pFkeQ2?R}|fTFy+f)W&Zx9Hn%fmw43EQ zs|KX&KmHbWQ?w&R@8cG(a3hW5BIw|yT#eyP&!X7ZF2A#S3cCre0dIv`ZySea)j~9& zu)3*|GEEXHj~;Q&RlUAt$lT2&dvvO9hhT0nJjSKlP2b)m)&aipF|Frm#~>$E+A-a{ zm6~7~`Nee;8R0DbMUVQ9Mqv*$ogm2Q!7Zua6{mdOG;_#k~iwP5iA?C}_?heDN#@{pT6WXVs554e2EUL(M3OEGZWPMW`7Tl zk78!qjWhF04yeF}NW)WwHi9wARL$Qx&(wt0i6qs- zO$Kr+@h=)%bYvVSa*70!>kwGxL(f?2Ik9$O?tc@*Z##_2zI$~dt5!fM@x?}DpUfik zX|brg0LoHliee^@SdmY+wn9!eJB*QrQ#V^*+f+76-8;l^gt|xnvWPeqUp6TAXgkPn z%gbQ>G~N7*apd&m9&1ODHe!P0M9KoIq}#*z;GQ}~6-(;W(OrR;&}kHtimNRZoxims z68bc%_}%9q$63yg?;1s+lAUR%U=d$E(Ln_sd3*BA{(HtqdaU1S1SzRAFdAOqKM(cn zL>WKYMiBj{j&!RAdjNW(zMpfRuLsWwCXU1OFrKAkghMqilQ+MIl^ADs7uMiSS!$5O z?lo6DGtml)orTw{^s9?Gsf!bEs9v|63Vu8^n{yz_GtMM7gDoxekt!iawBWCQ`u?3R zK9Eh&h8oac5YFQ@4VsvU#utjljT7m}CjTWbnXD zuz7~0Gq~|%)0zrxrh-jQ z7_wU;u}tnCvsh%<-P(h&27i`*77$fEriH3jxk4g09RV!y{YmX4yLjyRXzr-%-gwr$ zq1@ZM3`4*Ppc``xa2WY(LcMKuUCPO9n~pgwDmxxMzEz5EUMCSAq=u34M5WUrf#8+% zYXScqjdZdAB{VH3dNDCQWF#P0?NE@x*MbUs17OgrB0q96pHKlZXsJ0KZY6Ix@dXh z#4n>zRgSsCo6EF*XE+)c4%W$RG01K~vwCRy>Y$Dc-*`@PY@d@S$BJ03t)P+kHBp+i zeU6d8xb*FZ-=bxe5%p9rl@(EDr8n*eZzTCGaX3MGBiTAqeWWYl z*qkaG*w`JB=vdLHu}m4-*K~daB0@ zWWE4CbhxRlLE7J>}9F^&GZju?2dbKECv%i(kGvhh_*{q005HWOFj`(iEQtuuoizoPe6xT#L<4?!$%*kRZu z_142-q1QZ9D7??Z_~*qpFKk=YQX%Be%dhq&ZNZ)cI%4|9lEu^hl_opkJZnXC&9f54 z&2Rx;3N7Y1f{OqP@P)-+-(pi6p}Iwj#b6WhM&cl+(_Mx**01S@DB!AV3?_ZDOXf(c z2mj=QBw=X4cp!lqjQ4B9PbT3PvA)F52?>cAgO^aju6mP1MF$PrfZaqLmes7#;ZHfjNx3yBW4am#?l+q}`e`elVdHwe%2lEv`nO2TF zfK%bykG!Ex9CGh~!EAA%L-0d&oG5|5_u`2gM}H0ek0|+sesgu#HC#BOI4vnDsDH7< z2cBs*?uSvFl~vM#hcn9X_3tQQCIQXWxO!58f3}TP<3i#BqBa6CIFh*h?mp=C+6za4 zNN97`z|AM?9?|cLd|of#n0|rhZA8mf@v_>$y?GDN8Gcm`lNF*wb~~RRh!w-?!swpW^PK~a0jL)t|i<>Fz*aycC37OnExGz zhUnpEAF(ppn`|4At?ph<*$Sw3et+DzrMRcC5ms8iJ68uf7Qvw8shr0dE`Exz;v&xg zk@_P-ssq0F`8y8MCwfni-ZFd|_=(f62ig$SK{88*cAw7&lKmYVN{t-gA8CKcw-I-~ z{`zO0FjT%!L{wG)eE5c4-<66%%bvUyZL4dbd)rZx4kKhuIZg>VU%SgtwprEW>nd}*ce#|EZVu681Z!bj?_IcDqKfe3` zQm62iSP=D3_dU6{b^{3Q6AQtSl5t6U1EN%-)-rfyQUOlL7LiZJcE*lYR`ph%AE>*1 z(G$Z&##I<|OW47_659!OOikfRt8$6#(iHq8d$O@34UnUL11>gOEXpH)-&}Cobq1L{RIsgkAG*t@UqMhS+Jof z_&EJd_JpM_%gQqW&?Vrs&hg+)SK59KNJjml8b_Cq^tY} zq0jFyVW2-|yja34D(bybG;FpMiO)m@@c;a}_>}5bb-7H#zsML8rO~{NPpGPr8Ow<0 zd3#huQG>$ftWwDWa>4U?NjHk(=Z0zR8ixS32Lev_NN@Wc6uNhta`6KWcVYp^U`DF# z3h4QfS)JJQH`t4nfb9k4v{|aR_ zss0RL^=>w+6){&e0+B}K3Kq5l=^)X2tBcg<;e*Oq0VnFLaqy)Rp_Q+O$C3P=K<ypb~?5t#v zEsCD0kkg6abye5&e+w6-ideLQPg8nCBrsbgT+&4_QJhcIbg(gbbIOZyV##7kv)DQx z;CjOj&ic$_dC8N`*A+Gw^=nl-ckev>GS?}mxy2=T0ia9X8OysVyUm|vF05Br#!-dN8_4!GI|5b#YlJ=JkcBIdOp!yshyTdOrA3BA?Dh@^JD!)WWljI zGS*~xu;}tnYjlH750x!kn^_If%W)7?V2xR4#onm6|9;&;>g5Kd`QH6+-hK-o-ims0mIC4v4qLY z5sTEhG8;Xfd+*EeXxhpXi9QjdBz;>nhA)MIV}n*i+o!<%0mi=iDPDaXoEAucJyimq zHnO_af|HsYd21cMWN45{liFoNO&@x#bmv`iA`Y6o`*_#;r)6_*No_iA4-H|ZnR%Gs zW*k~+C-qbhiRF3)twD6dz1tA*B$KvbInWrVA2AgvwY`j7InDT--40rjK+f@I>|y+B z?!G3>kYzaL`X5X+4ccM952_H;bxWRE%r znB%{n&<|zB?~%3}ne+r8;7+h?x-NZlN{kYukPZ2?bT}X-&ByXdJ922xd>ikfhrNQo zft!k_FjUNrOyrR}#lm7}PyYH?+fTd!<={(t87G&BDU=u%zhRV`6XY` z{x;kuv<&BB>ynUayIejoVu-gQzs#PmgL2wqSsY!ZTGXcEf?K~lLJ3B)kHcX%hM{(f zIkuEK7zIBR?DW!zgCz6<=DbB-sh!gH1C?+Gicy3q_Km({`#4=K+c=i>QYz7J6@Ub` z*-6WM2b^`}9JceuLxT2>MdNJ@s^Nz!RC8>sH!*DyvVYL_81^h4SP^aP|FuJPaWta% z-i$eli7>9QkS-OZt7w|TM8fS2b?vdTXr~!@>V2v> z;_-T2W9B*$MyWv07m9ViD-?$G6}J72$3`leP&{YBb;%a@tj1>ye>fr~kZQNap4>WQ z*Lkdnovg0zY zH^CduNw5lMHv9XTxeF<}KML|o!sad|*HsFm>Db$qy+oTlO_6$-W7u!iq3>%L(0jBD zXqWo0;br7-i9Hj!;0sXt3snz6*4FBD9+0L@MoY<<$t!I`QPR{A&V&0Xs413KVz_Pf zbLockGD8K`VyJdC)#uaToAA-%{n3|wKUK}!hOk!5u-Nv%09)#Pa(6aky9z+uAdGin zdrY)@&^A4~2gi&zMZ8ltyU{|7p1qXNsEJ=T#gX{;+>m0(&-*6WzZb(VrP&7Ew+#4# zWvUQfpmaN5C_5{UK5QBR09^Z+xG<;a7!SM=r@{Qy%dDIdh<2aSksV8QKexs0|oIDc| z_d`E3lfylQ-y4jT7EZpe`dYT{RVU6@!$0A+xfU+loGxyTmg{V%_ciMXawEp-V_BJ* z47~sql8LrV*5ZQG0r;X;JFv)je~mupc#2T=lEo7CW-VE|a5iF7@m$7=W!HxcM#1zY zGVYjRb$xIO+vA39q`@O9J|G%z2O)yvq15v$R$XLHQo;x3$@6X+6T2u#ggp1(#bZ|T zKhc5_!@=@zV9Exxn*7fj=o_+?kH5d?Sr6X4kMkNH6;$l(y5|v9zUnmX>$k9o>`Xq! z^0SYb34?v5^v0pL-%kaTnrG@qvsvQNn9ZcxxSbRCi$n~9$b}L~GGZ|4TCg3YycsMY z|9|yHLm9$d8 z`<6E?@b;KFD~M08|{&7?%#X37@CZ;#Uz>qW)w&`{rw}dk25_?oyh%@NZu5tt@_pz0*O0` z3O~GpkP0B`ibS@^3*`hNi8G+(;r0P3TK?5|P=3W5y-nPq>tZ}5#UX-T-c~7n|2m7! ziHD~DKB0KMGx`tbFUQsEsDC`(DQAUY&0tQhSO$3Fwndrl?Z0RLOgl?`rN8}yw&q+H zrv5`BesK1ztnk4c%m2x5{OjZxVvEe6_(czQyyD`oU=yeq>9Vl)OkJYEwUi;b{B>2x z9y#{+_r(mm65lKB8^lG9Uk8$z0T|2ohBQYe23E#`k5Sj*`ltDs&Z(M2W}y%$Oppf_ zU|A7Q4xKnDH7?dw@ z;rT4Vz_&xuf`Y-pBIKP^P8fLN_&0-Aq>gWWN$6AtiD_8sv|L9${PjEWk0=PoF(ds? zd~RC5aO73KggP7u6(pQ)PdQ2dwbzG|MxP0DGN{&7nP|2gBr$p@3NMV}V8j`)r^ed) zyI!><5`^cd`Mc5E{~!p@zSs+Esi2LTqJk@=8L-&iTV);gyR^@ z-c0vQ8Gs@OO}+(=(;H3FAHW8UNn*GOkT@#Gf#EKiBna3-J zQa8PIEN49x^&?h(ZAPp3I_x%R9_x2VJf)HTkcDw2f$Pm|*56PSw{39=d+OEMfcV&$ zt;;XcjzQgyPVMv&|5>{MIHaEn&B)^lde!HBA4j)d6TQCq-C#e=@McVeE{_GWR{8$g?#{ypzVx;y2mCZtYW$md8+^wH==Ic3)`BJbqq0 z_{Rk$*hcQ4MN@EH>^zqT9=YRfCpWAfKu*#nqiXFB2)MvM#*} z%2buSKFj0`*u_+@r0TfZLJW*5sdWAQwwyV`4`uG#qKkA^j78g2!(u99r zwxx(;d~7*^_$2;IuH*1EuCOA=^_qDMP~qt%>C|=pcg#$^Zxrx9-};~LX+iq_g{Rv* zwEq(&_$7R-m7sr_)c60d0Cn5rc87MR|Hal>KtXd1xXQ*4k>8~=@yWX7Uh5D-rsxgd;j;YvzF(qrNW8ldG>zx{_g!DLJ=k) z_tz3`&=0X(w$7ajdm-BrN$~L?{SNXxIJMgMeZVTqt6MCa35l*KXPbZh0vWL6QPcVA z)?5)0e2?n&KPUBHe-;U&HXuA3Bd+&Bj3tdO=6Gf!wbT|Kg({os(B%kM$X~*tA&U#T6 z`(x@ZAH}aX5A=q0!)*@}PWIZM#eQV6Su{T&0*`%63E3V0b;E0WaWeM|ldJbx(Ixj- zEz&6d=L@;nj$ns>9SI}ES*$5^jlaG#b3+60Uff?w*S?WsiayP&fqZ%ocGSs=QWGsO zB}}QK?|4s9_J0c*@|mA8t=$)$c79>J%WO;-%E|uM;~*ak9!m_Dw0#>YB9YFVjy@ z_cTg0+RZ-j1x1h^2d9N=+&80yrHLO1W}~knbA2=abUTo)$B8>L4A}4#-zWNvzKrPI zVEOm0I)D>g{&};T%d@TPRp&{wvBWNZL;|DNuUneBX!^f=`+&dg;Z}66B_)XZ*P+CM zO1^hNjJSI_lRV1IB+dWVU;Oh7kTGxGj`@JyULX|rt-pQmH&!!bC8zWijqn%Rxz%xT zme>ej%NpT@`Us-{C*5}$M?-un;SOOM>sjbT1a#FcpX75)zw>K|^D zx}sGc(}L{e^VHqTg!kLOI(U}r%w_xPBzoMAapieepc~f$H5&@q7nL*>u?Y{WX^EL0 zM#(YV_#^C|vy?u6Ij2HV$Zux!HKRDA^l`<>p6857mH4fZ!QZdCq(CxxwaT0i5XT1_4Xz-Yab zQ7$vKFU7|})$&QI`URAs#GAtfD4wTw#-H%-#MyRcUUg}bp**-OY>DK8$2Fwp{m0jx zPr{-Na_oeBFHLD;V*GiN5A+8R;ClUv0zdIm#2^sL^+y~tW|@(om=@3Tru6B)Gzr9| z5L}~FTja>|@HxdNj+iiqSqc}pz!awv&r4!KC&cMw5zp(Pmv@6N_nvYTKQ7wndau`{ zagYTL=FI)*NmZTt&FHr|yZ664?+rOQ@^5PIZ~1yX zIf*%rpisHYqgrYzCRSSqV*eau3r&C!p@diati~nxva!q_cbY81+CaZaSJ1S`o(VB3 zU|+KlIboMr%Dq}`9nKMWUc}JfI{xgP*LU6gkFL`R0RTMn;jY*hXn_Xp+wv#M0cYWY zIgR@OYb*EuH1Va`p4@|v0I`dG)4ocIb6do9vsrkVC+9o?4??vuP){z{UmuqMY@bd|IEss#r5B`LbCo;M% ztz&NQp2oY>nNBD+8qPj^{l8v6FpYuL6;arTLJ8Ga@T^E%Ut0a+^Z7qq^($END(Dmc263-s~m}g2komR+(*cj-t%RS;1V6M#|}o64IOJk)S7o z_ZdT#0oTC$s>P)9CW_O`-Id&}n17$JuYWN4`d z8rx`@6Y$043rBr+(WKA$mwOoi$P4&)pOCCLWJ6Ln`*(A_M#77VOW8_N-?fo&AqCso zj?2e4ZW|^R9p@!{O~S*DZ$#ZoWAla;zr0xaY4J1rYf>SCC0${rdF3?Wx#XYI#OLJD zm>o(`=#0rN`^$e>52Q`~=dre`XqBP-$1Pd}G=1ND>Z)qkeBmDXSbp6;Wb=h2JTJ$4^qGe{p!Ijyp3fS)&)RCw1(Xs64F6 z#bERnPuJZ*y%43*X}umD0M*qJ;phWEm(@T8%X#amfSID1QeTcp!bWp|h;+W)={cGT zwA@uQ+aO_=SonN~uWriTzbzQd`jaTxc4^(No~mBm|D#pHq3Ztr*$C?fS37lsYkY-U z4Ved~skkjUe8H`Ikg<$)FP$;}B%J*l&^yWx=t)|2Szr?4{Jc+A@`YL4{ddd9imEhS zmGqAlH$Batk#ATrpW8XR=sNwG0(dnazjpAFBKatvb^1TrB5bu|R9>e>gnvjlUYU@9y zUaU#=rAY{niCcA{yB2?+$HCjCrI8*ceGh)2BPgZ3^h$Z4O#}UXhsxy`T zvaySSmI4p`I}`O%7f?i;98m5BZEPlQ+L@LCCKCIxcbSAaFj80)v=~TrTo$BCuji&B z!jT}jeKBfH=?&tz2%{P-v*wN&=y#%-tZC*HQH-;B&Jn3N*x#{ z1_28jK(w!@Z3@r$+leeF75eXeEm z6X>$Py%t88aiZ)@e-e!8TfGy`E@{1=x`|g$H;A((MtvgQj5116j00hq9;wEo;_QFy z6DW0%1<%kT*Q#pRk|z-BOQ~rwJJoRapWfM4B9Bd3A!`>`;Gbl7if?IKKKkK1D6tW& zO%CtIkC9oD6^x@EGfUqCXk=O!dJ*hTBsr3`cT8p~eVn_rK2J0R2Bmm*KiWIQaHJHL ztgkd7hK{r3Mb)6@I)L0}$@`Hj_dNhv>`>KcYaqC$%;DU*n0o){2LS&YBapXk-K zXH%v5UM!60shG|4ec@0rU(Ml@}rqFLJ3^thm`u+772TtM^HqhbR6- z;tuC-foBS8Ct!Z{2us3+e3z{$35~eu;Ru#K2_Xc-n*NZ0ZY5+mrTlAtD&_s)H^t*8zFy%! zG{cr0%eu9-{x)ZMk(2u$uF5~0(Pjl;^=dv;hK>!%Au)tv?8h9x0Jx!Cj8dzTUvv%_ zP9VpHE|<2#-J6*r4|b3Tt9|mr7}yU8(!It`h(pUl`y?~)@T&C-PZ50r@#!=K$uNLv z*b8m@HyrBxAjjZDcZBy+Ss{)0q|r4y>oC<3Jni(M?Pn_3hq*DEh~j0NCc#c-)hsm6kz$ za)4B78t6y=bplyCQ1TK9=)9Z5lcsDklez~!55Th^+0noMNhqX25P{z>a#8p>VuOhEu!LXy4--f@{VI z%C$;sWUhX>1KiQVmmmk<1h1UUL&t}07oIY5rul>?y+SVwiHp@sh;(z%j{rm5te2yJ zi7EMK^Q$^zN2Cu~Amemmt+o!wIsJjhf|Fzmbi?_pX21sNE{`LLN-RCt^s)!NH-nW! zP>d9^=z#V}?OVtLl=nKCaYmkh^#VXS^`=}kGa-v?#6gS{3wDpF#c%zNI)4BVWdj$RgRziHLv-pC!X!st*)!V{uo%a^& zl-{Ls65L=j5BVQ3P4TW_MT2m#wkb-!cAbOwi@}}GN@7#TLf4&oP=kYCVZZINPOOgG zm8QG&KNSt0xwJ0UKX;S!ZKbcr`m-P zu&t%F?kky~Gpj+hlXJ3S=Q69(I))N}H@I4TC$@!zy#b5Rlb|6COWrZnPzSF)F93XH zOaGJd4)7Dirg})UDYHBFbP0kImop=i@tZYbYkpsh3;10<%pn=HB$pkim ze;Uuw4*lxgd1MrUp2de@{1^skOrc>&)`h`H*D!5fyUfEwYr>^=Kf&Ctn*Ib|CPf17 zm_PtGIP|1)<+8bDTav+N$xYN(6ZQIt5^-8)UBiY&@2jo!2$xwc-I3|eAR63cWX<%U zaX0V9*|4d?&6;Z0OMs5VHG?2&B=VUx<~HV2Be?Mzz*hB)wSZc|c2d~Cs?Lf8%+G)!HN_acmbSu39VuFj&@w<}L+5cxIjsz1ap8PwKt!z`_*tLY zW6j#Lmk1>WEjVeU18_sxnRcIlK%-nkLS8*bWjv<{Ka^`+-{z9l9ly`rVf;ZyMdGR+ z>i;-II8tcH0FiEeH6{NANzf&sxHdz;&KlmH&xA=M!F6cy39_P>a^BtXq4{OL_&`~Rcep+ zgQ#!$i;xIHiXlg0uUz=;x^lExHc14U6uTeRw~y2gp*(-e>4>^_vA^UtuhG=WDJqor zU9?^yw@+eS6F*Et_fA!hTswbXl6FZJkG0}u5@pHjCs@HRddmyWaGH1)9;KQ5s#GhD zV&{>}Cl6+V-Q>d=uGJHF4D-oc`oghyxvR^YOm_fmDU<>~BuaDe(}$Y&LY+v$H6=%Y z2W;4SGNTmm!)*7P)y&1QISeblzz63P&^)WlK5EQeAWo`O)^t5;JN|UrW&CQc;gi88 z2owjJ8^>9kEg{-`MQ#G_OSJ`kFVGY`g-G_Gf1ZQ&QR%*mg}YGY-u)GK`un+v1nD)Y zowLA5NM5UaPM0ih)W~BNgfeiwuTI3qu#Am(r_D5i3S$)(;Eo?naqlPRZtvDs^akvi zR>IaV>s`beWS#k(*xsNO#mZ=eLVDX@{+87$FZj{^qr9&3oDo3vPv}>aHcA`@5GhDs z8J_jxEuTb+wgdK%_gA4V^`R)@HKP+&%#QpM#{^eR5g{6qWOmJLj{2@4HDJr|ra|rw zfDA4IX2^iuAcsi1XJ|ci6H2b+oD-J};;QzI?h?HdFjBzr_b--%#hh=8c_G1(QSNz> z9*=|5JFZsRJj?+OX$qy+4&AYDRC_jiYpfH5%l`?etHs`F5uxSQ^Ch`4AVI9F2lJ-6 z$H13Qra*OXMkAmlKd9mbIaqZKcN`}U|EGw4*M=fm=`=WDDrtKP;I!%-Zz9$Xn%YZH zlFx)_edetREP6d{> zo1$*ad;xa1exX9X;-CHSVJ?{`4e717X|;PHJUFD?4C3gdc`rDHqmhnibp;OTyDh`^ z4RUy%r~k^&yj}CML(lnR6tRuL5bts}49zLI!T4<&PT){XQ57T?*W;HV3YP{=!Z zbx%>Wo|YXh8KN-Ew+|oQaw7LK;hn7gxJ6o-C0EuHdvwwYdodx4iH*f(h`(gk&ENN} zdA%Xd5|21$-sz96#U}OFlT#%b&w(~)@kn_wS*uR_ut&MXII?@~BymkmhffYZ|L+Cp z6w6g5IPpW*yVx|(At^iVDU$F4ZUS@O*xJw4+?E7kA|+7>tBm)>eMw{=&9GiYNAgkW zq9*+k+t#l93E(ZY`ujHNEwE6N#!JtgOM$2{NS=zU&3k91E`d+}atfzmu~g$tfJCmQ z5 z8Ek?95Y88W@Kb|~SGxJ47Crs^Ia9QK9lubv+1OFMFZP*=Z=7diw;%r3u%8cxF^voE z1u|P->EfNAaBfv9Vv_27xJ%J4dzeNeSu9T@Jqjc-Um_X7YI`>Q`8m~tN_DfZvFIzG zmnW{V6qZlsc`B-e_Kvo4LmHTDYo+BnL(_C=2>2$P%XA2KAkY$`#Mw35*k=C>L5axk zx^l^t4k!k0=U1mFn`oFOnFR`y&e1=*NyF=@rLw+b>HApq8GHuioAaGneN_z+4o}Hh zE~DyFEGOTXUZdh?Pm~jr=y?aUQj(24l`p(n(s^UI%FEee@y_b)Y}in0SF*0*%2pj=?{| zqSc6GR_i-IjC{Z$31ArI*3gh3dB8|{qvl~}6|1k$I4(Y_pW!ybrX1@IkpH0_v6nVP zxP4L$?+YQUKoqh&DZusm2lbq+aJ#?II*f3eG?#uw?b_;1>c8T~D?&(C$f6~t4f_tP1_A+LQ)0HXYQyc%{S(=R#=v5U{qj@?l>7z zh-pR6%L3k(l1KQ-tNjd3?~$8J($qAeemIR0!*?a3C+iW33K~peZ2u`t&e+~r>dGL) zTz)POCc(=}iyQnUp&k=_)!@)0;9Qp;X|YuBsni05*Xvl)A$iwwP*Qe)lgG$~XaqU} zfoMVb%;7j`Em%>z!NwQVzV(^`#Rc?{yNE>zbwg>B>O23mf&mULUIiHOiQp1HZmF?8 z`9*RTIEL#;EYMcf<-nLdk1#7*pN7>6asc(-i^O?vb7ZOeoDJPMU|galQOl#Hrq&fpF_C2s7tV5|AVcVo(Gw7?(z-GOo8caN zebOA=o?>FEc00LQtPTs%5<-*_&ExX*P1ruL2yx_)!JR$neXv{CfLThh?x3O_)aT*e)$?eN9l<1UhK#H&n zryur0`guCq1dq;O>E=(vv8f8$8aI3IEvV)Wr}Y5 zmc!6j$Hv&S!}tp}X?o4))CqbEWQ%CfxV5YjpW@r`RhSeO#Ra&5+|r#gbtgq!IVT4} zIV`7dF}g#^CTn1KY^JJ@yOk2<-7bbct#$crgQDf+s6)vY%O4Rs{uo>=X`d?M=+Qn6 zHfN4rt9;(As;c(1Mf2SmU#@zKPB?qYBC4imWR$0C8GHMD1Z+FW>TCOsusPmJVwGIT z@dE8w)#Dc`O1NOfY8y{uty>-hN~mUJkM0vy<}5=eSRe>I=m83j*(@SQ$Kxr3WJMYT zJYDSXRfVD`KT=>rWl$F!Ws-|yW?VV7c2cdyYjmDwd&ATm$sM^xV)J`|0Y-swiF1hDC>t12|4qoYz8S;crb+22aD2Gi)u#{mBE*~k2opLHUeerl*h~hcVn&*rd zRVSLkA^sCP5CwNu3)9s*aboCDkZsHm0Vmu2qP0&*gWe@^jgGDVwM75@90{0bV7Z>G zRVz18b}iQTU$uP^EH|~WOvHs6AH%GDN)+B~(`>zM12r|IY^L2<~mLHMinbaqbAEz#aU)nssl5N!xSCnAOH!AM37MQPUl^&b+ z@Gjbw$yxCQ+fc{@EfZsRN3ZtdUMSuyX#;fh+(b&NxZB2FOMhE=+lbAzq=Cd=Ee=iV zsL;WeP-kD`nX=;~1xpZX*2?S9RL*8IkW)B?ZteK&1RoRCqWweWc*b{Tdzi&HXlpI5 zGa4MC_F0g|0V*jrxslz-)1Rw)y?v9y^38=EpzU|hJ6sfEYksQT?w@0gK6=9GcD>mv zoFzX|ZLo|Yoq2AuR_VTua?24P?5%3=B;QE1;Y`yj>oJfvz}x<1t-k60%j+hKic}KZ z@IEljc#^8u#_8q^pTEHV-yQtFuJL2KGHSU*5nnZH5S~or$z=P3U;9f~Y3g|^Su~lK zp!+3MsoEttQFKHpjXH34)L;~r&$l|zw6X&}kUM-CKyPmE!&ucCe5sP@7aNhZ3v_t`WD2PO6e{VVu<8KY3G1 zosAagr>T-n%FW%&Bw9UgD&~<>t-ZQGTYx0H$6*`m1|4L=O(62gJ$|Ieju&UdV`y*! zpzdxVZz2Sf-Ypy}|AN*@J17NXRJEggJ0-K7rPP@B6g*8vyfKMg>E?I;E??T3Kt>y* z+uv{!_Azv*ByGuGerYCzY^Y$BKB9Xxr-GUki(XuOOES**qJoxw*lz>1jxK<{*52*v zB8tw35<13Byl8ZK!0F$s*#s|06{+3dUw&_e$eAF{C|#qX7P7&+NzI29ZNCgGtq?%; zGS8GDzVuh9?`%BfalV^0Q_l0@*E1X@a#Su57UJ?h$3l;Yyhe$153}NujZ{{7MWSd1KtE5cEgzUm0R# zmiV7X-BV<0yVn+a${|SAr|eFENrxUQa_kbYw6tQtX~Y=;C9h8EerMM5RecCg?T+Qn z6*cn-6cfH@9n0u|{}yoous>}(<83lTIxOQ$EQedZTD+L7zFmvJ1%kzNsn8L(j&^?_ z#ihV<#lhWr^cy!%sDceu2GSN3k+z&7PEBM%7MkcoB(Ehu$<$MbkV3`(<|%-$l9YN) zZS?Rk^te2fiMTE^5Tu(Gt%|tK-6RqcW^*}(Ckj8VH%2F@>>Cz@_hD0T1wPs1zc^`i zoB!nm>+jrtQ!k+ki#|C-#p(eZvxNEa{h?Rf*(6dO8N5I1Waiwzh*{?QE;Q%5jwv&!H zOccIi^uB6_w3|Jphn%QRlmV(|wDo}qBBA~-rQeOa8OAc!!^9{Mb!xZ+?w{r#JT%>; z>jItIGFc6Mq}~@L>nO!Dt+kHlqz$0#!04Q}nNsv>wA9eS4THQ{!cmO%KEIA&4p1IG zs9;rto!&NfcV4*%uwfKS z_z3;vO7ebED5lJo$#>?&7vDBVQPq0S&q^Ybob0h`h4*36eJffYhhJ;>kruCpCSRji zUe;%(-vLopQ*8lW?xJb3Q`Qm zUL8PYATvyLbS@9l8&`5TqugJYyi5YOj?to$L}3v`N!g(sin^Rv5~lb(0a2>jqa#`e zO<1Ok3%aW#-%j|`e7YHP*Q_IdElitdi=(D)@hl?>TgCNw@dT`q_dSe;7k3274%m>M^o7`xP z^)wjfKonk%Sfk2fNBHaVz_hyR6B2_bd)XtXat$-s7~#$HrzKI+=LKm_L9Zo(p*;H0 zWH=W@6;~p*)25R z?O(~&eLyAC`G!Z<9il7_J-*nVn1X3U!}Abfb~d3Kn#{eex|ulNO38vX7Z26Zsjl#q!(RGJSP9ryUe&HA zsB6CZl_oxFl&uv3Gh9*DabrDQgh2Y*s;`pu&I_E z=$Et6u5_e!6-QRzF2kK#G7u-1kokdP#j*>LVE?o;@I&cSJ!C11uBA_#=mUZ6a`>{D=Zq+{29^EThT z6bGx~+uJ1Dz4NXu_OZ#Dv=14ClLRQne6s1>gj&Q%nZuz`yyenU6kp9)o=#djXWn0x z>|~q+>(j8qsfQT`djiGoIC)m`QXdBj3p|WJBqjz&+cA||Wu2(#aSDvm2D~3DT+!xh z(hL|sZr#nhiOly@R5}I-uk4>jrl#%S4jSe_{{AXig-5FBG~B%=I;yMG)7^@V z|6lgtFS`I^uG1zq`naB_0Uj2&du9V82l~z4l z?x~8E!Wco2GB44r{CaAJkM{IYJ}NA&15N&M4FTygr?35cKEKauuB2>5%u&y!;BVhA zJ!u1y+uU4{X%f}eJWc_s?Q8N1d>so4|Iyl5uo`QnCP4khJ9 zI>;vkgPn#O*f|AkhEM`;_xZg;q_!Z0`@M1ouOqR#^lSa~J88kekwW@*I1%{55f%1^ ziA9M{8H6Ys?+`8gR&owyh)1xG{ddAXdjD0(ordz=jad?-ANCDEP4Vvr)#x)I*4 z-kh%1+!9Ui2+lD1Vg1ceWD!{t)Mo7^#}SUQ#^I5!@V2va8T0<7l-ozUBNnGN_(k5Coc5ODbGAKYD!nvyjyZ3@}<<&ps?eY>Oe_&D#Ma)%%UInL=#eyeBoapA1 z{V3P&te-47sJd*PbtgF_n)^P)DvDM8<|DbTo#;=N#hy-PNwUsEtd5}G8=NFk#m?KU zH-2&WK5A%Mg)q~%&CatxXd($i%|o`*;M#93ElZUs85`x)Dq8gHF$`sjNZ_RhJ7YNhH+f7F@v92W~HY#TOo$fQbt@{`MvW7eU(s>>;~mJeBnO498I04 zc-=zbg2t5!T?wDzXc7A%0{L*1L;P-T5_PoG;&X(MZh+4~C86_E%5jfZf_Hc-*aXJx zZ$5h7#S5pK)H_)3A;PF0I|X;+B?>P6eKYpg^?;%k^z~~Ea4{k5gUX6t-byT!4!|R`JB_;iPky2b?J65B{I2;;m zrt{*vnT~G&s2SeH!H$c{R=CqkBTHkH$2e4|(LahAiXUc;qs+GFNaj|#0y7#1d+|U^ z&-WQo-%frea*N<6J%ML`$yJ~th(N3fVopqD2)jM~ke-c`f!2`!KLT6xUC2Bnh(Y!M}*aVq0)AJo*=&5^M&Ny)F{Y*9Aj3>z}Wh@!VDmzm`&H|%xE zo7M$b*s+x}^mP>%TYG2G(m>xq{|KE&+;w9z;FV&mw>S`YrHqnn9-`V3M8KNxx?jyk zP)U*MCF6J6kViT@iE#P%Dr_;)>2z|yv#LmXTO8=VXa`e@k9khLv1xxFs>zy|AP6RC zE*EJ#jY=VxzkYAmxKJExowJ@^(fwtji_kc@zYgfx4aeGCCsU2oYvxwnAzbs&eMkaM zl9T!*T@ig-umB^dP%y;w^80i0Wr(ShydPVn%h=nwDF>pjZ5!;>wC>?)sufxTYl&7W zvzw$H$N6nQS0B{x;VT^T*T(bjXZ%-P+{vz1u;=LdXZ#vxrIzZ(_=$7d?}u_MKcAk5 z%*=eLKmUQ<$;MnO!WF`;=IYxh-hh^7sunj@MeX|_#!+4J$GT9|Pjq_cr$Jk24A-En zov0|EU^N<>9#1pQprj#B4g{aWABQT0x6kJ(50Snf$@&7koCGJ2yhso)?Jm^s{+dO9 zKffajF=kpw_whoTV{$1`K!utgntJCHwMqBbEL7YocUq31_yBRpc3^`f+#ebtGViY8CYakx^xm})>%*QV5o^TIByeY&sBc2E;ylBrOE&|~eU9eH^B z+r$|%%`}SNF(=$fB0BiL9RI-*+R3(($>cO_24yjuQT4%fTKqW5;izil$z& zYUam5R`Xwe5XASQe7@vhc(st5P&Z(K=Q?Zg%Cidv)AwWxDcjJQ$n5>mPhs(8ohTmd zLB^|A-D>Uq?x{-np1Gt2u7E0pVu`#&^ab9#=u~|-q8#UFIEHz4>3>Vn015Re)?NNr zUT%LcvtJ5t`kNRsonu@U;BqTQo^kMxq$W3yohy4nfLM(D(8rE7t?@1S-rT@M69r@O zk4>g&nO|>nnMd41N`=lR>F(oW9otGHOQ?Ii6fiT1ww=@_n9dj9KQ=eZ{j%sfV-LId z??m=tF>%&v_K#wk=iaS6{XB_BTcKu#v2kAB3rm$P3?w$=agt3XcK!<-2I>grMr)tw z15=6mW%07LzYoU0uE*&^Y`GN{7SLL7fAKw~yQ#6yP@!Kwpj+@vbQm%uo+W?>bA&0l zD76Sm$lTk6bWIs?@6>1t(LU~SA3l4IoD!k;JmI@o;vJS4(-VaQ(i<(4G&bH6Y2F?C zKPd{FNW3H(HIn~6IsdvgX9lz=^`yr|qYcL$#EY@%)x%~X3qH!KQyc{fc-zFRNmP$T z{7qSxx~VMJn03bExi=Uw7`MA4A;BbP-Nv$b0sa(*C<{p`>V&2$!ZcpzA-xB}{yl^h zU0jmRA5b5HN%v2bN1JW+S-G-Ks2zq4M}K5NFex$zIPUqEa!b#jip+hney%`t(zHI$ zlxz6z5+Ndj=q1|TPfShk7IGB{$`s!!6a6zFT3~@9=94Zvgv3+@a#}tbzwVW*{U-4+XRdXF`rCc5N9n-W+{thM((=o8(;37 zBv>UI<94c@#buRXe~Ki+R6maugeweJYui|hNg@h)SwZec@3^dLIM$<_efp?^Q^@NF&+62#{;fSlUiqwJaW9XKHIpiI z)%aa4EPEsh4LoW{f-h$#E--(R8CGevRC>hpbn5vp=|(^A!s@f=s$~zBTF>+-TAC-d z_T{uX(bWoE(*E@(I%%M^`M0Lb1TwN%2I`=xFiQX6CJ_JEGe^20RRNPuf~@qEU1Cft z-WulmD$V$4{2tYa{x`u#dcPuIakUQ|wFDtJ4^VxJLZ|hUn_oG+|2Xi$3RHNpC%OOe zyDUo%ZBql=y669kj_lCAI?X+1a@5cV;n@AW#2H9+X{;2-(hGy?a z!LpXp^LzY`p<~5>XXGwk6@`6c9WAbftPU0CqunzEci)FE_br#slHWLg4jAH>9#Nlr z`gz8NQ|B#*^=w)suQT*L3NKUv4+MqDHxosgq?3X5OJqt_;lxT$^i4ZEjulw!)J7@7 zI_0c`Y|df{{n|n*{@{2GF0LD0Pi44vP{JmCP$o+oC;f5xXSbI{nwDvrWW>FCS0If6 zLnyrgL*5M_b3?Zlh`OP0A;d=ItEc%ZSJmsBydw{3w|9PE(u!E)hiQ(5(xRTAj^(+Q z-gP=|CamUO#{k62g#S`PPDppfIvQE34=EclxOk*ms-Jre&YV}OIRE@awXF5872>Zm z6HJXoVj-RJHI#j~MZ)!J+yD9O?UUMNH_@g*+#=X%{os$Y=~_N~E7ZFpm0psvm_)P?VZUj5 znLs`q^~ate^JB<5n)tL`6yc3ZQY5X2%N~x3+3Fz?L%mYc+l=vzxPLAQmMd>EbPKo1rvusvkVM-oE*>#BXvfyY)9eoxz$l-C(X$X2g#Le;5 z?7826hGo7+11`>?-7k)0F81S$k1M%X8&_S+3GS8Kmyo|L383> ziN@kb4VW>?flhZwUpUIR7Oi}v3$t;M&jPc+)@EnmxUTZihn1-ePF=s0RKK_ils&Q1 ze!_->3pd%{XH%I5%hJsrzoQ4PxyT0eLg*lJT@$5EZ<6pM2;gq;HVc;jf7xHa`P*mJt}NfzV1VZjA&{{_=6-XAaEbP*qN2~@ zpBb2|e~`Kxgzm;Ti9Mb0CU7Ixz>bT01yYXZk=aapFpH^Jb{Tzj#4L$Vp+D7>0G~E_ zlb#=G_my}-zby8~Q?mK6I$eb9HE#+rm+r}iLn6_w%?Isn>++6)qdd1e-_a2Ko82KTBNI* zHg)ZQH+nqfU<36#g2BmYa3OLXzM1p2xHr~@5kY;Dcf6!stx12)j(?tWc~RBev$NdT zW4nM_nz+P(E#@CJH`UK=1MfB&?nfP^e*!8N&7iZ?D|>+;(SxO@Ek~ERnk~n6%pQ3_ z4GA4b3yO{bRcOvOw2NlX z(F8=sI-q@g*?`f+z(ZR=WJ-t>=C&R-GANG0q-X-uuTDZefUi zkR_W%;j)5QbetRC;#!P?|Eb)04;MTC#tDO-_olkrT{NaUEr81M4lt_@9%cawcjz(Uh}G5BXwkd z2(DdDke)G6jdTQPtby_&vrVBgu`!eM>O_hCZ=7 zzRk0^qVHWun!Bf0CO)gc>zNGb7d;0hR!*CH&)-`j$|KC(ZvsbG0Zr>~;Ddhn0RTUG zNb@F?W+xVFfsi~1+@OTO3xpx-cz#Vk4JFO7YFYLP69dX_+zphEE`R|*^&q04oTPoG zgm;@SGMMbd_N85E)PH&=ZI6+jN!W8A{_bdtE!tm^uB;4f ztIZ=Vud+Xlrd3yAN2&H-+JVu-61C(4p*XY zA2W+KVrn6!!DuRu9{-RO(8>XMJuEg=&wW5LPU~>G$?}8)=u83#K7h)l65~Knxau4b zJcn#-yWWKie)TZXlh?JTPWa@(T0%Hr8fRZ(9J&XgkHiQEN^n3qA&H;y=?C}-!cs{Z z;QLw#Ek}PHg6SEF)f-KPq&Lfl4iDPb(8Zv^tA8$AW(f|dgN?|iD26y}u2aQu$cx39 z$PUZT8=*}9D7ZnYdvnk?oQ4^Ec7Z_GJn(saSz&B26ru3-rOYMiUx!7AP%l6^R}uLk zoJ#Fk8q|f$Y}R`#)ES{xd}r#V5#N zxvSE!v04Vw<>|7_L+aFRm3jbW{;}_{Zl337H49H8V-~AL14LU+P!>0nx&j}Gr05J& zXaVy{j8jfGw(SS*_uHvGKS0c*-<8zD)*q$2u3!wH$hb`YF_66ZtfjaLNG6pNIW%KLrFG~bww2R4HvvB3#B0jq&mzk!LkL(n!V z>pqtz;l66cj5kg6kf7awHyjOCj|)?M^1N;7=8xAGsVlt*mGnGq9pYlV$^8P90z>_7 zr5RXMP_{1pU`PT|;r^W<3!W4`XaB)%Bxg9fy0?mcZyO?J+$d)23ND!C37O ze4iB$0tv>}B@yHMl6qsJ~eT-sy& z{j{tl!&Fnf6EZav-Hl|_Zk-dzZ!$^kA01^_Q6X(ZzKATZL>54{Q_GI$lS@f;>1l_; z)WrPJt#KYYFg2&N9K{*qFH@uS#b{>0ch+7Br%>;};?(~T(bmJnE5(-od%KJJ;6s<!8?1r-}Py; zwSG*6cEN{vPezS}6EJGm-=Z*WP}UzPlNFu#KcE3g%RG&<(ndeo*LwM}kWlKISI)3= zt}c*QJb|4v8n~4rjPfsFdy$RnS+hi^Zwe_w60sbjVd1p6>M*v4Z|au=C)TF1GkkmB z-Sew)jybdi;qeQHhD^Vu?w_L?_rPGYLeQ-0XZrr;18y3P0RHwidSrB;o(3UU%vot; zko`OT!+t^%I{4Uucc3Zpi9S~!{IB|+1T>W_t!&?ImD3cB}Wmo+*AWkB0QimBF?85{eH+;IbFD&MO zNrI9A$?+1D-dV(0_fvjnKhF~gwWjDdV71B2e}Li(-o}#hkfwZ$Tpe>4AS*uYz~^#G4v-b6Y~ad@W$A*>X4}zI`&FDYoX?%YoQC`f9aec&ggmA^rD#L2=A9$OD*ApF_E@i!x^l;$Iz3Qww;>@@kQnoljE2x_l$7ebL^?M2 z`d7eo@woe`4JMHF`~ViV7pe({%+%kcftuXbb<|DlI_Su`WE`Z};-}Tf?;p<}qQ8Yy zQEcjdDHn+PiF;kBl&H^$*Hq2@4tE;On@OGMu%+{NO^&5-6!yD~=z@_=$ zFMZwt^Y&KbD)>ki)qB1-<*Y!4YhGz+9l;ah>pyRcJ>T2NzDGU8*0_6Y%H<-;O zgOOlC{jHm*EOB(=9EeF7UtF3c`PIiA=ZqFE)<;hgttthG)I|)T1q>MUC_BcKAOe{vVZGj&mee0KAYP9%OF@95>cITy zv7y&FMlcz{L-Wf#BH7x-P~lmMw_X)4i9tZ?|IpyMO~GMDp53UW{TWfU^|zR^86?4y zS_KQ<0?8sdcQI)BRSsx&xoHl=30QGzxbd5bVJk~U!Kcm*6zc*kPP)~o(-W37+c2v) znj#KWyhA6{;%%+`zy8zfrwXC_M>%Sk?GKEvZts@10YP(K!ZXQz6|EGgTQz>m@{j9% z@BEr{Z1`p!5LfO1${`$<;{=QOOTQ(&@U-J%!d;)gRZ1}k`2l;1VNnAGo9vR*$uv_q!0(zS1z`j~ z)NGo--o!#ns7!s%AMBTny~0KL>)kxAafF&=$tQ2rI`n+oYau5RZA_`f3Nv(2QKd9g zmp}rGth?>oHm^^rCZEu$z#=W~s6GKgLOyL*uf(og*sO}byQ;nEBD)t?b2vJ2H-Y1p)0^I+8<`l>3FM13RA`4%vwJqCPVyC4^UW0CPT z5#z-2{?_nynd|hgNOLP*i2rcDRMiHmkK0ydgW!ftLzN!!gdna3$7aJ^D-Ai7<)pkn zI$!1;C^ePZwYBo~Kp91+!9;+J*&SRID6JLCaIg8zajA!$+_I~qncm<03+v4YRk-2N z_eqM4O|+fg(3`dIOiqxj4O{bTX}*;u&Ps}u7eL3k;#J}~1)_N(0sm`QBo;_3=P77D z)m!?xW^y^b!G2Na5n2QKBta)x%Qv?2rVRD1*;x86jD5G3rlkkX)p{O>O^VHmG-Igt zdts$keXKNY>nkgTul^rfZygoY+pqrv5<@e@5F#)z^w2eQ4BbehAl)EHBQ4$CT~bPl zARyfx64E7#AdN`*+dSuYzUO&9=PVX~u-2@Z+3b7Y_q(p^MOIVRN>zC}1oWj3zxld3 zT_JZy-22KKg_~R{3Ot%sE3I~xulPNRuH8jxw#@5PxoLF+`J(u+e&n^R!De`M*KLn| zz#^dz&yMO@%YDjfDi!y6IVJ8FePIzaan{Yp>;aTLd*Qf?3V0E0D5lNl;f>?}5K|vq zJ+vVP23N#8p1FA)p9)ddH@!I1p`1SR~KkvQ}OI$H|>P|$Lrs(%iyuJ=4Qk2e{ z*BW`=NSOHQ?9Y=sc;uck3xDXGCE6=v5n*JmOhe%X%!fzqJjQ%#&I(Z4Umsdj<|hAFag+WoBqUQG&DN#YM74H z_hBFHrZmm^55Qn;dF6}{hLQ%p6ArBjPgppR$6h~Wm7}Mg82Yf2Zb}7K)}fE2U@JsW zr66c27}xgEzLP&AsFTxsr+ln(W@;2Ny8Y04q7cvIekfslLAao#iy2J3_eTai%&H7% zYDBRIdk~VZ04X89QtroqN4s)dzTlYm^qW08&Th>ig$n95KPe}@<(Dk(tFe5t8Gf#~ zjcb}m7ycfa+LrMLtRjBg5#)0hqtj|1vAwc?gi6@!R@*3WPM}{#OJ&s4#UPt7SuO^% zQhaRKr}a`tt9p_pFj_d?o$8%o+nTlb^>l*uw9fKgHzb2LzR1LXB+&v=Txj|xa>vFj zSnGR|H7-?9(ZG8N3ied4#*cpIE~FB3syui8iC(gvr`^_8XOcMaDZ zQ026+)Q|>X!ImEzmexT-l5O+7@7P$w+qb4Q2N+8M?gvzvuzW+nJ1C=lhY^H9s-0g` z{|4O^W-3hNEypEmsx!@MIQ}Va&Hwi*V{SCL>qzk3R!t8Uer~-hw=#5JfM1a+?PWV^ z1RG5QZb>a_Y)7FcUyeQq$Cb{Qf{l*f+p$kz!z*U`)P*cAfWEXBEW4p}qO3u`rh#BG^s`g5XO z7d=cPByEof&GognPNwUKiZH!_N(st_6+gtPLW&N}_+yujVR`xe_iH$lL@VV;=u%=I zIP(IephN0{I=4M!0;t)4b@WJFMLQptH|)}T;qe;Bn607;=#@o}4FdHcGZ!K+ZMp50u!xm>V0ZLOF(;jFa)P>3m-=)STKzr=E=H2caEy2$C#busFg* zvg(oW|7wlnCp;`zZ^aDKPmeqkmYseLtQ~p2G@h4ke0_Z`+V)eKQTq{mrl&xe1) zgfnMoK0NY3f61SyJnS-dt$bUl;a|lc)M6}bkKlMjJk$uf?|dA#%pi%!H;+Gf{FXx=o(H#&=c1g z)hO$V1dI9K1IbdMjL*sqmfGY9gI%M!C9wRC)K)%TQ%fBH18yOz?_Lc6L29R^g@gL; z4;romoozanJ2&bF8=6!6&df%Tyx*4r>dza?r08{OS}}GUPB?ktoR}ZNt#bap2FjIR z!jr?iiu=Mgv4Lg3o~{CVhi3A)yV7s$zr6seAF=mDWfn7c5jzVzca5>WKj~w|oKP(K zays@2R+^}aG>HEhs zAUfZ1od*9emUSC8u1$;#4LxFmp+sNcjI!@Ug>V^I}NRP z3u93;vV4{iD(U4SPyA~KT*_E&koU{|EpqhlyNt6YyD{)|4LP6FzUx)CHeo9JEo+$f zeSa`(T7x=A79+;dWLWWAFYKGqLd%k08$Q%uP_@ssahvUH#yBZW1m6X0@<6H<=_^(W z)4M0@S4|C@ZoGi1k?7F3+GzW*{cw-Uw=6E)I>B#@yOs9VKV;3>Y-r`(dYP>v{fW`MyS2W00soK)dP_KLT^B7PSYd zlo?;m-$bv<>G}+HIwC0`5WP#)?q2berhk(Xwzn!hDEAUKuGbgc_ut&!{wpAULLJPp zu5x5r-4-M)^x;={+qj%U)B7ZD)A(B`9?G}E$2~1-3tXoQI9gtlsft`3i83^04Rnpk z^`&appKKXv#s1J+q*7dE8$&q2RWzF&#P;;b)v79E`GCZhD*2QK>q{Ijq!IPi+FeKQ z@eCg_%S2Lhqyo!EM+9Dwl8;Oxq8DSBTJ0p2t4FC}9E$cYQYP~w*j5mlt6dLQG5rp0 zizg)szQ|p`M4ZTpcYS_}`Mm<9kKc_)g061Xdwo6>FNPU1%L~2|u9?JA-MZL<=a~1@ zz@~`geVP2jTz3@eF`0LxUiX)8g~Y~?61cv0-3Z3Ibkw&$0dHOyWjiUCe=+nG2{In> zzj4xTn1PU-q^3VosM(nPR*#!7K!^9b4<&CeQZFKMVx?iXjKFQGPXOEP0_?|_u-YsU zg`7lDNXXYwHVR1yDmvOT4v1`4%^G%8^q<%+4YhXHSgODnDmJd>?xh#+MIDn1DF5p; z$wHE@p<&uY{j7CxP&4i!j0{YDUK|C|;=V=5=3Z5l+A5p%zap~! zz3TO1&zV9%XQ2)~G%Q7q$bGD|q6&Od9a(8uLX7?Sy4;EYN%|Fx5>}Lcu*#UaNas(D zP65bJ!OVRdC25~(R%wwUP}kGaFe&+sw{JK9cj?IGp>%YJ*V49l(oxX%OsLJImQGYH zujo3&Hr1RFT@mKiy!xf0(E#K{9IUGmwlJRE%Q{#A3S8kywA{H zfuTC{#V+s$V*rWbE0(~}%{1{BQoK-V_M|`J1fzu_*~n!5DQhstxeKPm&yJ$m)H6Cm zAx2?$cQoyO<dLK)sZrTTO~`pVn>)zndmd@WOr#<1(TkhxdUF77_Z1wk-=9 z4V?G9c-{=78|SWFj)EAv_`%G(?=eZM8nxO~bUT>?@vvL&ZDott?2rz~NG}VMNs!P{*LqUcm16eVE?^!eU+eLFW zb>T-=$}Z325Y`;6ktZWHREca9U}S%)W!4*12)Z(nwQ(_supcpmId#Tms`4$vff6n3 zIKz+|YTV^Jd3v6sd1-v1tYP-Fd7$jqbL`z73=QhTyV~M8_5v}A-JV{3XGq7bdsROc zNeYeMJ-6j7nTY5ZZ+p)RcK4KjPU=3ClmKO&BcZDopS0$4(uU+( zm_1Vt-+$%sUMhd=oIvItlM8bF{fn4vBMH0aI6=6X+wA+oHk}{sPg$u>!e~nQ+F@dU zk{Hu459bzu0w#<#1l}AE{~`PYIg!wK?gQ$g1}mwUc0L-RY(zaBNj(VBKgEhTQ}c-_ ztx5CH_N@f{yb2c%a@JaLkgNRiQc)#PVpfB>jY`HaTE&*4V{8fPgUe%u{q@1ji4fzM zGUfGYK^Kn6L_Z3t)JW9Oa^6z?^f8X*XG@yqMf_e(`tq%zJIkIqHYXNI=&?X+eSRUi zf%ky4%n-Hfn4c$yk4KntG}oUF<}jo6fTNG=0m^KN+|=Q;D%MFfaoCk~;;f;Sn_Y$C z1#%!sRJYnxurm3OYCg;!GAL)bOKWnX86 zu94!u;}@@K;(S|%o~7UY{N;oI$~FoDR<>oj-;mK!V?5;6n@Xoc5+qfzrfk7KtLB?6b?} zhpa>4e-6|dJsYB0WTJRc7tp0RC6QN$7N+4cot*{(c`T@21 zA7|PBa+pN`lW)NKbmRJT=80Uv|FkAbzUrsF_nv&6)!Ap_LH2S2w@fYBQKZge^Ji;W0nMAG_IZM8QKqAyHS(vu2U!WzO@c z+|!x9<6d*V&deXio*88q!A8vO3WbB_(Bdo<6N;F|nZB4~hx5Z8cYc`5xe$O?Zu;HB z&6{(qQw6sU2pHkTP9!lX;Z&Ku2-EbVGPd$Lx*8SiURGi`oEPm_DIc0mloidYu>M?k zTt{Yf0#y!bbOSx3*VL-W^?jkTtqnswZPKk$LoWDJD-2d2V_~rm=+N%cz%?{CfGuh4 z`nDl8cj-rhAPWhpy{ELgVxo*b9ZHxlH4Yz^HA_UPp+CO%=rctsJjP;#Y4M!Zde}7E zxgByU9nMYvF+8dg|8|99DZSk>hlHTS;Zj8g2 zct+Vf%bsB-3_4Z*4~eBso1WUeWen;rQ8a6(OS+p&_@WNqmH72Bn4ZCBUPALEw7(Xv zGksOtB5N1h4GsTo#^vjqJy<-2Q+Q8R(NOukZY%c9KIXILOnBasy>a;bFA7DPc;@Y6 za{a>*FMB)q`ZxJT&MN}jhqOUWnd}V>p-E-tRGkw;8e!xmK{v?|2ymXBWpjSLyc|tY z@I~Q#pMby@p^fZY$GWOXoT)0>C{*JqnkHNda~etfs{?JaYAq|c!&xWIG^6&XxY^RE z*7jH-(Ix!aGEi0dha0G%)Xg3#@$I!^>)vzwc4Fh-CyOqJO%|VNPF!7$7T)j1X>m7G z6jUd2*j)Z+yPFSfF!`Yb<>?&SW|r)n*H$ff9vbucwS#tkNK!F^c#AF^4~w|*vAcNb zt0*k`DONkrEzjVzW)8>t)W&Ks<-Pn*lpg9$2303yvIz}T~4;Yg{2#gIVHf1{wO+= z%!9l;NElwsNgpD-7imW1HX&SdF@YpZ)J$e$bE^Mruk=gM( z;Ch?y@zD-A{3&^5s%*9cT76a*ed6(yk!?INvpQNAPgv^-2c|{D9g(zX!1fTS)Gwfg z4|W+Dz$oV9OR*?QmHaoiiTYjvQ0FzF^k?WOl;awNr+ziA49El1@=&2R)HDG(t${TYk;mpi~#$CH)}SpTQk`WK}qt z6n^Y`^&&SUzR1m{ucsg{-!_)8vxz_X`dDrJ#5y8%s@ZsP3nqqDn96iVi0*XBB-9#( zGNk#@g_=V>!XH4yoE*K7wxp*WV~9N}@Cp6mdrG)+rYtsQA5!n923?+-C)^|XGMAff zuG}wM#DVp=L@~VEPcPQtFS{a&qkVpW^ULlgzJ638BPy<_W`8gXnp`1>G?-o!J-Q1= zx{xs$-;XHZV&?pje!!M~ZP9w0**75(kxANxnn3H*nJ*lh%6#jf9Vkm`ckmFtB%6s9$2aq{lwh25jsW1;5sOS=3*Q(g2v-h zSdBwpZj(#z^Ezo~v4QKcR4R-UJBeRnkEVFnPks_5dN1nAyPEO ztg3AB>Mf2vw9C3m@=xNx=3BqXJv#ch+PGiovN3FrDbh1VnKJU~cViQh(PtjuH7 zUhq@uvIMI}@Ki(sH`x*PV_F5Wnbg14EG!#!cE+s>QQStf4CO@O`Gwxa?+5w&Qz2A* zQ(-SbE-xL|DANdPy~cpDOBA%h@6>1W?kqxZG@rbOXuPHCl;sY$?ECQb3tER0m1NfC zJKl4yka+{ppXP_9B=z6xD(y|z-@9?o-M%JW*{!|WZwsyP=}BAn0D9cDqmA34i6c|~ za@sz#WZAt8ukwa9&_=1+qwX43ai6{+P6WPFbPP|LnYA1XZ2ZTgp3FZhcxWloM}tFuyN)7X=-8bNjckn1ndj$)RoaarUUt~>);b|_2(r^N*a0XP%*3L za6E3Fj53VNmY7<&FchP{uYhrzJqaX0?imSXBsO#)B_Web7Ti7n0zKbTELy)hB&{l# zkIWMEEQyBMFEm~8R(GBBvm1Y(BYhoI=FHyCo@l!#o=IbsUgNP6Hkr@*y6&m@YAM(ev*oM!%d1Y2&b zLi6`KMj|tjSft7|6}d06W0i!BD?9HAsjFb&$$Q@9YtY+YjmLVC!2n9nPA?)wA>~zC zY7Up`BzSXO{kcNSlb&Rk;3B>>tEa9Tpt$4jp-N7P;8`FoWf*ntW zNbdptvkGr?(wvQ=$?bIiT+M_FXgjf68kX$RR|2vfF|KNU&Pc3HmEHd3N9BfhncHqv z;C;y0YY@9`^HAde>1I}|2JgwQc}sEJ9=@_xzEe>Vtk_kl-eZ{f-BMqg{j6PIjWT^rc_q^2UvE#nwX-^C^)R{CN|ua) zMsJoIo&Q3d++48i_XQ*MIMVgTv7uXx^h5qoOVoT683f+PXGQA-<+Y(zVk}AB#(6q6 z#ICwCd>!k)hV=}Zd(AVd<(p)nwX?^wL|G29MIW;86@n{AzvqUb1a|!n&+Iur0JRXJ zsjZRHClMv)5pbC!9VQwzq2<;^HtDqnNR&a}CofM_(-Mp2BtHFIt8T1F*qPJK56d-l zyQ^9bT&>w$J(hPJM0DDVN6u|?bM$F4veTR$5Qq1^c#o!~H@YdHx*S7|pVHe{QkNzU z){`BDJH%$nyw137;|seCMA5zi_f9Kb7re1>&#W^2W1eJ8_&(EvF(UBbKI@9v5i6@W zNrk83u@4}NDgkdC6TAe5<)jry`>bQFaP4bX6O-qAl*%fE(vPh;;Y(}w-#(( z7CBZNJay^Ik zf3H)>D5i4Uic`R3rEvUEqBy2zlwz=KSH^@bMTNDb^Hc$+#DP)HISu8D$Mq2TVwF7A zcEvK8Hk(k_vEjAEIyWrm3>O2Xt#>U9{n3UWI6e@Y?Qz{kMSoJE_FQot1k2-5(aY2t zeHcM%)i>}@szXLV=y>8wEDfGR&^FPOZ>`x+WTs#&Ie9h7wP zKctG%KHC*ZZyb>2&Ez3w%B}_B_J&EU;nSok^52#uNKoZlrV6K$J?~AbSiwi#90i_? za+YFB*`G*JPUxYHbg)!tQzRt;s{4emL3ZaQA;T3o{zD(7`tw1M5ns;7kWXef^schv z$(FE9y<&a8az`jCp#>VMxDa4$3_8{uUy6= zxqfxPxYhGx_xfOeiv{G#%^JIWgTbx}`3TDwS6R%WkaTUqKM}oDIXn1wlIXSvSAR3D zW8#c7dNx)&S?3jt6<((A4$%>JnmzmPWBOm=-!}!*%LwCTUiw7Z&jB|h-M6a(FP11e zM}^j=uQ(={d|QyV4WP~+K6|#7Pqa=(A4_(2j$fCCh-s4rr5CNim7WbA!d<{9>i12L z<+Rb0WnlWQs{<+~r^HJ#!nQH*Egnp1{dAt|7Wo;_di4(ft3MWn(;m+^Vza#%$j99fRVS6jMdhb{>vZ(Xdokc1)9 zT%Cu^ZNEM_ptr>5wM*)>M0Trg8mQ2O=2Q4+-#AHx(jRYu#5@H8vyVZzKcgU%CkGtmbX&JsiGRm~d2WaJb~ZBG<2 zQ)2Z;P(LpS@+uI1=llgFmqChsxgnVhPIeGNP<@e-o!&*u))xJ!3g3CEVEi^Evk zS4xpH;p9DRnpLl`m$^@SF8w;}9wzvWm3iCYum*Y~xF&534pq2omBK?A%fzF{wx`Ch zhFrkLr-#%&bX?xJK+vur>Evo^r7U4x@vEfyw(dv>VrM+s?PC86jlzLNRh%#NpS{Ts z=II|N))S2<4Z)FiypO~bwSajX;K_`D42SDtVQAflUqoE z7sWth_b$)pvNgA1hR|V(3vgJKgV+FD>+uTx`u|V-&QX!n18s2TOlHTYV|~Exf;RSAnmUQh@c$VsLFT8X+b2!qXbM>>_Ycdy4A^!e+42j{<>PoI4h=+IO-`31WV57=#$x`alz#**`HVnzei?CC2!Bc5(VGmb57f@n*|ZBcJ&* zTdat#(kICA#!=eELtm82UM_mt7$aNe_;VxcJdB{$5+0fY4!(L*oJ_+)lN~{7VH4?r zUyppe+d*_>_@}@Y=r~K1{DdgK!E85R6({C2iYedg%=qv41;afF5&KlB^h%5{v7{i)UZH#i_k%YLAd6 z%)7ur#LpyL#9w8_We7iNgiy>4h6I8`m{SgAfI5m+KQ`+wHnvzrzdU0+(brN|_w*?X8f$)<9E2IW8Z z?tAWWz-jOr`>t0f=6qlNDeVP%`KftXvqR-;wAs{wp3?4_|Cf?JS8KhQOp1foBTHC~ zD_T(DcGCgV8ZfZs7&_8O_btNYG zNfFVRM^!&87pg=b?r#pJ@Cyyk3Lw&=uYOUL;xOj_Ws<-VCCH4%k>p6+bcc5NNJKK` z%+0TL=q5yif(cD?Bz=vIuhr#D!s9MmW% z6`6z{OS#}$I!GJ;Twx@8S!0O1Y|@uG!?Z1UTTCn)zo6D&i0bg;D_M+jZ%f|M4Hwc7 z^Y+K|{HKO7nS43Acp{ULuwb}kHKdu9y8N6fD|4J^oYxGWS&HYei%h9aw2RpFI)+lY z?dFC}bl6mG1Cj5vL{YhoG@gkEGS07k5R~I8JwO_a5TLH>!>DITrj#ulpp<=1(V)RI z@7=vH81p48B-`vL-<@?{om2V|cW3-z4Rg(%;FgOaM~0fp7kGfD3{+r!eMly$ z%2k*#Udo8(i#dx-yl^_aKS|V7m;;5~tT)`0r)Q4I3)~md=cGQNAnqb`iY#}gkkOzskXt-(vDAYV&TcC#!ac@X=9y_4%kh?mWr~vx zRb@lwO86|JU6cdbxr?)ubqbpy)Y?tO6xu$XX2eV++3&RpC2XDaNtw(z(EKncX%5>? z8o+|p3*qZ6+VoLs{vp#?IPr@-vYHBSX}UizF#O>6?Ss^NPoFsp2B=}Mr?VEj4#fsM z&A>r5mr>S1L=IVc6h~ioF`EsF{^9qw#ttuUfOwXFSmalUVL&FqF!>q2!zsJHO$fEA zz5=tyiCpN~w$Ujx3}WbGGtpA%7Hlp_eat~5^@}jbDCsbsLf@8p0i!J3a|+!&@y^Wc zYUGtci~;P=#R-?f3{kgRAIXcp|4f#eL1<@zw()en%AGKa-J+%N##0b@5 zl^Zac2A*6Xtf0`a0?@_R+=Ci!(K!;O@92VdZ0{{b4%)vNg;Hlk8s7X}lAa`a+zVFS zc_&o)fkHKIH<7j_X%4+`&}N-f1g!~}lbY&Xg(Vqs0wO6kUwrLJm8ndf+)~QvJ|Z7a znJKwmv2+WYr>->PDlkL7G-xAbHr24_O1}gTa$NHz#b*h#KHQ^6WiEK)w&qEek}8ql zQ>6d=_4-riK9o>;uUHVgmYM;9u9~Fkcz{P0XdsxAA$F1ocvS}2hDAzu2v%BUdRI6Y zme7d*7}9i9wZZ}S4>4Qwg8V`SV>e_SLrH@b<%Q4?Dbv}}mXV0Kpn#^Ukw#PIY71rD zDp*9GEhWK4HATy1^_iFfQeJ?YINRBO0cw}5(u|K~he&d#}MY^iB^+>n= z+p1^1inHrhc2~PMvF?8sY{rAS4klORR-0rSSl|v;-xPfM1`;eJD^C`x+TziCEb;7Z zIXc^!zo^BH1qW67AkmWE=1@m1adqgON|)yK z<>oIYb?);e5!2D4$&YdYzLXqXn)6_ryudGA-l1r5Lsg_$#Q>MDRr55R`y#0*p2^hX zJ3{S!Jk7v@vTPDX!nM*3QmC4u*GG0m3zn{##T#pY7v9KEh09PR7navTv7h3n+W1v0 zVLJM{^tOG2oFMCTB?V`|I%HC2CM64RASU$`Kj5~3JO4>qyZ6CPoO08ZiR5GcQ=DovkvT~+4S@H643|R#3F0&PVc|sC8WhXescWYV!e)=6x z!7)6T-67#bR+JKxN4qI&Xe8`kam2@_!U91i8@uW}J$|=ofv(1Q#D(7Ga5Fd!>>s}zy2eKTGV%ZT@<51z8Jw;th}qGJTuoKf zk#7)0%1(a0eIoILv-1CV^Lt6Oj+mXl+b4vQ%C45Fiuao*EGYlS`yMBRNU6a7vS|1| zS;%r?I17F9$!^+`hUp39e_vnMIweplI`g`eA~%e$ib&y=&`9AOtni{cWP@hm}H94`gS=_t8?XXcI~JpQ?HPHTNu%iB;|=l25WJ z-9-*z6dDDHjstP1zSDJ?p6)5C^I4^b^$}^0+3(=b^3FY0Z)|+-XHp|q%XoTjNXjKc zHjtV3EdymZGum$|lj)_PHn_wgGQMP!)a|q^da+b37vSa)ow=R zOodw0NEh>l^Z%vcu@Xj=tOlGzjyzzq7x5K2%i7{6@$Im`kCP^h^wJpG`K_XnhO zwAwlht7^yQOvivsc{)0Ih5;{%W0QaB<&1T*YR4x2%PRo#6!4&@T`=}4qiBpAn^5tJ zuxdo|ayN)IxEeiR1Rs>q#z4uJQE+^yuK5I=E4}t(F<;&f)!|+&K2!>OKg#gTZ$eyA z0&LG71oejW)svzm{EM*+s8C#buX`W! zaPKdt;TopHE9b*B;6a+#@LBzq9x?ztg4J5UB-xkIUlhfWuzk@)k7DNcL*<1p@XHqN z0KiZ70QK{tRDnwmHX!m(K*kWc0wAT=U#fH;ESDz%0vTmI4u)@(ync3-Rn`M7E;Fq6 zL42Qzc}4svxGXOAT%wi7AvnHxYY0##D>y6x6`+S!^=3&%T{B$U6?&D5jdvCa#@?9l z`pl)NJ9}&|82>b_qa7xJH4G_5wt}>i{t_qh%)Teba+Kq@zhJATfQD zOI+Tw0ATw3x$OP=FNmwZ>->AI?_r001_8FZX83(hZ`(Qi2anCG_6NZ^sC8YS)<;eI zyJhL=$8qquJ1wtjmi;%3^rFRhk$o1!_qnrhrV!Nzwc)=3H;2r->LS^)?%7zpV2j|= zxjakiu<#idk#UfZl85RvZ*FSpU)k|KJhvAihZ8D(o(>GS|w5>=*m7QH@dR+r9>% zZQvs zMJJo7tT^T?2~-4+4?nKcr0gzyzWnfD)|xeLKIt0oR)JNd=9I5*uH`rKA9XLF!ES0y z1RQY2nqNOzSvqIeXmDvDZPQ_LsjWq2em}J77U`?i@SZahBeAQBG8Kv)&Oo2p2qnrt zJSh9`uNOK47R@5$#txb`m=9gdZ0kX%sKFn+`;_ovvZJ50Hu3g7SPpcoAi94(m^duv zSa1ykLTruwO)h(z)=Zzm;w|9h=@^qa2=@8TDY=JQS2KBz0=-zZ%XefS-W?;D%8jcWE_hnVI>`S4b$6SOp=&n5iWnjm$3 zraeJkf&HaDI;Y(C9Hfi_FZ0Dd+X zZx}4x4IL&osj1K1aLvCvu-UU-*>u(t9qlKzPiMFah9;1pp#&SIGSFAxr)E4&dk!NYtw(*Kjw; zlnQvKr~yg{aq7%ymo5d3Gtsysj0pzQT=3hb2Tn0=ENgX_*y#|1cBAH~_YjgMIjDa- zkJ&T_I}6wNyG;C;%L$;0LY3U-p%DFhUHkXb-}@U9m8Mt&(#;3%>4o?hI!r9HRe@Q9rLyx>8W4IgA?M%kXJ-OSyb>2Bi+H&(koY{Vr zFJ+DODiN0uk7}8X58(S-zg;rq44CgB51Bn-St8f%LCJo;I!CN-*m49oQH=2nR){az zZ+;`_naj}Rxd5{G`1NW)FpH|$k3Oph_RnYfTrHjtSf?zrCG%1fH2#Z@y!QHNXz>%V zf~u2C5@!N1g!3G$L=1|@@0D&1IJf-Py{9XrB5t{oA)>SX8^e0@!o2&_@u3tL$}@Hh zEs>BJMYGGopi&AwB_jB|R5R+*PBB*^SNQJ#IN`jK95)phyoiJ0?7|cxxqojb|GJy) zbo}z*%7_|i;mxU(729DZCki%*Vf8V}Owxdm69nV!DG_`|*SNm^()#V__VdivEA7C7 zw`EFy&Q{M~W&6a?yKdWc3p(n1WLLNQp^d(J-Jm zdUj~>V8atTI$kmM`J7HR#jVd@;4mp>O^O`62lc=~h`XIcNJ161C%R&$|Fc1>_wT>B z2=~W}{<4j(yF7du`0UnUzD5Z~Y_Z^3XgsV$v8fv2i~bIkbo5Hk>;)!VdcP)-Dm2}J ztFM=rGWp;y@T7A$k-yQ7Ck@K|)&gLzk3bsnuV9&QH*`n{c%CHst@iTcCLHtRPp4CGT%SF zBAZR|1w=#&i+QZ^a5)EA#aPaqcDmF#08(gVA`5FfNUTpDG9>`9?j=+x z=fF|pq(tmv`DBNs@;7*&twSb^wtfNpGWV57hlnw-0(DWkbduDJ90Z75ZdL#ycTXur znveJ}=x6I2Ob_00iQ2~t@6)*jVl#aK{J|waY<)t|OVtmYeADqVed^adIZy{@LVw^C z+f~I;mmw~d3BN&by-JF4IsB^i`FEiwL6l_LKi4w1lC1i? zl`5FoR3jSa@?ALnHgSJ)D10EuZyMSwh@CNr7Deaw4;gHfV)w>QPLj>1dC`aiA+k4! zT6YoK6;AT}=Z$d`00AX|FxoVC74%_jM*SIX4kdlt4o41EVS~_=n(N*c%Y2t)jVQz9 z-kq#)o)#1KoyhiOaR9A}RUhsdZe?vPe|prTf@$1gmlHG@*e8Md=PwYhd3~YgFCP_s zAcJ4ywjYv(4?n&$TqD=TaAY4WiaIUwZO9$qkrMsdgHcJ&ESGdc zJOZHB;na`{qHl^wQ~IlHi?LTNz!o0Owfrbp8y^%~ zk+7&$%xQre-oXr+=mF#rI}B}(@pTNHKYX=Z$sK1Oj;qValMc*%1WkoV28z_43TJ-U zNTlIgBU7umuPC_&c;-fJ&=pcT5QLc_1rCH`2hqk|Jr#zgkb z2m63d-Bit=I*^=pt#>~=0pQ8i*KS*(r!4WgM&J9gx#EqYuiCKs-`yAYVQTyy(x1{u zps1HV7Fa%}Rx0tL(qLkvO8ASKGBbh&cwHa>LSLRB4=sul`?amJ?IKwnwj6YPI3AS51dt=m^z5Q2l`HD z{{{J=kL3a2#5ImBYRh2!u|$@)wxNq)`zlo7I6SAgo|arMRo?*!*j2P1A@-SGSCrhSG(+?h;cu&|W)5Kzah<5Q0em#ZWPX-;`m*^NJ#OaI zHx}tFCUynp7slUJ)I>!HyvCZt@Sc01S8O(Nz5HE7nRHbxT23Egm{rR9vEmBE$}k!gs3i@sxlvPATE^E-Q(TnB&ur$;a6Uhw~D_w;&8-#Y%lug#9L*UV7!5;K$*5x=}4VW&HlsZ75H^ zmHH#6)+_pYYAx67A=h?1-!$G1f05!XbyM`zVeai!cG($oR|8ONcrG%U_@M=K5-r7g|!p!snlxY8>XZ8dh?FX^ipuh zxnamS@4hIfnbtCIwani4tT3XD_*fG@NA`bjg+HVO)yTp&#pYuM6+Nc6PhSkZXq+Y% zY`X);n=T3jB_c-#{oLFE4H`L*(UxxyHnZjeURfY7QoCBemIjOZc>MhWPGyUUbTkX* zAd+YpmU79iboKKS^~~p3gJwc{yP`M3Ak}I61Bw}C?^fqWv#FN4W^(zeD1)>+8S#nA zV*}-B12xfaUMU?gV>Y?#AH5pau{mbkKiE@g0=?C5UJrPU-sp>KnXYWBn{0Vu4}o#a zT4QNvpW^Tsp#icymkI~;Daw89vAE`8V=|gJC%XEYf<~bmqQfcIR&1HOelXc>CUG-y zl!9qy+3uyD;320OK6sOerzPq-bR66x2q{YpExqn#@a`bIUiq>Q0B;?(d5uhQ()1wx zA(-+v?#D%5K+=S0p&9n9!q^ z-(igh7hJl^$VNA(rgcs@RZipe`IFVWGgF%AQBmOAD((qttByVv=PtIYr~lm2(&d2q zAVa zy(47mRMRRS`ksS=7nxM&$YPvv4|OC-d{&$$2=q2h-UHdoTTi-JSTuuou*dLA!M6ylxKh9*0DgBc~D!0lv z^kt;RG*KP}%cqz*2wC}*&mPU==;t>?~pGBL|mHCO=w}ej~9JQhLW!iO>49P7J2zG$WWk$*MqL(CQ^Psh`+W=Y+L+G30yb2@OB1 z((h1q4Gw}bSo0jVy9`FJP5FP~<{|4l-}pk5`PKIw`=*wXV{bh7N{|C&PI%I4WXdl4 zc@eSPFGs>@;_#wb4d0WtKp}ks8(Uf&vN(oan!lJ&uAXMP%@6o@8OCsfYV>c(IRE(^ zFw#vC4?_ZQhM^Y^jyNwp8*>;BNmHstZd=#-DFu~b>wLRkV_FX!1(#D(MYg5D3a90R zb8W_HB1Q$37e-LIC5AT>421$&K9N^Xg-) z*w{)`KlIJ&J{7)>C!yEhPAg9LYXcXuba1$Wm*g1dWg2;R5^m*6A; zf)hwc_^bDK-t+DK%Dy>MlG71C@5YF5Cav(j4ZiC8VIgYvUQ3(dlYbdT) z$`+BWTeb%k@C~`U5c&egH_2!Gj2_ffRl?ZRAt_DMmQe1eb+DGw1+5W6Ut}X|Dkgde z&WPvfu5>EC;+$^>k9P+KAq(LlzZG5ImGJc*W;Vm+s)hLcTpRb8$Yf*;*WQ$> zN*1*Y57QdbzGWuCY*bL*nL3*S;R&JF$bTgHfe*TB(CF)sB8poP6T%|HO>sZ)+|mKv zq%s%8o0gGH%M3rP_QDsS1>!v*phZgRL9dfS#Yr*lqhC{4CMfUO;)hEp0p;zEiT6n; z8fU}Ha5AumE75+l5+$UQQOfgpXUKP$u7R0pK|c<(0T^z$nc7E>Br-@NITX>5(h=2@ zi9d|X2wwqN&Dkm(TI}!fk~&1RvCb@R)Ap&mt~c{aU!tbL2?QLKe0&^Y&rIrTI5YgU zocu2XDa>iA#pc8{#D;@xf9)_eTZXik9rQ#pIL$nrR^f#20ue;>uaIv(`1g_PjZXlh zCaZ*ocEz!G9SajJsmbEOequaKu{fu`n%D-D<@-1vP11B(!;h)Ggfpc+_|^1)u~iPg zlv-et+&Zy*iq1~~ZZf5F;Z@?En_z7?2g|tKqb=gsp4r&=&f;5uaTL0_5AbZo!X9jC zqzHfYG#5addJ*A)E01pu0L zLrS*$i?28lH|05THpoT-m8ALEQK$RKF|6=v>gzXmo2l}Um*=}Z9yFP)*9^u5qRH9r z=R{zpnwdM3Xq6O!&pW~HDe`-g4wv9yCypJgMP9>$n${`%dUwhJ&&OiD0eU&><%!K| zw~H7lEF7~ZS=hTYem9{`ih-1rmW@DN&FVS*=z2l`2aS4uwaEii{=zcPr9CEhZbor_XZ4>s7nI zv81?Ly$3R0m)CNrBo~`KVhwMip5n85=?#;=w{d4!NmzIa5?Vi|zC1i0od zKQHhrsVUF=NZSa!UMTY z8PiCDW#z-FruoDuwFb|YZZtz`%6Ch&rS*Vzy4w_}eE08^S|*g;h!fWy40#qFmH8a} zWA5L<(cT5Ren4JO$sMp-EpH_QK}_ zm)j52j+s3xqt0{z8&&R^6Q`||lWc#s{Sh?mudMWU-JZ6h=;u>|<%U{QJkZRyvSy>D zIkL$eJkVBZ_4dQ~SXEu6%1~54T_$>yzJ0ue<{ziO0puL4dWBZsqq_M|$3=gt$1Ea1 zxk^hDQw_#+foWS&IRzS!rY2mrj*>AxOeD~DearQkG}snrmT~91{T^VB^y!0@e142} z^&EYbbQ_wLLi$niCy}gR4a(+eJigjnlYYeA>9`{VDMJ#aY)$y`x{% zHb89H|E8Wl=&&D?#!ju;5qtd=DZvLuYRSyTIxS+D~J zqwOyLnmd%2F7zMN`MkJhkt7t*KWZaiW+3-eY9{x%%)1A2^s_UL|7`WUUUeiLNU%;a zq6?h0rnRct?Ev|cbXIvdm$yGR=wSx26}*kRS|ptFG%**CgRFkrBg7&KKZQu-vjI8F zkA|Sk<}qk?bD|C{Wmox%84(W#UG8q|t1EXA=Lkb6qR`QR80an)=^jcPS zIXevh2p|r{m9cgYjr4$2=kY^Vuaa2n8r(4xn1OXj_)h8mil_eLxZ1_i-9gdJ9t z2%x?drJzyxKD0tfEfU#O5}Xld*NL<~yh)a(w=E=D&obv+2qky!>l}icnVOmcswi2r zjAL|o)9LDQ2Kq#n@?w%H=6neRk0pt(V-c-2dR&tf2Mt;&zFu4JCt}f(XG+z4hZ9vK z{Be2OlO{Yz;wzobW6K{h!dZ^S){aHXGh`bf>L2>zZLM8?Uw-CqAXl+#?0+_*pFe-` ziftVIKM-|L=f8{OK>&FXC@GMY4@$vMeV1q{r={l}R#DFi zz!)<^b$+wzd;E?j&$#-_>UfRGFMDLOkC(4_O#r6qCgFI-rD6KaePl+;YC&d!P79SN zy+ms=mzIPVlPNOf3>-&p;lt^~A;KYQAB&5N#h6Ip*dcQiPdW|8PO`KDlH(MBvLQ4% zp($NlGl!G=zGH;N5mY3r%MKQo;)<)PA~FKFW5-rd5zAF?SI30$m8Q_KdacJhTt0CY zOKoMo_|}RMWqP?v0_SVaH=)jug8EZa{Dg=@k%Szu ziU~@Jvs=ETXlEewqaQqK$kG()t7316hevCD*l&r6dXLS&w|`|F4o90bEPWa52^~TUR17rtDy2rEYdEQiy53 zMKX6!M~X2|gqq$tOYyxh5}lCZaq_dX<*?{cS}|eS75Jo4u^4p2DbDpu7=~QU-`-$mPLcfBuESQpKfpSO zTBSUinqqxUuYB8MtEUD+!=a+~S_LJKN4c6?qlpQ3T{=gr&>Hyc@xtBrbvTAa>e*yz zDi)`%$9200zX!y1GChn4$+oSKV{;>3p=bf0L zY3QnYNi80A(Mi!Jj@T`jSo!`hUWk@vE{uoi<_jn;Z7?D^4ARZRtPL^qz=C{Ft&(>p zQWS>tc_<^oidloQkthu2_tx1FP~<3Oj^yG9RuZdahHHZkX04sd8yO-M;}z?X##E1L zE6gj^%nXM<*d)|v!cbE_lk5Q#>)SHcdw5n;^Ro-IOuW5TR?j6x_kMUpW8;g*%xz`2 zdv4ecVNXpKmlRcrb1p|kj2Y^frU8c&jm3b7Ww`KIdhrzp2{4pR>0Jne3>8lx$%f)S zGqbg8l2XEqZk9YB=*2lB)8Hb(;n%p-wH4!b;q#VoNOT(6OS8VYEYb&)6MgL{T8dXF zB*LQjS8ia=S6) zx2_ZX38>PDM+NNLUIgl5a}j0jJj3#XA50*NjkXMHs!D2OxAGgZ2#^X|*k{ksbX^!! zjR^q7SVe=m6Z!c`2cuWhEoUYn7B)WdrN$(q5X^y?Ul7XYTwc`de#5Gya2yI8%Ex+O zOY{uPlz34-8k-s1ay`YsQ;a@_8{K)z;^L%C^12aa?H5fS8K#QKu$qmemIll)SE5lh zkmIv<$G4cEDzi}^F7}adI6Rc3n8_{d$0?C4)jo?!Iv}aUM#v3#HrncRT3Yk%Lz8Dk7Y)WoavP+SPuap)HDJ!zNt5FHcaf zr5ou-ErE{^-owAt^Q#8g1{{l>jwcUnI3@vDw{n#;P$$l`~##0NQ4|G7qGC1YxjKi@5zNp?ca3^$n=tSQLg z=YEBYxJfpI)%R$qFPtcxO@{Ye@+(b>-rd0Rd?Dd5(XJ-W%~u-!gU|xRuWg>K*?UO7 z4u;-!p4@$^%&egm<3<{LC~M&5h|aV2jV=#5&L7%x_;9+)|Cwvl<%jx{319q#Qlwg5 zxYUb}d@!#24FX(2;V)`sAq( zw6RN5Q~t{I;i4AFN+z+lN6=@cQAbnrptO)BPw^lta5>Yp2obn}_NkFfDm`5->WD4eYnD8mjWIV#itOVxrLT22z=TT#GT z1tEp)amB*JMvjiHH~t_nr;c9mlBP`NFr`j zu9GU_**vRbc=n)*3JNDj_rUQL*It}=Ib!xR%OUoW5|;jb{l7pM^sqh~fhJ$r0Fb=t zIMHet_bZ5bR4eIQV;9-TQp3Q5v#Q4MPG9(^n)l(L;8R~QuyZ8M05|M3vTXmJcexzK#77*{=fLWfc$M-Bvafw?$QapA}! zj$ZEcb^sw|i9?@;mcpCH5vEmI{ZWajA2QgeVrkEnzL!VGn z9H$to4gQqz6UF>2mfMH?=h^%Nn|NQ5Dfe|&k)=)W9<0or-KIQ;LE|LM+w!AJkozo75<{|saIe>n&Pz$O=0{vfmX z-(mkhC$ps7ko|Fg|Mz--JdF2$IS2#b{K^}|{?p3+@0IkazX<&0dVhTR+kZI-iv&!o zN!b5B+#fgdKPR_JE_nar!)!_YmxKRW-p~GNjoQYPp8sWV{`hiN+HupFehACoWfaT*A4%HpDPjo3yU`h-;3dxDeLa zGc0Pc!^IYv;E_c;9EV7({94~OU^uPHcQ6ia?-1Ez{#`}$oD}~b)BSIA*(is#`W0jS z@iOTx0Jxh4-*EcYPHD^|$VBV`Xsk57n3w2%kxB52NTof^w$Dg5W3tO^!TCz0v8R&n z0{c?GC00aS1C>ZB@))$=ud#>dP>AdJjAU-n@b`HY633YBkpr-83JYI;)Lq?ySypr7 z!_FP`Tz_5Npv7Dxo5F;_t74X8ZaybtiWDWn4oQx;?d`I8y_h{`eYse^#@7Fzug1U3 z1J5hCmXXl9u7q_2YDBEtXiz1milRhJrEyx;UTL-Y5<;roi4sd}*sxaFKCu0JbG52F zw=Y9(Hr~Pc0SYyQ0tul=Pqh7as3BMf#jNEVEXT|?{BgfOYN+%M)3O4XzB~)i_M#?F zKE*Jg|F03xStlC?Yjmes?oESw=#ENNn19Ab5FfCU3ex&^rB`e0anKA^B> zp*YO@%x^y)@eb`$6LgLPrRFz7Qbw5gF@&5bQ9YU|ybjTJd=B2ZPC!~YDPR}Hgz_y= zaz!)0UoXy_{Idgwnu6)D0c{iN6t4-cvaE+(LAYZ85E*Mfq65VQ1Nt~n7It9Q`vAy` z#Oib{WX$-^yvfNZlxCLvVdt7A2q+(o^07fe4!aoP7^p$#N>5NoDo_DI`AzcaURoK_ zQrnXXhvt_PC-vaC_r?RrbwWYQR(EJFK103$hC(VLD+|(=@zn&V1IIiOuXIFWL%0d_ zR-l}618@(3CKny#*65UbU*Y0;lo_+o2EFU(>;Rzju$$(IPJf~ag_NkhjC)IfW~jw+q}_JhZyit z@#Ejv%T*vwxTVErC=1tUh^A!t1z>xx!DE8f0PTj@H0~aNhDJU2?D(glh396F{xJoMkNe`4K0mQ^zNYi&Xc8{-L%TImzvakR?98{gR z86)!^&c-$H%3))?B9b6K@c)T-2tl`KEmy z{_gdx!UAtqH)_4_RD5%XaUTuc;s8D7?|;aGQ#Y6lcV9q})1}?bs@c`tx6w3-?CBlW zOZN^&K?iG=Aw6U`(ApZsWweYccY4jqG*HUJjKu($6{45O;38e^hsWg};UOmQLDK{OJg)eSzDe2rU@Sfi&GC{J5J@)5`i zL-BtV(k+I{VPS2$>Ex98N7#G_v~ak{ag2DaF9G?qPk|R<6pw)Hg~DnIRYT0OLPlF* zR%5^6N%gzGnNo1%fOdZz$hTbb28wM{qKwtlbQEdUe8kxu`OTur`?iA;ieHsQi=xK;02J47i1?w$ zKLD+jmwDa(&wy1RWB@iQ7vEdr4glgarLJ$kC!yAepldDyR0a_=^6>lgbDI|tHxCNR zo{0oiBm^sZB9o?3;Y929lm>+`w2UN8$yQkKFiHd>6QES+&V6pS^TI@-&3Z>;r!p0* zl-cQF7Yxdlr8Bh!9a&*GF6_VzC$tO3W`8e}c&BN?B)occXfKdYT@KtXjoFx{f>^|*N-0P3(r#iy9X$jFkSOX0dW3kUnK2U6HzYn}UpN##jxwl0&97tfQ2OhrSt|op+pBor*mMCH}q-)Fq^JsI6lM z^O*O`@B1>k{j(#A4E$+ulE_b>Y$L(YbGLrB0GFm3)i+m~#a+EdSDA_A$z4hjbHGCP zfLey@c9;ifZKj%*fQlxtd3IHVm^EIWzQF6E zVmE7=-%t860qV4|yDBY71-$k}AEfpf{@PO2=#Ht|jt#MtPS=!&qkH;j(`Nga!b``( zEyW?HpUNu1`Ps4&B8zsJC91SbBmMi=rzauhsJ5cb;CkvL690 zG5YO&^6#pwF%BA<`4{2zn}7(M~qrO$Hny0A0Ze4l;Y&y7C}9H!TVga9#Bm> zn`~@r;pzR|Z6P-7dTk#e9KWq#05=10^h z|5Yg?^ny7JWf(&;8oI!snuMJeX8?AuyO%5c(t`ApXFI^}!GvCMc~Xwr5x7tCiQO_v z&>Ns@$SiT_FlM1rGs`3k7axU6>y2QUffzZLJ7xi4J)jZ6mJ`&{1+86R_zb`c5^q~b zp1VPVsC&R0e_>t%mfUG&!2J*fAe~vDIQ6>_iepaEdHVpEj?a#h9nx7{bYN&m0%Uwh zeKE;n$I_3@Mf}6(< zv-u9#K#zH!dLRhEiR;l2>}xFPx=D@_JOC^Th71jRv&o6eVB`6 zY_ng0RAQRZ70H3wQk$P{>3!vFOCA-9WhF_o1QxTxIGcUx)L$&~0KS;%nmp)tDqC8X zDx~C5Y{NX2X(DAVm9}?H=x$X)NsOG9iE^6!t z;(AdnF7i!iYTrq*+)qzE8XQKaG=nM9U0y~GZCyXp9a_ic+GX2=30rV*({fG#y5 z6pl(gAGCDAsVgZI6U|85Mu6H#UNb+16CWKfM2fT3Y%mT{f{%&fB2HzYi|@8CCKk2) zH&l<}D^{Y_!)vyHox<26LCXpH;;NQ{(P(@&6o2cOh}mW{arr!B`pjLslIKLJ_g|ie zoWGf0+~dgLb?Ug-Nq+?2s&kY*T28E_=W$*~S41%K47TDPHB-EhK@Fh|G`DsFs9OMm zKE!5x^XhQM_(<{(+&@LV=2cFIvy2s1|B||V3CBbfV#SogS4hRHV4r@&;H!hrCNuc) z=Y4cOVc)inz81oPqc!~-DUHba<|XqiBd_et)D(|XadSCQy%2^6gLsW3EJadC?vyH> zBRmp|T!#i;v!o|%*Mdkd89w^Hu5zDBlRvGR3tUgzo_%>k+ZCjce%* z2-y{sHPhwu@d9np=lzLTij?1MmJuVA=Y@vFo*F99smX144B<0_S(g$0zSI!Z=0wIf zxcDZ-&ZuWY3Ch$kRz+o^O!$zPRK_1(d>SgZs+<6nA@Majqx7uk%qodD0pp^#s8>x z6+h4nBurz6nLY1D_AgSb{1jFE7Sd|TB4oSaXY$* zsd|SpwCu@zzSg(q|7>^Z^Zjn~jS8b<^He$|RLPXW_aLP`ZHFoG{vNAuWnT=#FZsLu zyzLi#Ebrf2h_9brM_*d-OH5v>u#u>b<7nvKux3v$;J2}*8f@<2Lq$-h7C7Ulr6T@j zn^F4`A+t`B8lpL#jAbWRnMz3uo90YpPt!>qsa=|kH%VUB5J~K7Zf#}m&LdZ%z<~sM zGK?mZYm2;)qjnuXdH#V?gSoLPes2$B2pUUr4k)ISehydoO5@fGbfa|R4O&sF8+!Y$ zehv{CjC&<#t!~)qk=Hy_o)2pIOlivd0=D%Hsq&}$pZ&CpZF&+saVilq1zWqc0&s<5 zLLb0wC7`!=V`h2<3z5yT18#6qgo<_(ykaU5Ml{Y1875${B@0An*1pf59Fv!4=EcRX|a7foZy;EN~Qf`v?H~1 z6&WNsRfIxLjL!YgGs2b0Uo9}k6dy3I<s_D;)dt>IUVP2!^Ex+Q|4_Q>sH6v$eR~DdqvDak@+v#PPODR;3B+zz>tZ1fGblZ$#So47ir5UH>0ZCh zUW*zF8$GF02!V=G|ASJp;ue6KR_}oLk04nSbF|nB`Fc+u6m6)4*x_p@n|-RKqNDMB z2?8X}^>cHuh+3*;Pt$Ov9j#fcIS6<-!SOV>dX7FZ@n)#{B|nsx$u*H_Zs6u{G#YjR z51THlCN;aRkL; z$x4~lJ>Z$y$m#A}wk|5ib>HxDs`paCCU|qo1(x9Hn^q%RLy%B$5-7viyRD~*5rX1B zbKD}cC6?CiqClAGP;7@AB}6S`MiUz=D2w(i9}<=MW_BnsPy}xy*(T#p5h=OI$cd80;j4vVJ(ICbHDBCP4 ziWtHH%lo}Wi~$SVG%XlER1wwhxMGCnXnYt+rbfGtIM(`7>zAwI;jfs-o_rZNj$#!; zs+-dtcI+^1T-fln+LtMWGeCTk90eor@SIQ=KZEU`Vq3Z9S_m^Y5VSBzB3K$ms+ZrS zRvwsQewe41Se!ikENv`;#;MOd8NP%F>T2fWz*MdW`wP#X6 z(D(J&MyNFY!Vo~+$cY8rMgX}hEt{7>p?zCA$7v0cn2OT}Xc)H&!D>WGuPz~fGa-bN z+W?s^FiPK)TU%QPTw2E9#t9PTvW@?|7z0#_=YEK(+MyY2k!-k13fBBOOR;SD5aj^}q(je3-AXM- zo>_yGRowDLDeC$Uv$$|heCxdfxDRw@xiUB9j#M8s>rBy#j0?TO*alZ&(^;^w&?ttQO{K&&UTC$FNr}4 zjho{afLA;!-ZPu-cPlI4BaTIhxpXmPx2}rPeW~|1-a?9sJ-3cUAx%QQsZ`>JWs9wn zSESEKA$9rMkB^@0>BYiisJgKD5) z2omTrvaZY9MP26WZtV0PNJb=B`$|L+M3YQ-15m>SYt!zsPC{_&bcZ|$*vx(=ZR!+P zV!@gSC~JP=x7F}{ej`4DK`CKM&S9;WH~}Dou{am^%uHa0WHVi&gQW$V z!}4)SuEI=hkkVGD8+3-Q)`#bJl03ywrVapv`M*%0jY5?KL5{Y%mq){}qX-U{td^O!TG!SSlDRi=@WDAjW z$Ses2x-?~Q;#gnDy~_a0evwOsbcq{$)YVUw`-LfCyG!5L5xquPQdo#R+;9%7#~dP; zB(HGcO;hW#e9s*}>|gQDEA|5xGIyTu?&nyY9%`cilQcDZxkzR~X;WJ%%bNf8f*N;L*_nuC26%OR+8B45jjT>iJ$83nZZM+Kq_2*dEA!63ZINf8k=x8 zBH@!QE%_aU4HqSfti`e!!G(Zjigu4ml#hpI3KwOt92|-NmAm4ec}~jby+pDKY7@IT zk7g@%Oi9Y+8jZf7pGtA}92Ugwa5_4i;VVkSl3kC7)TUsLmX2h*D4-M##y8K*UWG`9}WO)HlWy07Rk~O;NqKStBq$ zty7is0ov2zegloRJ|ER&wg6U#J2gCI$UQo1So0dF5-FhS9me60qx!EfeM!~#JS>Ch zEcAPzWcJgSyCXzs&tGhQx2u2`Mu!cwTy5N&JMC>a;tx7B!>+zuRb%G9#jeru!2nJVVa zt2Bd31mZ&#YldSCfQf+%MVC#cI3JJvuxc?L)9SLcJl$vNwqNlnpq<8>y} z@5vSsmB6>T0(Xv5=Y{Odo6dktN`48&v&axLQ$oUj`?E7{W9q#WB{Wg~E|+ZHp2y{E zZGEzBgj1oRLUA$D{teDjeNtUQHYt)Hoxn6byNee%=u-1xpbke+1Q3ulw>_DhP@a9J zi{ugwZJggg!v)61{Dm~x2PYKgH>NL1%NV}X7y^Jj3~hO59kqD&lXl#nit=sPb_1&R zj#>OgIZ`b=8+T<34k8p&AF*GMas4<uO`;ri~GOzvso zE7Oje4%5`r^I|~=_eEzJ=*=LW(r-IcYSn(G;~Sp53h* zfz;psY%W^ZFtJ0Z`=y)91imv0|7}C=IQD8ogQJulVf7a)3i$2Q`B?(?wI^VneFS|M5%P?dKTHwwd(S2Opvu)+a!9nOg`wrrN0P-H2R#guAw);^ z24J$Bj)7dl95U4>xs<0vJz;1W%_{?Aln`Z?FV#w&^J*%5$0y^?S-j<|VT4&_1(hxB z7<`C3vTquT(4li)7F8FVgjf49-J}yfCl2h?Z)obpiLg(8FCa+}hP_W(48f_@+*cuq3v1byj_~=mA z)Q!v>UPotpvJka$tco_+PC+Mxq1V=hSgsQ37-Hv+Lb@C7-e|`9-1{B0@W_3m2M6DEJCq*GI=sNjXN)Wshz!F&_0}+_3>yO zA_bY7n>62iIb1|yxv3eN-8PzmQDBL#cUYG{irZNyK%d*9xZ}HcY{n0ob0jaHt~$7) zCx~b)K1V9wZ4})lnXp-m?YZKh4Z&v8mT8a16VNU+b?9IK+f(H&lGG!385XXkqH6m& zgycGBmrnbRyh6_>OK_c+6FLcQ$dygF+DU}gxtMwBpi^L(qeN%ES|KaBUaQX#SJA^vaUu0LKaMtxV9C3ae zW*L8-$KRUiA-fm$zerS$(Rwkj^Yh|bB8c%GNIX{*Yefys&iVK@ej>c`A5Bz{X$-4u zkX(QjdOgqSK=f~c#2>#v$hSba3PItF4IB;vZg}!1((~r5Z}s6kOHS<@nGI+>Fl%&% zZ^PjecCk!N$>6L~oDZlv-VGf2_x}PW& zHIGG-#lqBQJO_TwEZ^Zy*_hTpU2E>RGIOJ>X+NwQ|5QbrR|YcOuSqkSrc!H?l6Py8 zf;k`oZ!65N{w5GICFGMQOJa-R$*iX>G8ZMhSL!(9waUE^I`p`Z z35Ip7iD!@Z75DzmM~$xiK6$`#Fjn0vypCt=1c_aQ?r#?dN8^e-$g4VTk@mfn6S#+4 zL9VfLY2(*7>$vV1f^ke;gCmC9?(Z6@+4OJ3=f2k}q1z#~!9b`WuwUVEbzBra!3#_x zo2if#bQ(n>8VwCGG)~NNnh>`hBZ#hB4l;7jA?QX%aqVG+A@{;%Tv+=}e{DMy3CKA( zG&7U__n7hjPBq~(bb5III?3kFch0#^{rY915K*4?P3{~amf+0DIU%^MwWgY$4djNv zZnJIhPof-aNZ#r8Z2hCc5{j4tGToEmWv8~8?L@NsgIpxNtlX?zq?P~hm0tGns{T%_ zKVL*MW0RtO&bIhxhLPqw1o?o>w=IIWzg(qBSO4T{|Boqa6ybypQTy8RC*$>HeVPrs zhPu9@<+;=K%$8>46~`G^FNwVzEA)Pr(K~DZ{^?t<<7yUsCR~9-o?zEuGy^R*X?n$@jh-8VP^q9ZR(9DEjP0F;RuE^`uZZ()! zv}|Z>Xg}8zHzblzW!`NIkk6@BW}%u^7D{seTMIzQx6Te7BLS;YrM|u`ACfDcZ3g&O zWR(^Tk#u&Y5W5Vu?DMB83nTC>O!-8 zB)1`L#!&WX=!m*R^DAe2{L_Bi(hsTzdFkv!tBbj>43Q=5o8(tpf3Xdr{o9%WUgH1z zX;p=2)qys(8s6(8R#W5>Qu|rwdjjIqA=q)Jo{h}3mA*h7LF;;84r-N)&g&Bdk9$6M zySFg~)>jkMK6aL|2@MiI(-lKAP$QH;nWfU((3&&)qGj)nWEw&qv>QHTwCi1Rj+xG% zO@V$bZHvs&IYHeskB}*idQLa2i1u%x+W%QU&_6!~;QKq@H`MZYX-3%6oz!QwTq2=J z=sv=-3ow4vi9;qVQG#?NBA3MQuLW(>Id5iC5>e1BJ+lqxk>2t9>iquOum^ZQNQDT2 zvBmaSRXcI2c>B0d@fJXf-CsnB1OVdgK|leK|E?*i4$eI4Ex^mCIS)HJN^*|_fFc0Y znC3l{F%Js(INLqjP&!N3yA9{LMOmgQ!#>h7dlMKxGbnN2J%D^U&{x=*){T;JX7O`7 z2he>JyTBDh-bamj;m~ko>*idI0fq0YB8WmV*eBWdl1|LX4zgX@ijX_5lSYsR_I7M_BP`>gV8^p&^QCV;jB zNb>rm?B3K@gu@Eo?}XuzkpomaED~@!Fah;Bp=RATrZ?7*FSq&=o0-AgR6JZ%VJu2R zq}Jbf(6qw#vVwS7}7i^kq?d*P?uiGOMC7E%$bt)g!Z)`g>Bm-J_X?}q~Y9H1}6m^&_D?xG6eBw zzxDY7&HCSgWu&JUwbAxS@woFGs6=!xSpiHp?>p_v<|a3g{?=Oz4a{eR2m!ILOT!gx zQJp7A5N{ID^qb@ZD7TX7r9(s8tv8o!8wM_m_rRuUsFWN0oaGNLfG7d42|EKn4TOZb zX|me!4EURLsg(w7_+D(7V=97tiX;73SahZbdC4oqVwc_H3^oT@ zxX?5G-^#}`<*#Kf-Tc!yq=wMwQa72kJ1tX}Id$gBYy*>?>NM#B2)G)5k=J<*u)R{K zKLOj#WdUKw$?5i!ZHA)Gg^k`Cf`Qdarcgr8gNkX9HAk&rEpzs6h9UMS6}ydD3@fTD z_{Li1s9M-`5`MF>AhTFFS3uR?&-o?&p(0PNbb(y&RJP_z2J_&a43N+v5dgD>(urQ6 z`DS+NqgLB-OEfCpgEWpRUc%zakOQn2`Nx9CRmun@Jy(Fe-8khcc$n({QPXdqFcrb~ z1=#kb#2HFzja+i3a|H%}iTXD)`fuL3gcjc4kh)2$-K-s+e$|L(#M%Ig6n%g~jTl_; zF2BA|DJgyKFjgPx3h=)o4)huIGx3DL`k>p%+GgcTG*5`TBO?{LRX(qOo%|FD_#1o} z#SzI~7)}viOd%JTV$iWu3!D6xx{kc#{v7lkGwlm0y_yoLsHT_oFxtNc9i5p@M733R z*@Yy;?Mg%RTDa9D3%|cMa49$&#YE@w7#auYQ{k|B-<5*CxOmnWyaOO~B0ItG;7?E_ z7B$4?pC63rUk~=i`R7*Tr#hk_yhCUoGql~j{lr-N>n<_7!3notGb0R6`$67m`eHRz za?rVSi~n~VE-0-n#_P|8cV|nl_{^NI`%|d^avYOOaIu;R#u9?4O> zH-)kSU)?16(+ZZk1DQzGj?@DjIz1#|?;Ma7G7PgsK8DbZ>JSuIMK8FEti&J}eLF<) zdyb6s>Iqn=$AQ%Ot|{A4EFn9HwevTctd6WS@u@+1pS+pBgFv6$ zX%+Niv4FKH2{6$4F3qnGeCVC|vv_DlxDn2MmDbn|m~I26filWnf=fWyv*07|x8J>4 z#84zm#S45V{&5XZ{A;K_)$^&dNCbu-0a&KM_ZEh?)jC1|4BJ|x&Qn6t-w}N49M|@1 zn;{o91NG4_&R;#e5n|GY=Yr=+5&PYCptXPvr#CD9_lU{k2-H^G2;w|WO+#Vi8~*dz zl}A2W69CgHBXQ>!{;c@D(B>ICr{kx`0w_oE${>_~-~d8zx`_q_uwU%vS@M1IFE+vE&zGMXH}UTF%yn$In>sr>NyaCvarUY_SG4SU7w zrK+oE7OSV_q#@hP4kjD3RiZD7f_9}b{Qb#4Qr>%OJj*WUCS2eC^v&!2oBkN_0sa{` z^z&#$%C9G{PMcN0@bNc|u;%5V4WrrfLQDeX_ zY3lmuwn&+EWV2XBCYOU9>Qc6AcLBb^d`pp0X5Jh@ReOa@Ncpi6w)8(lrinbLrc zv)Iw9hdQT2wR04`ak&bJ~KO|AKX$f7DC+zug0Bgu~&xqvh75&>iH|%@oi2D|ipIowb zEkC~M^nX1<`2|D*V&64n^Ht<>?D?Tf<`DiGxhiJK4}!f+Qtn+5qsM=bM%+*22!ttt zr%?PB_U}8FCv7?)Dv)6FC_?$bgbJ-z&Zb?6ZDwg<2cpXDNlnUAeAY6ZSa0kT#cNoIvqAjv3{c4YYf4tdUcit`t8=S3tDF@l?)rFpM7as6anP~8NOe?i$5xcyi_PgjOS*tW~eA58Ed5+h&ljW@Y@8|}CCZEk-QMq_GCU76^-cD)RtfsvAtw7Nq4gN>J8Zdl&ZGo%o}OLccOJwj)=C@%143*2|+a3yPnF80oP{$Pt(y z$$taD*?NZG(3X~kv^H@jZWvYZ1x&%Zv{QcK-+F4*h?tbU-3L$?2yO7Ng7cG)VL-qH z4>?Rd@G6wglAvX)hbdC%nsDA;Ok?--E;XFh7pcz>&H70D(LrIA?^*V|FRk|)>nBp2sh|AsM2b1eI!Q`K-<%17U%O|^0k4u?|~jG&v5!*?fd4P37j}# zw6v1AB=vf2ZPS}G)oafGlvxdnd zYYpZfJlBrM&Wq#lfknvNKg>g_;-RFq#gG=0$J-;#qzfMh28__7SYPq_V6ZSq2w5^{g?(Xih*8QBB^MCR4d39bn&cLu^t?#<7Pqshj^O2Jz%+A;Xy5HPT+0&r# zY(wXs?5JeMS;59G;qRJpNXJPgy)wIBvdO$V_tFT~j!_`&m=lfOnKP8HttbxGepD=EVQs z4i3N*R;U-Xl14$+?hT0_EB7x6OI)RNJ>^9BPr+2QiTMX@W#ovTmM`H)&L;{p?P$a8 z@%%?E$ou_>K+=7r^Aa2-$;k&im6u!dagN$W6zmv$wGSnZ^>7<&a|PzNdG7#)c$OTJ zeO=$j!C9U;!8SI|_Kefa^~L96{suR2MTaSSGIMuiD`o;uKUu%L33c?7i)$PCjgazn z1P&3DFjQW?Xd)@@PVS)_buzA4Z;b>|oq909m2!S#RnvEFO1ybHS4zbaN-8pAN=o?I zyuVmfD#7a?S8WM+)yfK(y~P~(qfg`d^$%`ue7V~NXWy>DBP^@1sLF=>o6sFs8JXST zD{CrXZO2SnIaP^(eORhex!jlpwpY3%FXBIn&rDB$1|7TqF1%8KEQ@XX!pZwfW~WYF zbf30{F^?nFB6&?zq;}d%rPJ1_l`G#+0+q z2&xwnb`BzNJ(iH?Cq@qVd%LZpuc>NVTiM9(uAU?VAIuGT7~dw}knCX&zDQ!hQ_35V zBZtS*&iqDh05nr5?{kiuq5zBA9tuH>Te#a9Cxae=MG73Skx(!&{qmF_ zEQkOnXyLcQ1N1zs>W+QS@sUnuW$ z{70`@=mV==+;T$fWYutt@p84vU0*U!W7z)!`++lmFBfaiA-IEoKYP-0R-WgKd>F9t z9CHA!N(6A{X40+vSTj(aQk;)8O~V(Uj%5C6F%@*m4#j?W&U^Rr(8ks&$!PnDks~nf zmayIQ`nE*4OMpu5I`=1q0z}&&ZGmd1B&ZRksBJ)(BFe!iV5I^x_qamJD^mYv^@X)gCb$d(nd;<(4L{;b2UL` zDUHY#s{V|TBVf4nhiK4yD3Sed3^N=Rm=yiF>vbvyDoBGe;LO2B0h%ADif2EyCza#6 zcy|rhI=^xGTD#fC{?Y2zrq@<)0Ev7m;`~N-28=QuFtw5>Mqg3+p0se5BZnW^|KgI^A?6b=VK_KYZ`ID5F>jiR+P&duc4h{I$K{Ea{lPzAmoWD;l*oH*SO2tC* zhX;M1teLO83LDTC2&op^BUA4GBm@3C6sz}(U4A1BRS(zqAhle$=Ek$h zl{yceRUeSYZ7aSYcT`#W6^=00#BWdDUA<`2=?2w(FJgEKzz7;u>p%T*b~ZJ)s0uhD z6M_maS-21K-y8dG7M=Lc3qKk=V5CxC1DMV>Dgo}&kwgy>ujV(@)BIG!U_-CK zQjfRok3W%)MLaB|J~y3zR8PiO2P*&XN&kPa*Je}b58XK-w(pghwufPOQoiTA3^ttr z^AX0=Pp>6wq&WSUWYbk#;*8KB{tUr-kjLd5!bJ5}ZJt?tVeU^w)p+iRg2@wWbIYXl z3s{(!ycFe~DGsL6uAVf|KbFHD1s8PqA1IV&Gay zshA&Uu&$|233{X^dBzK=e5q(311E@C9{cPA{_%=MOrRw|$U@4i{mT7p>{x`zQEvDB z7VNjM_?zmji&^ejQrcPWM+AuCca}_q_oSm0&gJ8Cdj%e1<4vx7VBd$fsv{Kq_HTyIZ6+L^8d@NQ-vSA|J$Y_?DMQ+ zyYu}1uSS8)B>MLgO{DR2e`fy(JCO=F$&jkqT-Dfg)!22P)^VS1xAOk=Dd;nf4>m$f z%W+J48HHgCB@NS)b9=K!OV_vs>!V_A_|OjNI>)R&Ye|;p_0_|H;heKLm7@~5N)6R93Hx?Fp+p$|+xDEgW;R8U=YPiY`;E{jsB7iLf$($cewg-G?6Gizh!rd4Zt-)&~xpar~b$_tK9 ztkEMy2lJmw-Si{ovl9c&V46Syeuk;T8ad8TP=Tt|Uj46$A3nPS4H5An<)~5b>Z}1g`L~USnlS;Il zoXcKpY)gA2zBjp)8S~8OI5Q93ImPU?a4$?n06@OW_dRK=)DQnJKh@R}BUAm1tNkB$4+$?l7bjiRrxlg7 zxBo+|^8YX(M5sAH5klk(od3;@w70ze?#uN3B|*eZq1*qSu1HX;Ie|aJ7SM11uOA2x z#!?7gi+BBNHt-L?ma{a__dZAR1Q_1$nr-Yb&UJ|@BCu-K=&b;_>G9Tk@trH?HSjca z5pU-*=ssh*{I%J<)v4paXKjfXQiU;vWvl^9N@R1~h(Ylb<1@H{3`~@q+d91c*Jon@ zg*G#Nzx!i%_ow-Owz(?1W}17Bcx(1G+KXPrlDG9J?fW}vy6A4Lj0LQPU-p{>%RUWf zFKee{2s7}ZT>%F^y_^~x142r$bX0dU>3y%c{Wi(8YbInmPQr((LT7m?R8{#=xA|N* ziONynUu$Iq=TXwF4Edu8Y4sU^h&5Y8%2L!qUM)ckn zAR7V>;+iwR{vb;DF!|mTeo!@L2vQyQ_k_CN0&w{)0FX+&+O!7jWq>*F97J9tt*bO7 zeavnFAnpcmJQ8GrEhlr_Ny_aO|G4g&{o@r1J;x?OK4~CxG*M zPYyrd1e@ppoZF=tsDbvl1nj1(-6ro#~GL0akF*7f}cvMOGjC=3fU`0otQ#*?+SJj!o3J++nB<@cRp6-K(UmR$c&2zwe0-`yF>I=z%I1rJ`x}mC3 zfPUcvDJECbUzPsxM=0w*nO>gu9N?9m`~-BOxelN+7wM5%*Qn`x0I-A zT)lmTAa z!A|di1UX3||7-S8_)mH;F)8e3<3FEoU!Xg`gTlb;J83_~X{BMpQDk?UlJl`9#c9@6 zEnxr1hRhvFw7q&_S#4*Tk9KOIK2yv9Y4=YU;6W7tb5p<%%FQgl=r&=T=DM~wn{+Ei+roASP{N*emtdI3QfxMe$Mc@y{Eb?3dJ-E^xOI*} z*8mS-BgJu-l)Z0cb8&UoXI}4r^GyKlE5@S6?i!n)Eh)}6dM|}aWw_`4z3Y}%OJ?fNz@j(cWFE5`sdAw#$eGe>+=Qf3^d(} z$UFA~)Plzoj>qz&AHV-*cdsN4%yCNaNnSY$v;mXO|J7~&fB#pi{GZ;YsFyn)s%gD@ zz=vsvYQs?CH{3L%q3tirX+><)Tgp~xNRF;uL}l0&XE^KV>3)I*s)Aj!&2`UU6<*<& z0J9Vkg@~=2%6)T ze{rC%*0$gX0}g_SO#ruQR@;aqDFE9ZEuV=qa)aCBPi@u&Q2x3WR1c_2 zz|#M7Lv=4+r!4kFrlDlTJ}TXR`-g^Pis!|Y#>f=Qz=NO&A!axCxUskzydXfV&iS@_ z0%5;^<#oIziPz<=&8*`F)>{p$dM|*hT>;Kv{%oz(2j0oL!FBGZ<+&oZK;ME}FjI}{ z$pg*?7yuGOHK#9jQQONm<;?@JW;^<^T~VK?W8 zLHor3fmg1pr$A`8vlT|8J4=IEjPeWE-5Q{EU>p*h1Mr1LfE`C=&8Vk^73DF}N7eNJ z*)f1OrkU^+P+?lrbY3JxnW^=2FC&ZkEPVU^2XMwAt)}${Z=tY>)v6`0JO?>-K{@1W z;A*=8STo^xij}t>9W><2h7=Ob74BUE3*UpkRH5K#ZXnD6NMC2)#{)+$@cC?2w!k3H z_*B!JH9BSESWddB%SIui$fbf||382hEO z@oU?u*-#WSsZwU7Gm*+5nI!Eblay0y-Xd7(HmTvem6ioDm>&c~*QIToRuH38QI-)E z!d3xk2yJ`@Jh9C=G|j#GN&Mb+td-s>X)MOxbz}Of^5dDZVo!%B5!lQK#6pqz^#qL~ z3-n^cO+g;vL^%?kCuD;c6*NHoB&a*Rd9v{!TW*we9%uKs6qSe3O{MVoJ?V8}GMFWrUsOfJM+_d;4Q{ZZw$q75sX=qL$DIG198IAm z^HN4b9)O3YzAb2>x=yGWP2V~R0NO$dqziTBfw3zBY~D4{-L)2a?gRugx!>x$&xVoJ zW-?S>E-=L+n9jBoM7W~Kr9WOpJWkyjZQE>wW@u`-jwLP7qSYk0qM-#H!Y1pN{-c4` znKcoMwIYN3HXeDq_;BbNf@~YMKB#>|Ukal_F+nO=uz=UAgmjDeq$al?jS_vx2QWLg z0QxP9_h&#Jd9O|}Y`h?cLqd7Qb&~~Gp9NE6nz{y>{AT%u&--BrQm4IIeeVP-i_3~@ zFovD}Dr~??pCH}_<8wnx=0HQLXb(F12gx-g=8X(16V;;)25#(t?z z&G`kC;!K-Tfs3^k*(fAdOnP?AuRnAgNAtuKT#Vo`8CL7ipO+Oao>gN|N@tpR2A5z`v0sqHnC_LDLH$8gzS(aSz_-DwxB!d zQtIl-(4GJ0+-dYZVYl}+CV}^lOLsG|@JF8hTr4}Dz*&r8Y*}D4!c~5p1-1-GojR*I z!=mzzYX0XfA&2&7Qy*zzu3>+kYqAAB0J&(P<4=YQ6kD9mf$2-Lun;|CFMj#pZG(jy z<=3g<4E*QPIyl&x+ZGTF&Y3u-#2nX8M&^7FC{ZV;2woBwF0{TkZd~N9UBuNMqlM$j zHAdFRgVa=tzGJF5`P;vy7&B0ehu=)RDi-U!1RF{fJPuZ_w7HnA)FoPMnrWK>wu~D)bC&xHWHSjR%svKX z78k&r4Rv>AdR=oGbPRd#{>AQq^UVG!ZUBgp7j~Os%wvlYt%D_t01-~pEB|pvbzyVo z1<^vh2e4(K4nLJf)>D>UDx$5)td8Zx%I?*RPVzG3MP!k6Y&Riy%>E&oU?zl3;3}th zmUBW$Bx=(mFN2$aM*D*dS1)}(}82Y+u88HW9Wo!lS2VGCA8F zG9v%F7yqB3MZ+X*PTv#5Dy>0tVlLJDlsu?yyTBY>S{RVp2o6eIWJI6qzPY%A;GbUM zmM()}y3ie{uz-~$c^-!vSO+b7HK{&nOgFd)gna2$OO48j9TfydKNE8OLmXlfSh=a3 zZt#c{EM9259H&+5oWaO0=xYjk+ zYBrh;+<)AyTJLIlYT8Rkj%OyDCT8Dl1^yzQ?^B#3IRV!2NwCZo3uwGc0ySpm{xI_Eu?LzVkJ@>_jk7UhoJ^mpXcw$& z?&R=tpd&ndz1$lyv({>K{ajpG=SJc)80#`lp%~zq#t^mb=kx9Rjot4uqhmgwXdughn?gI)! zcIbDQ9FHX4z{asfnlSase!lFG%e33){Z1lMvao=@-Pen$cknq^po~!E(|E{QoTzNh zWVAM6I0au8YbB%UuAyTrK_gXk%;T!}9j-E!8(SYJI1&cG2$3}^Tqao`>(Ee+%!{_GKMq3{t9Q>g238E@gFxHPQ3dV>Et2>iFEy(YG70jRD_wBh~N^#tB57X{oc z+~aRq&V)lnuA@cj0Cj6)jmGRnYR$z_k_nQd10l`CJi!i*S8VlS)7#mmdHHwaykHdI zg;I3&?hSaxdH{FGid2>!hT*1xVr#wWX@9H$CFS-&NJeG)m1>XZkmVbRaYUKwGZ2Jv zk|B_fNz|6;cFe?R;qx8+P#Re<#5W7gAZFZD&yx#Ir3lUcg+ifuQ0{PvwpoPEwqEKH zhk+oWRM~$30Kv5uwf~gWs~Pp1!Vnp*JR?uZ z6v6}W7;?#1OX(dVxg_Y!X^A1fOD@$`zI0^`q}R-6E4S2=J2wrCR2F~9ot38|_s&kG zFTaoy16D9*9AINa8YrvAPL^n|1^XO z)fD+PLoWI+e=%*w8Nk>WJQ>DyXiQ+P*!p&52-wlY))ab&wyIGV z^uUGyVRNE#S>2z65I&wy#VJeY#yBV6zs+`Vo$FN;PQ+Ti%oGWkFJW)fofmgDBB8IG zA`17-HfRIE;J0L62^>Mq!eYX0gk_l(TzY<6Js4|8EG&uHkgA6xVrVErL*Dd?jnmH8 zo|m>m=CeH$i&BXnwuN2M@FyV2~0Aa~p@7)?N%qYEP z4_myGu^`W4$A=Q2^ayUefg+9QD8fHpQ);Yb6&4r$-tIc_$!LwSV?nqA=1_nmrtj_5 zu(-|)zXu9mE^TE-)lsG;Us=)Qn$HC~hq5PHn;42YhO}%x{q~}jq!@YQ%n;`@RtO_U)488ZQ3S( zRizw>CWw#m@MD6V0QkF_AC|=qc6KAoud1uPc%_Xdz)hyK-z9KSO~_6`CtMW8YTM2^ z_eSWvxnxiMlmW-Y$MElAn4(U#PqZ($L*}0FM76=um-ST&Flo8N=Zez_ z+bzDWPPOYO2>$6qgig@U4&_QdO5Z-xMH}Y6*VIw!WoE#_=pMQ*zWZ-c@_L2!a#fC& z-I86Doxg0ls^?2Dpw+bGoTxz-dRw{f4a`T`Y?A2`ks;TPNgwaQh$$Ufz0vmpK)R$QJFp5PC zmUdeVo#|97zZ{pIn2KiV%|Nsn;)OI~`>zWZ!!P?vx~4Z1vo5g2PM>uF)d8+t9%<`PX5|x|+FqKAd`0>zs4y4^j}TMm@r^XlG4& z_bd*`&^0aZyIuUsOs|OK!z5@W1uzO}&@nZVOz5&k2{a{HAI7R5En@WRmV5DZeIz{@ zS?Y--C6MS1P!w5HK-VdS8291LmogPIUWRJqrz9)MUthXZ8qhZ1(iU~%`Ih~`8Q|}D zQOtshhAlifr>6K3ezz8==2I{IcZF>ul1Gzb^ro7sb5o0B@FY}DlTk4JD!SdxJ* z5o6mx30L)ex>h-Q>tiO`$k?<4m#gr zUYe(d#Ye711zbM3XBnln*KByZLb=Wzq$+RjtZt}x_gBxN#xUMrk7=FVD84$w+(ngg zqV&$#ECr&T@b8(QDvWt%i6u)F`@g4IO7^^$4;O8x5lLtyb6_*OU8ycolBgxfDZs8v zZ&F8o*1zMQJtClRA3V~YyI7B52V3>lb(~E*+aBgMT#GDpp&o(g$+ksT5t+vg!M;#4 z=og?@ys+&-m@9tE{glg58%aGNmx+K|iM zoX8vhV$I`KU7NBDiVFx)FE*qf^kugu_dBwC*K-(+C+mf;e{+ryaKCOo@CdhfaM=C5 znA=u8ecg&z>wliJ6 zo+JE}14&Jw-OSg5Ej(>)_v-R4_!@xz4MTRqr9}CC#{p4y1X4Y zrA6g`&8l$%vw3B3qEeMH6-RZ_*ZSJg2kRUVXe#%{&5a1=pTsD8GtDO4KKUYTBj+Ca zlmS)qxOwm>{A*{i-6_$mwHZ!BdV5#3zl?Ibt{g0}F?Ox^E=K(^f{kwEc9;EyM#=?( zBXSPfm=#=A(E`0esV+z7P)Wii&d&m&Z&RsNT2vbb^Nn};hV>TjT=z_xvZKkge7&;l zPpsjix9zgVyn~)va|R`2T9q*lkA&UvqD9G%emN5Qx@flPy1%WxR0I`;{CQ8sCK%t6 z#IEwY2MeCGy%wb=)D1+OvI&rd4HX@p_we=W{ucUv%#dr2S@M3o?ts2?@pS zRce?Twn|Cxq`maTj>afBs+%888W0?U3wNHsPSf4s8?k%6XaMJ`iEBEjBr2$WC!aaV zQ}uOc&i|W4{6mQ;*(BNQm8bW09Hrqs|4;a6HFfXFqKea6i9A;#Ey3_YI+e_y9$C)W z(uZzkRmpAHcQjp8*vpKvriZBee*U^F`4RSxn~(hb@NgvwzR$c!6^ar+w=1~T7Y3X| zYaEGGYzNKufhL8AvDAx7fXlu@rnnL%rGMz>A(k;Uii^zyH{Adz(J#U$rt-CKjjiG+ zmO!Ce5IO8uUNUQc3H1@HgSo)?BXijW-t3fnBORp#ZEL3`JzUv`x>3AC}WP zf#d0eGd;JiGxf4|uM&G-3_cCYG12+?siWFZX^E}QDQC6*9%@_v4TYO_cffLFoK=^E z?VFa=Vwo9`Mx*AdjUo3bB&tZQIFB#`>p$h20T1$Q6CU|k;%>NAB<`olUQnG0p6Plp zc0jU8?z-f=n?c0jBs;Uai3$gsLqH~)miHNn@IH_om(ZtYT7t$O^~yQ_ik61A{y$s; z0J-be=r}CC2=9Pp5MWZlHIT%84|*nqk686~jokrc;S{u`0SajR0g#{vrO;q*%PkcC zAI%M5Eh8WL>ZSl0>981D@Mzbj?SvL7G`RsFDsF+_>@m~OtZgGy0)iGaUN7e>XndW& znsdqNb#^#-G}J%xBTfcb^(KHHGu{CJ6L$at`rLwurU!J3Zb72mf;K=&E5HNv{)ejH z&P*zS@!0FEW_3WJtXmsFDreA*$AJLLygGe&4%4&_v>f>+0s!damT39L5(Xtp*9OZp zYFIRzH9XB6JECrlZUFLbv3UIWHxQl53?qxkjJsw$>GCTBv6by5G`^^zR+F^W>_OP9ndONonXJ!$f(VvBTdGZJfK)v^f zuE6e*q|p5^-49xJ)dCkR2Y{-Qki7Oc2#D?KE+(FbFVXyT1GEX7%1g!!0tvWJ$_g`1 zy)|!Wo*AWamGS3lxGM$U_ch=3RTY0>YomBPrg*$gdvvqvLWWH`%&}a11E7&tJK2KI z0!Y+!2^~1RatJE77dT&JDm!_7c6|#9WARV#hqGe|*X10?(w{T|+6;RXPlRhsM^Aws zStZBnij)Dfn>?j2%oU?kx|>5v(TkySm@%abJZqp6mYrC$u2=mc<1w>?@FWf3y)ou5 z86|>-v?rIjt;dV8ny9XfHiVGDrUIr#;B7HQ=`SG>^8)cyq^!EnubFRwQkgmRGr(@- z`=*1i=aK~t_R4lc33aj6`M0rV{=>5mUcj~Yqs=(<2>32`?xj9A?E|qMz-f16WdcoO z`Cj|8_NH=UO>v-PP>PtBxK)^m;>ZW>>S0Z8tzsYOPRRSnPGT^MT zEB>yc(_>8e;o3q4R-~XXy?q^l-kv;(OweBpi61scsmM8b#xdM16l-H^mg@a;bP%LvR6ZH8uIfs%|G+4#mLS%d_ z_)u)d_H}g9+wBg9YeTTTe9nuEF_G&JV3?X5Q-Jd4MHL;KM}RIo-t*#PDww}) zof2NgPzDgSrdGqQGrdjzA0->feI2X}*_HVItV1g~92A~+##NbY?fs{s`e)7^V16vVzf|rKU&0S_XCiu_jT~g{yQ`W6@M7K*cw@>@l`0vLTgsj+pHdN^NV* zCNlMzpvxAi2_RZCc&aSSwZ8i4v?YElp7;RuRKs!!9Lgovl835CGmBAf3`%&%UPysL zk^PWu->U8avA;Q}K6Tj%y>E=IhN$cZowCv7Z7p@h$!ruyY*60@EFwItWH}zhs`@At zip%L)WS1%3f>wzKExmVlW$~(5FaSKv(8!il?Ew04^5W78gO-iQ>LAZ8_W^GV6=q|c zLY#lcRHCIu8Ve^*6};!13UP`k(Uhu3UKTD^r6o0z#D?ux5TZ-y(ImmFHQQe6)mE4p zJ&R-D6%s2voq_{Lnx?NxFu*`Jwsd==VEQW-wPn;!-!L3NpX*tJ^4@Em)Q={uTXtzs zdTyZP?NHSyAet-EMb_+k9Zxo9o+^f^+dX6_MR};4-CbQ6HFxDse27WxRSr(bxb9#O zP7kWY!)^#1-?#O0@Srku?TgK`<}+45i`mu6`pcp!2;7A?!2M@U;SMk*xiLQS7P+1Y zg@vi7$(xlfUo1QUf{_M-fQxX=#BvD{0ty;Fyw4o+Pc3|zr*Qhi`}zkT7Hjcz#wLR& zeUXV;j%*K#qssSvRchKH&LcP#*=Rf)c5X&@Y1eo4zFgDKe>R-l{1g>`4faQD3*uQzey{Cs2V3ZH_v-yVdZiCHmT<)t_p5V)_JI z+O8Hm!IGed_R{ff`C#=do8K#0%Q@wYNuJ*jZ{K~%B%AMkM9~Lmhu0l826^k0Ps7sk z9I+7)L=j{q#Xh{7igJP}NxsewVlt9z@5D-jGXixD4cmsb)GeR85wI}jY@JQm1LD&9 z19dA}OwZr?96UW1kuhEze5GB(%&6eAo_q~P>n%1*@$8Bu0$Bs6HlfF-X}9Q_)QF#U zy-D7G2Aq$SLo6%aOfBsl!TP6oMhh2*sy&(7dE;!zla!|hjThF(}bh-aLp7%(**`hvq2+0jXZR5s465DYnR z=r~z|_)jMHHw|;*tpE*FRs3zB;>gtpY!GVtDFc1NJZEYr*QZ|ajK^1597^IS;V93I zVu~4Unh*eaf$B^Z-XkBk(NK`Ux#WYTmc8b006ze^N5HD)Q42Ga3zP zb3G?FOS=r6<+a+{T-#A}4kLt0K+9NMFkYuuuyj554+i3B3=&G~xD~8u?jkj*vGX#tW3Bo3i-sa_A z>~d@;ix>=T;l%Vmv@dv_WJg^Z6sqnr0Pfz`tk1r_;;~k5*eb)x3S&o)y@98GdZ6$Z ziqW@N!w>DJ67*R?hDuGt7aJiCfE*~}C1Wf9v^oC-3XH8F?w#!WoNIs<*L4Z~o~|CRlmT{sYRAy&BHl{?;bzuusZ#EnK}4*uCX5oHiW0H%bDE4xwG-XW3@Or?E_ z=mN}{(~7dy-;Z8P*ncCz-?^r8d0XEt#qOt>CaE=`%y)z(kr}kvAgGNPBP(-%Q_qgV zY0~)Bcb;#hxs~$Xu6Z`|#KM_}4NK&q$=HYiOJO@Al)#U%Ow1s4hXxqM9_^kyA%z#W zJcv$g?(s_`uD@ZpdmcKRUuU9UD;P~CjxX*QzkktV^UA)OjY6wW9VS6fO6%$U%6$4; zR2r6{!aOjX(A8=PX!`2DRa`(%*!rPPS{>dLOIrMn$w{eJuI}V@XA+es0x4U~ndy3A z?%d~i)WYg{b26U=vy2L>`Rwt(pjLKa?!&J4b3eAAvt-VgEQ5ld1_nJJ&iYdU`wvfS zY&Muveww>a+ZrK%@OWeJ*e5l*2fp}2C70g>2)Hk~SOIM#!4L~uH1~$lc4!9|NhlI>l8T(}-nV zcVE55{S>{1-qtUX@08VyVDxZRKkeXbjA-lpuQ2)?^qiCAx|b(VXYD5l`Xq_Pbpe62 z$ZNhf^@>Qw6jF+)NztFV&J3%4}45KiEKDE?)MSJ zE;v1~BYq?`${IYrkpti4F=0P*z6DrH7Lr}V>q!y zjdzea=)zv^Oj{PZj#-^Wy1mIhqvuI;6GqzJm&F1)0}~M59KW#sG}Q6wEx7XR^D_#e zt}J+^-`mAM4)H!px|8C$hvgoPQgUI5N2+tJ2@ zvNUH<{77c!`w-OSjlKTt^r0EOCGP<9DW{O8Ky}70-LstJAE{31=WaR5Um2lDbX0#j zGN{ZDQtU(U6+Rg(d$){zlnk!k!BNpZdi;V=wu~|@z4BG2FOjRX!nGqLlW0e@~TwAm>4*L zaP;mcg#J?da8lf)^q%Z4Z|$-S42ejQ#%Ur|QbgdX;qBBxL<909Dz zA+aRCG~KC+m_KvvG};2<71&`FutDE9tQuAfKN|G& zq2c9B>8GfrpZxW<5i3SpP{hlJ`?0Sp6@Ra@*~u!)BEOqBspF&NNV_|2uQ2}bIx9uC ztOcJDBV+5dS@38mWc2Y(JrAc7m61qe{+i9cj{#NH zoPn@5#*ZetN_IRo{4D`ltyo>+ zW2nGN!;e5(fwGz&tSp(T<>btO^yzf-sX#Lg3Lr5wv)&JK`)Lt|a-BpsOdv^XZbfLZ z!$cg^%$7Ct4*2v*U(!3YB?X z>v1H?Jp3WMIjD}` zV@~9wpIP&J5TYi&TNRjPF@94{jj-k|;5_@?FLx5XN04l%YVJl{!d7Feyokf-FXA%< z$5a9?LkonzS>~LDmj-SJ&zI&xKgJW>gm3xMm*2uT`{S_oWUrbQjw9hc#fgyKr>e*W zn_U^yhD^SvV+qmvdN7SCiMOq+PJZO5Eq_NN+s7gY+rRFwGcPW z0!_sSMSRoA3%!a0bJor~k)GlCn6fb}Xv7$8SH+J-VUU--qaHjk*5%Bid&W+SVu8 zc-?$*_`n%C{K|e<>#wE zi7iGpiC9XMW1SNC(3dX!1+^&)A6&;b5Gp@z>kftsF|feMp^{8CezBt@JDnMb(YkV6 z>3}x8I&Nr~HS24SVsI*9HtJ86iF&#BL}gkmL7YHxM?v!N<))}P28x!CFHDr*DT7{|#^KEj|CW!XXVspM|a$?m))M_1r{C)HKGx}Ly>|A`( zIM<|_qZTU{_P2XMG2d+;WSpM?LsTYZXK{I)R4giI5K|^6^=d&Lx}beS@8uw_G^^`` z)8R>EB?UeYIt7DOh9MMsQjv_W1+6Vs80T~d@9gV?$Yi$fOC&HNJDDj+hdQ-Nbj5+J zF17bCPCWy~gh;be(Ajj{k#T5^Z*Jjugq24X)U>qyR)&pbTP%cd`OJ44Ip!Vwzf=a5 z!Ioj^aCXVU%( zi|sWnBipT4o@|d-KSNN&yNFGIya0UXhel`#b<#iQUGl%}f5O)iOMHAl8=q7oR-<4p z{?>3^BBI=%x~*V#!nEsaHe{{`APDCUed%Xh0?@BPeDC;W-l~^Y8U2#*6z% zTN!@qI?tQzNX)Zt-sdv1Y#WN&irwyEmVAK}-E_%c;@|Kzxv}*&;G#}Dle$>zCK^|I>8=t%c3*w+y z*ypC_5QHtq%<{)yd_d?zql#bqy1{c{Hp#$Lg2(J=WoyQxH%!mFaU zh}I!K$7E|A4pk0nxsN=0P47mrb0Cdi3c#FAaZC2@oP<-W6L6d~Gdiq$&F0fEA6597 z60YC6Pq6NTX9$Ovak+Guw^uEld5S#bm2~4-@(^T@Kzd*?u|5#=?Qd2`w_Y}gn<154 zJ8ubo*0?Hhl|RZxEKtOdEL|;si7A$8r=)~iawR5EE#c>|6*&?rvmI{55D~c9b!?M> zvYFeyx#lt>!C#+<#J$@Q#}*VWzxeA;nZ5;(25L8MJ!F`))jf(EuQYt2uOv)2`*kf( zC66)7iPa-DJvx?&$Kd&d2XhFyk;O;hPE4Cg=`QSQM$%4-6J`7b_M4-L6>{rC2g;IF z@d@7JzsT{Yy<%nhNR3v03#8eWWlA4X@6XSnc+7%q%Bu}wYxpdc%>y_I^1dC(H6C&C zF|^+(ym{j!@_yL-z}qOcV<(Mz4rr?RHmzNX6HkgpwD&a(8Jr$Xp3(?SVoS#r_~Y<* z*M}^JN^$O@oaS1xfgONnZ4OF|>Jkia|8TVO<`NQ7UIzSC!>(o*v_v#Dd~=Kkgmu#| zN*@Gu-*Szt2Xw*xI|&~%T-j5KB6LH$h)Qu_xz`Q37&~ zOz-+cguK&DsIjfb>hpC|Vs8Xirk`j;!yngTa5o*_h0jFVY=7A z_v#{)V@qtjP^BpyWgMwuCl)q(iJo;#)U+z)EcucnS4fv+v$Pq{SK+w1fL>RPQwdi^ z4?B7JW8C*oa$F`4J<49A=e;U_xCPrC5$p}ViRco0Ki2Gd5X{iEspIw6bx~yrmYGSh ze^X*eP3bQ`zg9ZpL5G3~gkzh$xa_IIZ4cfT2PR$NSvsq{o`a!#ZFViSbbI>@P&jl2 zQ`~6y6DpfIMURZa3;P#S2i!a4M+jJK$~Szy5pCD|cbaTbjV^@qFK0{~xKLGCG(2|k zzkg?8PpHFGbDDZfawp85NI6ja3Qh0kU7$hk$$47RjO{?X4LhFkS&Z_XwId;8)?)VO z%R2ws9V7?;nl0Nf9YsEhImx!4@rSBAmY+!A+Z8v*Op}li^$zEcl`5N@V!iu?xbx3p z-)8yM&V1||4f%(!723%@DPZEzhkL;ak7bp4ROZe5Nk(C@_>GoOlF^PLGf8H0%kRwk zWnbgoSn8lTiCE&TS}2-!8JZpnEgk8Yq3%5kd~g(GK|BfN( zK!J8n<;=y7pV|uXsOYP-(bW;P%Fr7~wCx{KED)hu^$v9D|MmmxXbITiACXxXzE zeeaglGLUKL{dv^1WMPZl$5zIHj`~7de$VXd1K3jhnHmYI8txBem%OLQ5jhR{Y>P1H zu5bOmFYdutNtdriro6u6&#nFta&-Lg{=^e(;1^?HouSQJ(aYn15vnKL__}X}c<`xr zTf*9WLrL=ksevd;q~J@=U>*MVD}m2P!qU{QkMF=L&B9Xkm`dT(kPX1+CLRBS!yqw^ zQTo;aao_{w@Bzpy^UG~L=}e5G!h1Oempt9ho7f3@6Q}(|jWb^XS@`8!nhl?9AKCA$ z^x?VcrcVg)f3Pxo8bGnbHhgl9DVA%t?pV?3b%8yKV^)5x!HgybSi6|jUL`W<$DDtA z63rSux6|=Gj<8OO`NxM8dQJsb=OmpSW!GXZOVm+@m#784Eff3;@3<4btH$Ut^~qTc z7oMucCR|;gZr-5zuwqZ7Y%mx62JF(CUMPrZR0r3>(-H_QDh685(v9b)4li#8DEDU8 z{dae19?NissIhS=g#Wrx^i>PplDZ$~AHLGp z+`?;7o&O&IwLnV0?q{EU#&d$&NT*+LVAp%&u1=3%AdN=L8gCG&GR&-;=%JQ~S_7$_ zM#^EV$bDy|OS=U8kvz^a>}UJox6L}P%1Kfl>3|_L!3v35giG%lA*^w&bqofJKj$(W zCTIW?VV82vmnz~MCEhlwNmYfw{KFsqAd+NM(V~h5Z`=A{3e1wuy=JdYvtr$g&sa@PYd%yebm9AZO zE!^Gmf}0{A28N8e;wtYUnovz(&K=@4*++>na}BW zEr9vYRh{W`CqDqJNR|dnx&NN<>8GFCKct+XYb~&g0G8>%IIgBzCph{XAyWBw2K9^^ z!M@QR0MiE}YUz{^W(_TJHED3`iD{U;Rja+;Ge-!9uEL8KFT^4Z#{s6GXVqa!LO@e4yr3){4RhQR@5~k1EiLSA?=z1NLexU zJb3URm^oTemChFoNj-V;#CDMtw18&tphZqol^dKwLraKon5DyA8vSWcZr0E<`i&}c zjRa}K55p*ojh4e)NJT!YJD?F1Zn{jAA$(X`bqfsGayOIMpjzyuld8Zb{1>|Q7! z#5`-__rL$WK%`2;na~{C3`++GLb{upQ5G38Mqm@jvB4 zpMC}wUh09g5fp-47iS!9|N22U8fF0Em6$9p;VW8v7OS0AO{~@r9Pssr?iH~$3Q{gS z%B|=eE9e0xDB-qR!C$Pl7{DP8CL!EV^q_g@49<;dDuFMBbh#}=(U@|+xq=A4!J!IK z6XZTnGP=Mtr_&$Z>$#~apTl!(K5pWbCir`oYY_Pq%|3 zGX$hY(bH&ww}cR%5o+nwX+*T{r$7Cvp~VfIZ%72-Ia#?i4?ZQ0F|wGHgVi)ukQ7oi zOt5HG!4$W&^}yBxcisax&J%a#nE=s@&H=PsD2e#DxFldn);%2+@ArFQ&ciE9D}l^H z(Bvv$vAB*_e&^Ep2&QDyWhMYNgZAK>he94H1SzW`8>fPxKxuL%W%-MVs1*o!7|xIq zjx8oESvXS_m~(V?#atXSrDuk6cu#?>e(ELWAbnS90WCD6Jqh&G#Q}=bC36NmCPN&y z?MFK-w-pzudvs?1z!Q47=UA#%I9?$FtSIq{9vo6NfdkIeRY6)4H1XI-TRoVZ@`U)= zL@Fh;p<%kjaz)dsJs7}lx3fD!@XZFy6b`@>7|u8%v5`jTVLcPmg9W=0Ib_NK^s090 zAF@`i_H4~q_WahSx}6FBGsh4qW55y(t-ZNtd1w8h!)fqjrV9#SduGXkpuzIO0@QNc z=@O$%@WgV>O3}L6nU^(?MT3(TG38rQSRMK}mPNAWcsjwTk~F8D#FWnV z>RD!jbRy&uSPmnqb*N!IWNg?H7+`AgEoxieaE1dm;hv!p0pqqOfr%SS{RSad^)n}ls&cV<&4hldnckO%)Cp+A zo2)e)pKV0-tsb@mSM*IdA6yHID%X@|FFfZPAFR5pw+sn44vRLcHgEYLKbtr}i4;Lf zpm5A#8Ja%=2tArc9H(4-aAg(UYaVmiKJlD;cBBq~C`ndt+n>`cf!Po1XFU%LLlwT9 zbsS?xZbYfVajc*y8JQ_emQFo}kI6A1E+zEf=v&>w69AkDd_`3;^G!}LCcYbA*Iis( zShj}OhjzLCAF3O~!%U+HjPur%nu(@?eqe26%UKn-73)m3Zco4JUqje(*HZ$J=~{i$ z{cZq=HaW}z*R%)e!XQIpH6^KXX;N6;z-G>aO(r;2cew=&A^dcb6;IrlfBDN_v;yDY zcY%yvMPooWQss!po!-8JD*j6dnzfp^V+g8nq^@kuXGNk)5g;Pp@+z2{QO~1IJ%SY& zJ^HQx!iMGsJ={AOSf)-z1V6RcF4WnCyMtG(h}c4tt`q!SSvQNK*roSqpd5w|#Ontd z9Qw1a4{gQ}IJpf2EGfzriNAaI&eL0<5z(lm?|^-qeqYIz_U`$I&pDW4yLb!?#@SWgl^#o(vhnp8$`B} zQq-d8avBg4b1n*t?tS-{yBo)7DSD&biNqxe73EALx+wH$ruas`;bGjS`7A$|PIGSP z2ePbCD|#@RZ_av(qKwk%9MDBkr6?e1va6F<95~e+$Kjew2F6vFQjl?XP{N!`z!uGr zXaLM?Srx!Y_>_werUNt%fa^=4M6jGJ{zRV6M;LOW74Gzp!3j)B{b?StueOrGA*(rV zO#1ycga>ReEs#`kfS_%y9N$EihOGE`_)KG6ZZB@ofCEz-opY%| zb5diOP7eaJDjc+&6u!}@)o7es;(z-2119CUJP+%94Gbn%HARPa)=0vsprMK_YYgr= zpl4d3bJwKG?CePi{nQeN6bw~4%Nq{95U(=k0NGB zO3>zlH*elp(p%)Po(LEkjGkZ-=q+tMu=T**_P~eu7Kt7aG6W6@T^#v$brYR}X5U1x z;_MlR2|r2w`t@s7LYT)R+0CfMu_qP6mdVCO-INJ$0KtP$Wxp%X;DX;lqb!eKA@sTaGyp^UN6-lk==wjynry2*&ECR*31G zuisw1dgWnPG$kyW40wnzDRyhH3IXNL?MhECgu8pa;pc^stMctjc*%ZGYsiDo8Gnu<1^EY zx@1Ut;Z3bwKC0wSrB2{?|b{a z@?6^X0MiQ0mX8L0TAx|QG3pY3{PD+>SS5Q)h(%X9uw@gZ10&Y0{?$^5!&?AIcJ1Lk z-E}qn+lBfHUpKamtO>vbPkZtven>q=RaXHLlj&N-TSc^5vz@|2Pubh~8ms%)MSZe&YXjnb=%4n#V~$y>QG z?~ZWQTn?KDi7Y;(`->p@mISQ3@=%WUP|hJH5d&3=I%72~JzjP|i4Ue5o#5!k4ipsK zYXsGHufd2IL^xm}YXymewra$*hp{QuImV13ZN1y`=Pn8+WbdmR-foTj*FeV;mW*n5 zyB$ODs_;EsY$Mn5u7;EYa1R|=k$58nAFT%@KJed841i;njCBQoEAB{_SFX6-3snvf zZB_+LIuS)n10Zd3O5nvAzQK5_%N|1?sxk&#*$$Q1z2$xvZw+7=Bg9~))1KeZqL6P6 zbnFZ<4UheP&o@_Gk+>48!PO+fwEi)1d(p!?N&rVKtq81JXz;Ra@k-!LAt|8-Ewl=Nb&H%8|M>6n z^6*g!C1w`u2k+6i`g~|O;S-d@b4Fj0VhhtScFK%68~>9clA-!jF;#L6Q6(RONLH*2 z4FktvV0xy9H_blgLAN0XtUVq-ek_sz=xVDevbx%!rOh%-zLLXbvL!=8N&lXwzgO=1vV3fo`dlYBRb>lbB zx+sPg5i?yE%hCnjTOn>k{I_r4)^91HU$Vrqw6j{vRs*SVXbHrM2UkL_Yl=2_(wrXR zIU`4FGUxee+;RrfrR3q_`_(jxhU_V6Ql9M@w?$?g8YpKAk{fkw8n%4vfvpGbst0bH zcfV7g1O?$h)Q}1ph%DmcpmB;Cic)~V2M$ww8BDTkaU8WH;suZb-%NG(+XoxVj%y4d zL@V#yBA77W2%1GPdmb?w4G^d3Y5ScmFP%CciMC?4iR$2kDk(5?=*8^ctgg&DBDICM z^@lSE&)fuG^SX!XVffIrgy>BN`YmG2a0naG)@qA|!^bsNsQ6NbOa|`7Hfs>r|O;&26oE>Cqe3 zmptKsbrL|H@WIa090|j^IjmRH(I#VgO%AVVwqaJ`F&(UMUKJm7nnfx;+5=L(jc=nR z5A3i;6@}(A=OZzB_2kJDOq*m+>M$imzIiJI-iBN*S!v;WkRlI8bIR?0ty(NBv@Q<3 zPaDjdkW_Jd(Q(LdU}}4Ms=`FbCABf*Oi>jMscI9KbBSpq+1f>iC`twxCeudXNO=Tj ziEKELT1t7GI3Cj4Fehl@cOhz*-EXf8!B{s_3u8rM*yB)6 zL+z#9j<>zNCq*GAx9Wv=5`kl3XMxL#eJo}PzeJ}?kI9fktRzK+uafQPO_h7CPP2c`M29}AsL}-cSR+KS z;ekmB{W*YEl_&IETwJ70j&Bjk8MF(B0F&J5)_HW|&4SHqjO9Lw`X<@cP#>DBl;NAPEblwM8N z+~nAWtjlL!&MQSTRA90azSJ-vxZCaM7l_aqx+U-A9iWvVJwlv;ig4Iu^islNxnbjY zs2`^QaMs``!5M@PEolgb0Or*UBz#4u21L}}6#q#PDYO!Sl!%a3taS*fDnwub+4)Ew z>X|+vSJV)pH4UDV6-nqZ0%-u~pqUioJHl9Z9TB{z#aF%)tc=6dm##lVSPTIm*T>(fv_HRFC!%R=$Z=18ClkG6|0JYrUn4WA^=elqkliD^O z2%ebLY*9>Rnm|gU{SE`m&3Dbs`R1UsIY>u(`Lwcurb5b`JQxX3k_UVMH|mE8WUea^OGC;AK}%2E{zp= zfEq%lT2+dYVNfk`K6^sWLrNq?0DPSzm+YdwBSV&mDyrn@bRu9V$1LF}w)wGM#9Y)I z23v1=XobM&5hackK69y)W%9(y z8a-7-kmV1$A}|hsys@oYQgW7$c{b4_$jGS$aJxA>aMx=QZH*1J`bsT~Co4_&Xs#xe zX0(*gbs8mhlH7LG;u>|NHJ*o&*Lw)aJgHsjch{I?Vbo?Q6Zs=2VlY%B0_G%F5s``6 z+qG{JER5&F9$ zC{F`Itv6qMmt;GgOHLdF1TW;0r4y4LIg#>d9fdTUjsN2zSyK-Yoaq^Y?8-4|h=8%e zQ5385WDH^A%$W!pr*$t6TlZ3BPD!Do9{}dmvTpv$fqoMtaaF0UXu(5P>bVHK>y={` zIX#kW5~r3RfE<2ue0bW@T)s9T&d;+k;s6ChN(^{8i^LIfosXb1A}5Jx8Y}oB4}fK@ zuu!DJq~9157;Pw+wrKIbi6R)VG`{63j3}?qn2rq+3!z%% z*xkV&h0_lR4lG8tZ0muo2ky8B4l~nTCDBSO5K1_J7J~$u31K3L+Lo0FfkWVPJTFq4 zbgg#OF_SaFkDwNl{>Gee#a6hkdd#4Lb%z^=(FxWL{p7U5C-CX#x%ITL$Sx^GrvuS#doq`Pf?lJ|2Vk1psGz~DQ`dvRGup&vBDXUnsB^eMKU?1ABLaq>Eag0L{^ARfp2^?@Fs;%|3KCO7b6$rhThDuF zuZZt?zNJ_g8ZZI4%CyCSM+zzVhIu;knFgNVkmY&r_hMS2Vd9`2|68;<-!W+=E0?rR z$7Yn1G4%ruooac=iYbuKitUKd5`sGa6 zb=wS2D?DPNhlqWkM`Y|eNilk9bG_{73>eEYFVdSzJ%q8e@;fPJ@vO5rE|De0H`;PF z4qLed3~*B-0xyS$0k%<5MAtgCUY4;YwseS-DvNI~)NmU!e1_=J;h3!BNC?lS2BS)E z%OTl9NMO`zU}M|J$pNE_1CWM&3V~JO)J!KGihf7HQX^Zs$k4TD3j+`1{v}R%Ygx4Q ztJ6=i*3eF4?JMkleG#VHJ(4nzpiVW_X&rK@c1;_9zke)Z)+# zFPPlFe_wFpnD4gev{9$iU;`sgFeH(YVggNk;hoO}NiZ0CC~^BJa-d^1L}G6LE=OBm zOyfs$7m0K47J4$7vNP5|1Z z8qQAyK2IpuA-#cu7Dd4T)GE3W(r*cDYr4u1hg1J$wVhe ziBwm!!GpuGOvt4$Zm8O*$TW-#A*a^aQ-)}O!31D2J@7uP5UYp7qBq5eXcwj(V;`Dv z!_7iA6WNsuWpdgKOM{kF7``EB%gKDmB^*A<##OX}ygJDJe!2c{psm=_33(UoAqhqdJXkE6W701OXhihH-k1%uNxV zatTL6oFLSVIbcOW8;))n5#SmEBPq&pq>LK0O(IMaJj*9C3fDx2JSK6EI~QiXVY#S3 zyo{D0LXdHN;ROUI#ZKo7GPD8LOTg3oXxGTQHkj@{pY3>!NF;IBr#1|o#N`rBDLoo% zz?nP(h=I1Ns;C8FaJe0V-1GGS1eo2;)4e)l`g-k5Oo=8ec{ zCMWT0J8=&C2-mx}0B7TW?!g-^Om~|znuh!gqm8-LP3O4|J`ONK^O=AV7?YB#Y6-*> zQuM=jT8C4t2164xI|GRAoye2LPiL(o`|jO4d;oJ$Elr#t!u7Jw2e4|@x5PCHzqZJO zrMb4iAmZ0d!=sie;>#`Y0vjQUoWW0=M0k~}QiM^97;#@* zI~tD(;~iSZD|U~L0|?@%A|;{uoS+)y-FfN&2ZGJrKs7v*~R#*^dS0Z3qL6#tbP6Yo=Q5iz#9b2_4upA&MMDN}w5?+m&w0 zJ`;(+Gwo6`hmXcbNHAQ&51B|+r+I?(Ao3wSa-8YFR1wGYD9jSP;RIl~+DjOs#@<}4 zw8@F2JOY3fE@5U4Myfe#yNJLn>OdSsB3YM5jFdO|tyW>2Lm391kqQMc-tv|2_xo-r z9b2Ou7C8MRZlB=$tS+C2Q98fhZm{Ha$?8L{Qo@Y}Mn$sK*2xxvFdS&HXn?t7aRQ*- z1x~@Yz#~GZ#7e4&D>8AK4hF7k;Q^A0ppb`Xxm4>A zwDCWkm;h*eUD|_g%k)vJYA!^V0OrY)Cui^XY{^q=7w3~k%UQ}DUh7!XhgT8p3UWiv z;l>!L5{s;3D$)ak&IcDdK^|6HE7GIAin=$SXD_FgLbXJUkOT)dqlGP2TrV!cgGrgI zH-_011I7@*AkkACFwR_ACY<7R+-BqFVBN4cjpBQ1ZHT&A%vOpzidst02!=ZK^I<=2 zn(s9_u5!5E0|3hDRm35I(a zjaCehPRx2$5nnBZ4&d-{jDe&W1*-{i$a;G&D=-?yemL0LjiG7N3b#h)yf-1GkgO^W zz?j+1eS$fi;}}8}Rqi;aettj8RL)y(3OG;EjR3W7+6YXidEXlBZ2SkvIuYW`2?bVn z8LA=p2!QE+f@x|%c2fyXWpQyW{(R%jrV9{ymW zJqsj=2J2jdS}=ln2*VjPKFtPLe9ZhoqzwTq5l_RqNJ>$jIbfJtM-L*k=~U&NXcHw7 z;SSRaIN}3N-&;8rgr~&KlEZvJ$hF;)bgh)Eyc;1q&krSu;`~&6+`@y*TAMI;#Qd+j9B#GQv#4SgeeVE zLhCVLJNAylJZjVaIniYd_wyD5T_uCtsXg{cU3 z@ag)#+wb=dJ1~7<(AHAjkECcO4(UzZzGd*?nDyG{@cX@;ky7HJD4D>$dACY0A9FbGkiB2}e2iWPiG-_{VG6d(A>mhdIs7BFiO z>vU3*280K9H{I_FJw_dH)==+B47b1(KJ=01Y1RXm);Gc=h0o(0Kzyywz8fE-YR11t~mJ| zCPR~hp;gK0tO+JoN3>`s$a2elpUhBeuS=(!#pNKVir|>j8?KIk=41-r>jS`a@*&eH zm*d3Db63jt8?7sZ3$NB|EP#P$PG7Kxo2Z-X{{8zV5s%Pq-7haM>Hq4huk4+?;6QWj z8be2fO}p`5OULsb-7EG3D{5=ns50_n*4y)H+77B!I`1!${R;-JN9$@EV%?okFi64~#hQ$dbAx{_BcG_d&VTjg#9XFW)!T33@n0RgHy4WM-&A#!J*5+TYP;epivT7RjnqT17>YE5y8yWWMBk; zRB>39nU@G=)^*Wz2oAm(5HkIWDVLKfWjFrl!Av}YMTA5IpEmf48iD{QVn2HH2uAEO z$T{QGLzQKNwSn;7>JETN4kH+KErdAdBV6_CIqs%hRa;U-5UDNtffDhMpozQFF8vus zl~fyvY-#I(tq1PB2mZcwdC#th9g>J3cF56)IAVl^7C2NDR&Xj7O+@*LCTXZP7`mcH zr3w*jUOBc(@KLUp+-ze^|B}t^|1AF+ooO2tx#aPeoNT{qP%q)I?h%~#V-aj+*WM#5 zZOQQAo8a-C76T*ls@m`{npr@vcDFqCL|d|3raJ|Y(~oexjZowUyVf^U*%kB#DG^pw zQRvCJqFPp(=X$=1>0eP?TwEX|V9NrqrVu*eF(rxeJ^urvJW9E_^ zgG}=`&*@|?KY5NIyS7^q2D2adV9wElejv+FGY!FI{e6+WzyTq8MDKI`+IX7x}-5V1h3_n6nvx}1FEXFZNtPwr* z=B=Aoq773HAkmpi_JI@{0H^nR4kQRHp2mjE-L!L*vJ zMLcF|FkOv%<7nf9x5iA(!6*@&@u|f@NC^A_f|ureZ^%i4oye;i za6S;>Tg^cWAg+;6mO|?&gh2(&TeO6RlvgC4hOVQkNtpn{sWy?6P+Qd~%ngPl%)&3h zt`}x)q0O47gPSj?zmJztf-2>M3feZzh|rrKRv)w2nyB4scDYbdd3jI?=cxI?$lydQUCxz07*na zRHLGnWjuZXlOS47ZB6E++o9jAaW@HHS(HnVoPpFEQk-&vIF{?IgE`iWBZt@RH8z*F zeCvU&2kx*3uI9G)?}5-DiARD_kxM{9@I)LD#ncvG3UFddOnh|6s&YO6P6>=c%xd9< z9$~j1;P3{D{jCK$fMud8wDjP?gMy<6ZXQxma&d9NNdOO9ObqnUk58>v@&x_D8D`LL9a5~iGne(r=6x-%n$=YH09#K9CLgh!+7 zR{T~uhlw*?7p0Rw^-r~IsnzfnJx%1(fsX%KtTQtWPmmHt1m*E@6!iuG8WAjfIS1H6 z+asYJkhy$dG+?Si%Uy_=a>-Rz+i#M&?miA@)Z?SIT^1d(7zSMRowJ;tE(912bzqm3 zwz>jAe{IQU7sZTjMaTpWm>f+Ng;_kCH_X5&Bt`QYMmCOI0<-4+oLIzaI$OIlTy_`5@J0VCfX`T>WO;+*~2dg;~1e6H#r7&*1X(bm2Q8hjMvdVyi45TOd#au6+9 z4iaQkE-X0B1dysF1x$mKw@;rw1y<|CEu)D8TrNNPi3tfqNk=>!4ns;Uz8sCvZ<528 zEZVWtLzW=Z&Q{tj6HmmhI9l;Ek)?;ARBf%0(}|MtV;%iUa-b4gJsHM%(FOvL&P-YakDj4INTp^%F8Dl5dn$}Q~kiheBx)5&Tv z<7afsw;tGf;0}A>YIA#69u!bmFET2M4uVDgU@8`F4q;=7j4=Te(ZC~BRWM;lV3HD7 zgtS`CmS))p|+*;MtDPap@HA{6=>tEY3@D*M{*p>NBa?gR|JX&56 zWZDYbnTV4eI_(~L%V!u>a!kqN7deh^l-$4nAG``ORTA_#)CdBEb=-Fl9fZ{IwpFmEIB1kE~cGbfi!Bt4P7yu9?h(Mp9C z*TNCW^j=K>R;7D26Roy{sa?5U1K|;B4+a(_PJ%v>i0>E6{c<5GNd4ek$v;5#+icuR zEq?QE$6Gq2!;al<$07^><zER8d3-;)=f!D1 zDh7iorysG#D;^~<4nWorJp&I@tJgVZUiI>}qW9!I?QxmBaKWy{2Ty3AjassaXA#X1 zD;3k0!d#V0_>{<#wP&+H0du4E6Y(&L4lL|HjV^mm$e8uEGZ7rgTAv;r3!lz>bH`be zVM;#H5(FA)E|A~murDIz%+$INz;f!zlPA8%dHeP)ojCc1M$}TdOKjiB3MIZ=aF$V9 z+#}rX0b|#lpnI(c#eVG8=QAncLryxQp~0o+Cnd&n3ty(^oD|ZAlLKnqu*Ox}NO&Jj zFk!!KS*b|5qBrB{bSFaU1f4cI?JfW)x89{zt(S^jP`4Eg-qp8fW9XLTnzopur$k#f zOAgfSm}=95w%OauwMV<$Yi??_66akA;ZDGD#zgbk`<`vdkdq+7vexyUry>!YUy`Z$;?-Amvua>Ox zQ_dO7;&xMR5dTPpa`S?JerZIpB|ak*Dd(H(PGQ~E77J~TuxC(fL+i%xyp=6TOax3c zhiB`Brn9_SmjegbsZb_t{!IZ!3qN%QSUX`O3V>$!j@%SO5COv?n1pU{dZDRWCrznU z3oM^v9L10bXUb@NKf&Oc1Uw(S{!@$Od9JVQx4qxw62R*&Z#Wvp1Zha^xTJWL>JU+= z>4v!Atdp7JgCB3!irOWoKM&=SX-kjc`=9G>bFz}#&^!w{9eZT4B5?$aJhk-j6CV>X z*JAJjOAT>R3^_NWA*U9TtS>RUKtf+?Z{nx$D4!rBI14gl?JeB}>E~5f)3fdZf-9>^ zpZ7ooZG>|JQc|o@^`7YA@wj0?vvw)kqNDlg8ij|n&&-N=arrrKJ2gm#? zWFc}Mq3dc6YsH*+PoD2 zMHqr&wJ}!g0yD6!LJME!XqN`l?JPnH;KT(_q0*<>^cYAjhVxJe;*8|x)ss9Fz={fy zbEdX?CyRdQ-T4S96N+PW6Rwv=HCa4yk0=@gkd#yCN98O$OtNA)jy~~2V_sTE>rY3+ zF#O~XE9eB(ZfWa*tq1P52X1^y^WNMN5yS=NiVoRDi;@--?OTqV8IcHaD)GWxU{#1B zX_`ghUNRSn9zJ{sAQ2H#(-7F{iG?*0^Piu(pu|Shv}7-2^U&!)c+rh}7Cpc?;u_8O z@836{SsaUoKHKqOJS%B7>C!Xi@N<=@l%vrsesB9z(&7?%?6f%eD#Zj>K z>VCMyVL_7wh1Q@2;<|!#_KAaJd&N~bZM zsyK4#pncU>1W8d_Robjadso8i6~5Y_3IOc;-~WDFprq(0g>LwvgVCS1bumu(ti++j zGS?kvId0+Au%|&+gE5`9z|J*{e774WL0z%i?F?E)wn8Rlhir%7q>mm?z$7D9=T}Ru zft3HVV`*bdNtXrzT<0JZeeG{K?yiw+u43ZTg&HDR zGdZUq($jm+1AE~A^yyQ~t{s0I8v*+fi9p+Z+FpEz3T3Rc-zhHlO zF!{aj33Q)-{<-~tt%c)M3Uvru2E2!9;lXIDXQu7>gI|>#Y?`0*<(SgJ*mB1?sB(we zgix5atmu3lsB>I#o{Utf$9s4C{Q;maUGf$p&`Qzl;BxN|ciNePxP6|wjFr#&n zWv>!n6+*YP1yv+P93G8O98P@rINBHnhciTQGM!qRD9;jH6oZq9i{hQBsvMg_iV4m@ z@M!g0@HTTvgqkuNaB7POb%bwV1gMKdF7W@(g{4s{4v5c!~hd=J5Rx9 zx>KhV%K646$55z8KRq09@0e5C)jL-ui!XP)Rm7l~AqsO|aWKOfAF9I7 z0Y&sQGg@RbxjRS3$H|Zs^IWUZPsOVObFe%Yvx`+mM+?+TREevFx%x7M~nEf4>j0S|H$_--4kxRP536b50s@#*qaNBya zAIDkb4jGYX0JdNhnJ%Cu9$?G29@u)|yY7L*J^w+a;*LPzC#Hp%z6Zhl7n~ebWW^oJ zV)H%10!36xVRJTfB_fuR7Ar!ML*IT=(@}_X2UFApVqnVxuWX3;^rQ_FP0*{J77B=> z8=ZDLn1a4A2h-QE@Gw%I07EJ^Z<$WK@!|ZNxA-uoB}~bh)$x2U)+(wpSDBU$x)!4} zTXcGe;c*9Alg&vfu~zfwjN6a|)pEvD?j9v6V$vCy!MCOp?jGDKS?5LaYZ(Q{);Na8|=c?2sFqqcsINke0X>>#sJ>%cUG2pB%5@oy(% z4CfEN<+A0e(N(*EQ&kfH3@}w$ELUVPJkA==3XZDuJL>WcSDR0K+KetHtmX~fUh~dX zha<+DHBGMW_j{dhb4w3=7`YHDgT(1MlIMQiq>0mj$V1`{ zLa#FM(>aKCshuX+dD`nz}slG~FD_HSp-N_DB@c3Zxf(ZggL+Tl;G%w;dk^?g& z6y&7by+$0~6TcpXS^sGcrsQC*M`4gKa_NUZ{GpehyEHUQ&Q%2&c(po&s}3K%%ApEV zzQ`Ri?$N_Tv?>X~v8rNL3#kcYwHIIK0ITAgdjZGx%xV~HQFF+2m(Lho@nIq$?RLD} zPVkod9Sj(TQKti%lUJfB21r2j~R7=8u`3J=bl^@wI+-9@Mr)@rRe{V@^wK!X&AAvBo&2^=3d zLe5GTC@IYsm#8AEwwo7WgYE4V{H6(<(kVYTPILFZ@yr%M4sc2T3o`fa-HXFRgt6KS zL^PPc3@{!F2%1bP%#m_@*T{i_u%H#)Q2ZCZNV#@sf|5(%pO^yEz!qnSs8y^J$|+_2tW#Ze;w#VY~v|_Z_56TZyOEQcCAo()61z zm=ZvSXdohYN{i5DzO&Y?37L-59f39yv;CkOK7Y(; z&hiAP#s+aPIaL8>IHfslYQbtkNNZZ5Ky67CB2@+h3|lQ$5vpi&8(PBkrlF!IG@c11 zSQ8)<)OG*~r&O^VC!?fXQ{81m;f$0^;A%S7bm;-`B~rx45PU5~=>IgVt!(tx16vP# z*FA6*8cxmFDqvc&xIepfPch#ShdJ>=EE6n>7yp7GZj&lh$qS%35ph^il(T}vVp5sI zES@581Tw1x4|3r>RZGOO&w;dsvPDVeh|np)I|cWuvq&xI)Bo(*GjDL1b-eWofJ0Tb z&`e9&8plLZ!4aQbhRp7ko^r}XPkH!^QsqqN-Me?Lnx#3NzC-bn1!TQ-KvYq*_C3JR z-QC?oh#=iHbVx`IUD7Ea&Cm@)H%K>v)X<$GB_JIVQc8&W4L9ERz2E(RVxN81S$mxo z&kuJCp&g~<_?aNtOOoWED^ua{8azP>Z1G97x<5UDhMqpa#Zade#5I(vZ+6PU)$<9s3cS0H8vqv_%3+y}U z8DcG4)-IuU0Wx~fM0Gle=oH`v6n$>R-rXWv+|Ovm!))ZRMIJLon=9}Y(}s`j$m4<| z>wyP9Q7A%z#fd)E0(+jOu3tB+OD)bBS~;>gP6ByfHITi$^mz?dJakAQfOR27eDDtl zIQeO|r#3h3$SFY#z<8LCn`mf1K!|M5uqsJP3A(tS;qyj^F1%7`SdO0eDj7?rH{}{x znGp8EM3S)*v-`W}PYVyW&vaqle}#NRde_6Vy({SC^H#u&`m;8%Zuntls?5fxZ@43` zh|{?I6<<{^vTMqG0*V6YwgT;8m*?>-E398c@JYMB(;2I>p%`NvYJu;gygc}@iWs?ZuT6_VCiRFWSk{U-!?W487 zDybZZsLu3bE^ozC#Y%JGMO6oS)iEa#Nv!F5Q$cSA_B?hob2R3 z-26uex)zD_$g0UO$xuN6EEGRu$iJlsZOQT8a7@-l`%4&g71d2uxP zcI@?u=ZzaD&yu(#8DLT_! z+GK%!U7{?wHk^b_Sn>h~rPGlt6y9O&$IMh*Y#|sbE6##7Je+5vUJ;r?krfdKmHyd8 zUF9|NY#iiD0*W?2RCi7PaO>)pjSd05GIbPm0r1)N>Nt9~tNgI)RdWp9>T9kL*Ji`$ z32wEsgM2@R#R+Y13dKa-3YCq)aqHeVVeLF>P#wly0@m?)smGFApQSUm2vLm8s$S?xcjbt| z6k@P3l&WW7&ifiMXBM~4XS^FbZkEx2v9?|ct)9GAWQyk(NYtdENQjBk9jO~q3921` z1xSM6`Mn@D&WSILi%B9a2kLQGh4Hq_gAJVUe>Kc4wd8e7xBo#Tdrj|UhLM;4%6r*L>Fk2pe~d zzxr&2&HKg7={){6?FUs*!9|$CWPmMhu!qOa)*OLU0m(8)K=fHQ&e^?Z?g6K)il{Ka zI_ck=!3@xX<+0~DO+%c{NWv?oD~&`H3x2LU4nZ8k7A2CHpYpo_;mH8;cm1^ntc2C< zoXzix-E)n3SwUjpNwF4#?ky#2gd?d*(j#hn?HTwpA`Oti5E)3ATo^IIYQ;4I`M{v)oF&xfS)?cHrA)ms@Rxinr`-&4jVNS1Y_v4E?!q&*JHrf56pg@q=WKaN2r z(IcVL0RXN?1JATyc~6C}c4FU*1m1uPogv!vi-f`e+Ndt~VuCVNTt*`_p*96}m^!+O z27latZ1s-f7zas$c_{$KU6DBdE# z1_>7Uy0~ie576k1vb#C(5(u1FO<%at>71s&v=%>zvwwJ@eC`O@jJz0KK+F`PWqC1F zUs0liCtZUg6|e-|g*_Bzl=F5I4SEx^7*1D@Z=LH<@$qCGmv#X|<@F}sFXei})+$`| z(?{%4>r}Bcw1TNM6nJy3(;$ZVu8a<#C(w*cYfD5zcYLpINs8eB{Nr4KrJM2ZSCTCD;>B7bXb2OpKm#a?{`vcg1Y)6zlu8uBDv zd|(VokM#;D1lAarH`rX-v4RT6p(!n|x(B5no!nx@&PLwR8Rv2; zUm9uxk+ug9l+JNE8-4T_j0g8|s2Goime362PcAUzdGTE0gd@Aj*&q+~D#gXtIp7q!@pgPMo%s)0$an3IKc{L-;Og!n24PZb4^Fc~$d zgqU|MTftt71t{p>XuO+VI>X&U=gbi2Y^-zof>+r!(HyT6rI5c8YXhK4G$a0$XZQTY1*eNXLX3 zd?64fRMnBQd8SdI zW>2eynU66zv$`6*j)%&irL8TKQ0iKmxg(W7V(5kIwzF^ zbLL14yMHP0ct-eZ+J3IcdKVbosUW0DwGY-tEyk)>i_;c=HMBIHYW6#Mf*#az+{WlY z{aQ!^69>>YWZ3Wg;rOF}q}q1IKIoN4pk+-QN{Xj~5~sdFJ=eAzP|o8G>Iac1Rh>n% zYA&~(-im{(#yK)j|KoS=s+2B`Az_#mRvZ}c&%bB z@&_m3BVI{i`9lh}Am+r3(u~eG;Ta&k_)5m4b-1VD{+A?nvFHF1!kmsO+0;y$Syo}= z0APHdu&V3w7blhQhIp*URVXCDuoFp3g1Bb>1(HnQuQN(Q{G9^onvsdb)+eBr z7ZYVYI+tLiDDp~5oqHbzYKKw{E2&WKHC!l7&;0e0aHSQCqAn3G@H4T6rWY8<@`;I} z;HP;wd}21Zgp4==!-K77I!bl!1M6-D8o5sW@yq_<6F-hX_D2LzD6vMk+(PrsAb-hC zoa1?^bT{^QkQ%8Mod%FX5Z4pyv*f!KntcV>9;WOF zjrK?&-Y_~m!;(dJiOHQp7g#I?U@{hiMc5U6Eyh)8l#Wa#9$<+_<35tfy}=@80E?Aj zdg4^t5$UK?OURIuDDxV%!(+=^4U5CXN8n+MQ4ACSbT-*Xfj#kp_gA~9>*%p;feL>B z78YNarl<`>mjp9Tt}mS`5|p$Eam?K2Wml`mI#viTg%C&3w~ z?Lgz?Y-Yt_!$?}y`Rn1R$aG$uV<5Z7`CL6mE1Ui_`i`RM2dRP1!PwA~W??Vc3?$!= zUy}c)*Rq1Ugw5rFdUrNhXn^2Txr&yUs{`9LE-^YSU?HQDH>Bb0{Od!49hybiu6(RN=U)e6Cc^u#T~0V!?nSR|c%w3^ns!wRh_bV@`?f1& zkM098wkXq?1$T#3ev+9ULosds=z(}1!*%Q~l)YtCczZQGm4`Qs{yZj!}x z$H>=+`uAEjl(*p%hviS1zj{$ZVo?*5H&t9u3khXd?D28%g9m1G__UV-+ohhkKag$4 z_Yi7n?SX@;uBJ|yvL{V}`>A#rRlPimNGC7jDNDiiZDffrGx}EX>K(&29zO=a{dq@BBUb>w{Ej^x-4g?Lm?drj{b&}?CXll`~z1Fem z(U7vFW9w=ciP&yT=cOdtUSQoY@<01cV*mqx}&}mVh^` z*n`PP0motvy|SqIjX~iDolN)PR8cEo>2YW z0bb|$A+HgF&e?kz_O-qOHHLM+hMo!^9zq@ZztKBlM{uTQ9>*F+jelLIW=}m9c(vt( zms@xpSf^CsS`U)v#JBL8$7sLRi>f3GD{G`trW!Sm^H50p>wTz8wZ9XqQbx1z1?SDV zV_%jcin{k?81ruAK6@o@cBp04sAN#gcp-ybFu2?*j=K3-cxqjH4 z`h(^_$GEVI5R}^Rd$nvH(#mB3hJSE!($64Kt&pA*QUi+OByA_@A$Oc8Ci^=nnomr^ zvl_V+Cksk$A7${*jfbm%(Uy}jdgq8X9gy|g4dz82seJB#h;1f*wT520%6XJVUVr5= z4w-BXpAZfi@dSV=4#y{bifh`C=&Ng>7F1E~IHUSA6_wN|A)L1k8`K z6kP}?v^1O_zIt$E@W6+~yrmIiwX88xK4JkO)63E@R@3HM<`Y({rH(UvMpKnGsj&xu z`N!$2^US`1=;0HG7BK^$}VtinU5 zzPxA|vA$}`q61vk9-Cgbfr5}|hM>|ArOcTAvuIVUN$5$%i;%h$wf9oR=`|lZn2f#Y zc7anl4N|X~B*IUc(_(ex;2YIUJG0;jhfM>LJ5h3v6w_hZuQvRcFOHNuM`u?2x9bi*k(;)O$<;g9Vi+b ze}}#O)@iZ~*z;F$9{uTc@aN*Con;qm(!bg5e_!%oIq5;{XF}@#2*$p$B3n?kaa6~J zyD6`qvf~G_uw?y;OJRCqW%HdXdxd3{+e=JCMA)6r(Gyuvum;&7AT@FvvjkxZ$Kzc#=Bj0VO#_U3FrGuCmo5N*aZyPeOvp zM2D*EiZj{r)1gf%B;0CWObIh&JC(NGiIJ6RiAf2QkZO{Ayu=I%Ek|DpM=0LL)F;vH zq1g9Tj*2eGF$;y`8gyXYe3;OoP-DhCrfLwjmr-R??=)>y>B_4jAKd;cN5z%X;Jgr|ID+UO;!s4G2$gxGz?mjM4>9>NMq=ki{jjKv7#Cv+rP>0BStMnQ zi1(iyu`Ut-l5|83MCm|PgPRGW%QsN>JF@t{GK;2+#B5wu{4(d#qT9B9qF#z{x0uTb^*(&k`$WgI>1 zQx-tBB{I~7hZt&hqnUa#%oL{q0$x4G+;)!p#o7Drz!Uh9$nd}! z?;pHyvGwkY8|hH})XAJT1wn1~^^q~MDRTQF36XZ6MDrIeYZy+?pZa5^my@9x^e%Oe zK&4bUvQ5qlsetg4M*H~}pw9}Hp5n7mj?(nZG9h#t-P1+R#){Yk{b4wC&K1TL3c?i8 zTn78HQt68wwai}+RCg1lqgfDt>-L*#Of!8*p}M2GjPpRQ`Xx*c4!LG|>7*}P$2UTG z)iW6OgQ#{egRJ%S&89a-0pFNIC}y&r(bOxuzuQ{e5WD!0kwyQ8@f64wpE=FOcnx~(b&Q(Yuc9e zGS?lEpG+7-vVH22S}~G2Z7)pzl;{9pHPNvYjhc$?pORwt0VY({64hL;Kjqobs-b9E zuiFny@m(Y@=>oU!qv_NAd=v!nf*2GSRTYQ}!D6#Zyd@IlyrTs^x(PyaIh>L_tOpVY zy-^|v;(E{PC}S5q(bJvLjpwnm=V@6)C*i*UeE)JQ{v&8PailBybyza`X+uM5k3mim z!L^#nKb?atgEjQ6hMC^~{5;}|`1R-wbUZ;0Z-YhudVjK#y4W5{jupM?9g$3L1L7Y} zQK?WZqYr%?|7%fWA|RS{!kkN!^3THiZOikz5gy~Yvhq~*D-<)h@~l#QL@BHYF-@UlNe;Vs;fToDaN9pEFrsOE7C~#OuCq%`|l{gp1WZ3ZB7-88O zPQn6kFbmRj$H6;WMcrJ*wF2>lI`C-t-b z(zd6&^!J&hQu7affqmDoxS(jA9R40C4Ic@FVqZr-0Y<){u0i^I)p0U}Mv?@ujZn~E zjXCM!+A-wUtp@%|WrArHluVelli>9BpP#Jg~ z`b3_jWJ_?Q(+l~}t<(n2Vk2)Vdh&5Rp!+Y0o-IMW|GJuMdw~xuECZslg6cYxU;1HwKDC;)Bd@no_%;}j{V>3F%$w` zYwh2^417B6%H{QHH`aU`MXfdS{+GXRb!MH2^O@X#F5$m+{)98(eYAWzWS8Df;cvj` z-CT#%A%Wx3#3KabK$_|nGxjd>W$9on{S%?7jUY`Lb~N#C7WUXxuoia~b5k)PxYb0T zKvZ6G=zx=t_I;!PJ3zkKamT*X+lr^bc7bi1@O>d3f534Bh^8Uta=OKyoQ@fY^D~-I zga!@WYt{G#&I&!qLDWvWw+nAmWdJ5;sN)$`hNcY*#Ac(S&UCf zar^iW_W3tQNEgV5!yJyFMl+@+gec^J{V$_(|1~uMBX{EVUOJ00bI&dc($xNvr2Q3; z(1@Bnhvx@WVFN@et%7*vdw0T>$WkZN2bG|%w*hrDwuuq_(1z?K{gi!(L_yuG91pKs zOVq!y_fN`w;R;(M4_6p1ml*bBN6$j{!w_X@nke%{MKkCqFL7Fc!S%00TtwGAQFiAr z3tl=XT#=bff-rSSd78(BJ`87pY6sh6!pH2{0RGDs5hlMZ0}Lde=#sd#;qfFjt}svM zRE=W+O{5Zt<}4P)cd_P?v44F=M<>7f&N2M!tK0h#ZgY=)Gh||RDP%qM7WO@jr{(aj z@cS7hvsc+Qlg(pvzY*%{DoKA*VkCw$&|;t?CNQtSi!_>D^ii9_N0zwv2!5fK^q<>P zvmb$!vpN>HJ4Q+E>fTYqBbIhz(Nuj_P(^%eP8=-KbtFnjI*sBE^I-(f`>tmdGc;%` zjDrY%SHp1YK%fyGnpCjeK~mcZU2R)z9tGsJvrEv3ZUbe2A%S}GY&4-lsEZ%Ki+ciz zCezye(CvB9|G4fr`1i#B@DV}jr7S)R&d(I^lQ;h!SNcbI-!e`tu!!9=ABmC2s|)c0 ztY{F*qizU3sSV_1?}^5WnV(rhq~C_i5KBaak#l`}gXGr+&&*l^I8B#FOXq{Y;)~3D zF)k;8qNja!foI_Go9ndf{erAqb(=#|jF<)zfjmN8rvXfFh&GK?Vk52JhR4TE-yyB( zsn!mTC#3iueVpikVJPsj7+n&|>`k)o6 z4tS$bm~OQY@a$*EHMLvM)iV^yJ@^?kjNrcZ9AufABRH8F{>3U0-<72w7mEJgRy|V? z{*36XS;f%udF4zsmF?dmIu-x!(f{+;@3DU7*e$)hsgnH@Zs90N_Fpdbzl`k#WmuNV z?4Oe=1{ql)Q7l}S(U-LSUm9+KpAob|l=o^Y zU9&!H9-_NYX7fMaU+-xnWiVCtk0JcZ`5fZmcy2!x`oQzdYEJbKTwix?3~qe4IVH4+ zwod+dP5u8qy9k3ek|%J8-QS@1UL+L*r)A~nJNh#BMk+@~WUZFdckOmjo4Lde!=0D} zz@^*)>3_a!Ol0jD9)c4>gMlHB9gcDkr4i(8^4BsdNd>l}YyrN1iTO6qr3JG-@*F+jofk`Da=^6pcuYSe;>gR8Y|@8`{&aAmV=M6V8NiJta~xT( zT%?sup78h^L7jVMfS=ZXEPUFaB7QIUl+$i@2KmlbRTx-fZ?6}@UMi|+SKzbUEC7@p z8SEu3Wj&L%tZ6-7K$xS29@Jp%ExD~{Wj|j^*cpFdY};NRlmF!pO_;>)Vx(F~0tg+&5|&ib-PI!H#wlVeAf6Xw*)rjNm^GG> z<+iX59HD^hwl!2%yODUUd)AKu?^1SL79^!h|cx#Dp_UFWbI24Xhqv9= z9jlO*DJfsHQ})BvB$JlOw{|*!W)5TgQ$`xC*iQSD!rjb=Au^kPhqMpdJcnk{oTpS_@aukEsAuOK9 z3NZMR2a(8bl3!05k!sf7-K9zcL)&U>VB0NBpFpDYvwf$8o@y&ISTKaW)fw@h^Y*?0 zzGU&zS)tp)^Ng$H27@U?Ut_JEn)A&8zSqONT%hlg-o`KcY_>iIkS^w~^5@-FG>d%6|z+Og8BOfOyUyZGI z?9Rai0e#Ys##V^l8KyFkAB!r=rnPd1Dasvi^HRIEEj*q!&Ml39iX~hI6z;k0R(TkdQb@53T6V%*7C%ceSz$IR{I z@Bi_kHxDR~@y4fjxPCgR40;Aw+RMhH2D`BN3#*#zT&nj= zseW|p-YCU|3FKdHDvpDjn4@2)cQlB9dzdv(h7lFM92nsHK_|dp-bEZfm>m1QdGD`@`Xw+7OzUx65`#6A#jzTA3@1smuV)KyY6N~3FY*~F+mx)Nn2wsX zQUVn5^A3r+TKrGf6bDC*hA(BSu#hgs+Eh(QM--5ldHRe}+Lu!h3h1fUf+~a&nVGEJ zm%d_iDXAY=#Ld=XQg4-{-14HoeND%kVUrTdnL`|i2u#DN@g%Otq$rY(*pel>RI+HK zz)wE!AcWF)y~m2*o(^9wm@KP%*4B3fBHU{{e2IURf*vi$AhgN-dYRfU%_^=; z!e&x7^v#wSj-1_(-n%T{TwCqA$~6M3-Tuoc{--m0>8Wdi;&CaCjV^dWJXdCeie1?6 zvsQeh%KY0aQtFe7_kWMx=LQ}lbjEu!ZF5QUu=O^ej$C~Nb_N{VWpa!}G3oCX4Wl3a-VVE2b=YViH1HsP`(b@`;*dc=DZK{Qim?Dk& z+mhW2R-oa%UG?TaAVhiQ>HUV zDZ~tyFAtOtP;{M9#p1d1zzt#<>b9oEGXH(i&F_qvAd!Tu@bo`vLp>N@v0A_O?oTQ=?)qN$-6Rf53$L zl6dh;7|X}4Ku1V#S>HN4%CsT7UU+zE0HUBr+UfsElf4 z`-uqXyskfBi(|diB(<%Deq5g#WwvO$on)Pj8VUCgt?pBuvTvfaUtS5{J}LR|dB=tn z0IXBoyw3V)6G@RNK{(X$pD(%pH~i;Q09N)XFm(eg1j!tugXGow2u=7IG4)L3bkiat zific=hNpLj+3F4aMcT?urF?eKZBobM_b*|5{FXDg)s^L{UD)ZNB$`|tnR3`CYOg<~ zoh{96PBK+P#*fbmFW+^;U0yZ~?60&YYTU^lwo-8?eO~luoRi804dQPDc9ujUFVp`% zB*g{-0|MHGkYU$K$exM;#Wfn21F{Yg#c5x;J1?1g%tuu=-L4UEbum0_K&&B&0AbL{ z3%#=x!g|r__)APJ|L{a=76Dh*CRDf`5=4YtMfn@znxUWd0ZXQcBbp7>0Bx-gl4C~U zQuC&hXmF1J&BY@tCQoIRF~u*M;M#hJ7>SKO>c~Ep7(q z!%>*k(4Lv~+m+RWP-7h#H|2zLe334Rt5_}Ja@$rVrUkvFNUp(WBzKC9#5X%OPJcbZ z!dRcELx7<|zZ>^08lz?-0G%Itvhk%KNQCL{;pn1@)(iiw1#pPJ z94txwwzDKHE_1!@VDIyffVp>e$e->^+$OcbdUO-XK!n4@l>x416j+9hEaD^V(Li zj}`X`lG?RkZH;`lt{pKmOB@$PN%3! zKD5a$E47D?XB++e;gl%0mjYP4qoLQ?>=7OHZex!Q&--3Z+kKX*SFXdd$t51EHKph4 z=0&tN1B2yjsFpE1%l)2v3`6KI?cu#qi#7)@o!YyUMul+~CU#K2fUR%c%9N#e@(lkp zmU-+B%sPp2d-J9`yDmGW1W?vaGVVwX@;!8~mWit2`BIdBP2E=SD3afNu6aN3~TNh!+M{BQfNq*eoeQM-x~ za>+$lb=26qe0CApi4WgC?5$tmn2%2t@3)l_wVI#9yHEsFDmHMfK0+TcRQ2jq9WrrL zxH?r(v&q?g1gYe(*uQ45h%c5lG3obz{2I<^ZLbH|%^(wG1szZX3HF$(^bK;iW>&T+ z$v`G5r5YC0B$ip-GVbggxp@4#rpy$kLiiG7wC&Sn9EK=%u&p=}+~w2-H3UN7Na(8; z%$h;pDqXY<&y9+r-?1qR(Kge;#0&z<>mFHWLV>?UrzGA1-O< z$WwBEA9k_S;fJ#~^2jYNA%YsJpQ)==sJ;h)zTsk}A%?*^&Ot;0jOM#TKE+ZDd@R48 z?*&68oK+_oCpYTWIx#_;J+#r!IKfKlE?#9s0S;a`P}Jf9nh+PBQ;_7W+vT}RBn`{6 zs>8Ja_czpf#wz=g1yz#gj?2`%213YZtKG8M;+1C{SF_xLX#=;jt6z7M!?Vo9)a+oD z5^v%#rkaUaKGjG&aU^V~zs|k&Q%<;S=)Ry*jFQ%z@vM0h-ZT>#6)Lm9d^0vx{4tmF zdoHMF0%L@*I|s^XDEDQA)gkLeYZx+P&6wnchfB2e@)qU$i0L$ThG>BowVMt9l@b0w zFT>clVP9;>W5M#lvVXG_)Tx+>N45mbSM*KfEEJD?8_LGF4 z0m_dwGa-^{AiZSnDTRP=Ey*CsAtHkpf7x_q*#cCRs-z|JJ%z3zyMBYKy6DJ$owd&D z;W9!V0pGjNe`8%E3yRQa?+G0>8YWR=rU)6S9Qw`Q9y(egwIAwnZm{f>MY6Rn-R z_d0gC%w~+Y38L)^Wd7ZEUA8pHvp48l4g~}(7k{ryIp?Z**YV_^VFUJzHG=u$w=v0i}SVALPBJ7I&O+-A8UZSJz^U#qlxbz2S@oe+6#F zlLh$%&c3tit9i*WaN-L=MN1%@&5Epz%+u#ujGc%%I>4SlA4FIPjWarn z{K03WtU{5@-)+s5v&VH@C)v6sP^Aeo0O%rLjzc2sO^`8?UXrfd>Sp~ z95jt7p8mi7wDs46Sc3ULVlzmyB?pMC6hWWFC#o$cATBii?nL=!d%u3@R_z@(#{Qvo zUvW*cgvm~HoHV_7|2$l> z;&z>p$}pD9{~g7?`HyHLQq9ex*ObnNXJ@ta2fN3~WPx2HwaVYaJ6|!~G#W_XpA$bv zE<{}n!&GISkb3aw4`VZ1W4gr^2lK8o42#i=+1l`vA{dbBf$hk|NanW_PrP#-wbFax z(>6bB6Y(4_1yf$PDRlQ>81F?9&di-m0>#hU(r55Q{^@y)Dm#v*%}@!BcV=k0jlXtR zR|yQ7V1X=efh{p4t6`X z8GC-OkXO$ws9%hVaiDC5A`LSa|OX| zQPgZY$%MY(uHG-2z!MMfNGHR@_HkJfSqCF?i7FFBLE7FdHU{9M*Y@Gmnt=U5x>AqD z#`0hvBP6;;x-VnDy)8L5?oi@&8<7l*Pt{;;V8V)Qdct7@-nYsm(Vq-LN+kT{fu;~k z@Qr7a;5m@&xrvVaxmAHHvI-dIlvNU3hV5=ng))E(BxG*y3*T8zmnuCB)hQ`%s?^of?pX1?wkl+6ChDOl-I!Kv)JAOfgIY9{W!KGK0M{J(!%S0`g`^wnWC(pOP z0m_kPRj~^!H#v1O* zwhB>_n(?aB1-&Sk7Xc#WTv-97?-F)bH6d{gb`*M0B&k<=k5@fLaEqo9N1&16rVO(L z+qE?rdVz@CPw3I=zP`%hyM|cSC5O~DA7n{=jJ*^ve86TtnuUFZbo1c{V+f6otbf4L z2*F?BGx|06AzU+!$-3XRnJODukgT;xfettURQdG>nC@F`hxh}E72@qpo9bW5|G49ig1o#MaX zVk7?jtA_=kG#ckFj20C}uZJ-$N+k@5BoF9K7iBqlNg-It2hs@luR zz8{jdUU5X2b{CSd;GVKSge8P~c(h1IuJ&S#YX*@RJj_i*|LgfQ!l>(DCCZ6qQ%ps6 zq-47?<*X~?_;}U*Dz_nHv~jFNi2FPiyHzygN~9bTA4APVFp)dG{&O46STK#xm`v{N z992&8`=`ANR@q@_wT&JIrqdIbF36U2c>imIG{|`pB<;oZ`RNkG z%me|#1k<}btKifHqCB4U9BlammY6({*eR#Vsnkbjj z6PO}wa0qu;wm}|D#pF%q@fJgGP*_BM8$v+H-ZPK#@tk1!Yymy*SS_A=A*+oP98H$b zzdy6Z`dSMQe!8CB7EK!^soe7+Od9+3zwT=1f85pTBpe00QdSS-TZUQj;{*rQd={a) zky%7|!qJ)u5^rUw@;R^gP5X4Kv6zJNm4hH8BV=`mj%_V{s+wO#!^|jj4$4h!YsjOX zPvdBner4vn7WE=qlxp30d+6mw4a=Dd9-iuJp?iU%7h7h%)qpTJ6*fl}O5e`;Jgl%; zA|)LLfe+D8JqpNIzi+{m19yaL`^}n+h?F`#Hm`N|66RJa&GHA9wY}VI4z*v`-H$sl z^lvLFd4|o{1?}4arO;wI3S#uPEvIiFRKor?+8?5+DA}oQ$xg)?%7V7>$SvHWP^>=I zmT$5AwV6%8PGcrhip20;%wx?r&VS%d6C#{80o0v}#A5!?Ne;|7!Xsu-SugnQk0{8{ zVZS?TMn^8}yTIXYC-Rk0yE}VD%F2}o2{MdL0Fd--7)Cf|r*zxVG+5Cv@Ldn@`wh$Z z+pdg=GE1%Q^P(1Lz3_@F`$!leYb;3)*<1P!ph(VVT9h-5g3yqPYxKlbQp5r-5QN7SXl&A{7>Z2gYLh6#&JI607(PT%z zx3~*SQOQOZsBopq41L9BrH?s8xUF{q`eLs66JF)ZUh!z5!7%hJ*pv%Hz7G9NQJ>tKt7YEgcmP|L#sc5ssneV&g4HNvYz=7@*S9 zf_m^8ZZqK0Qb4gV^O6&eZ__-Tn^!h(|8OM0{6eSh{q`JKj|B#U=bZc z`It`C8ox9C%NB_Hd_9&%3IRpTT_&A)YMeCa(Mf$DAScbl8~%Vgj4j^q;!2jF9p!p) ziedmCmmf4%aJBcmkGY{!q!emiIi5cTvABOpSpeZ}ev@E+E*PtcR zkwn9ktDt2SD+NE7g)&_Li_eeH*U)&DqcdrQU)jftIRIuAg5>ynm5m&+f`*AQ{fTho zW*g_wD7kJ-^@pwF+CxJLk~zV85ReV}Oh$OIpK8>?K_w8nFiD=oY^@`DAB`cnt7>g5 zpEn}}5rPErrfQcsbRHGHZn8cHB~;c7OV_oA>kM*+ZP#zCMQK z?0w;%rd09r&7HxhWc%~hIVmz&Hks^Zj9X{%V95VI$x(rZU;n-GTUWfOho;bgN0HyI zO>u=vc~U4`SFGV2=UXID4rhP2Y@sLT3%)%!?UqY(o=CmK99MMZXhH&NM70xeF(>s> zM3=G1ZoPcdvF=n$;2Y3M&&W5O+cGu8uHsISOrFZnG7U?NcQ;Zh&zUI&t*M$nmXh~E zIfJCjzwVa`hO=>MR7AOJRh>}@owUJ)pX!nlW=Kq3niUhag&MRE@Lmh6A`a|CO48AT zm@|hkJP40-b8PW!hSFc-xxlR5aBi1s$tqacv1Uh2ttyf)85HJb zcoUaYKpwlH*THlYIG*6ch&TE%H-@q z*echW7EE^a`Q=k?E2yVKDJOcweAN$bzzs@lU6y4ksBZ^?DCSHFTPu;B<*^z4aXh^OjufZ&dZ{UVGW2rSVb~mUu1;gRNC)%kRabj2VPUjIZhZb zXtu1rNuT2)rXVaOX0$pMlRlWm{$1!P#PPGI*-kZ1L;%4V{>0G&?S67Riccm~PJ+RIVKV zOw?a*gm>?7Z{xo-7UJFdp(k?Ti?a~fKUOUzmuQSn3B!)Z669plf$Y5~GXEcYZxt2S zwyui;flxSwyH%mV3GVJL0fIva1PBtrT?==2Cn0Dc1a}DcRJ$g2#p+%m(0 zD3R>(!oaflTOK+KnNgkg|G-}ZLq!k&gNH}DUij`<;i(;vsf)@ z@48vTrB%I!SlS2HK#CzPlXNY!+YM&Wu zv&L2#nZG$s2q?~UW}dzNrXkMwH9YQnjF+PmE+m9YI1-9?cckoNWzp4-3wnv192+vT zBJbk{Q)Pe5=))t^CzVRdknJ=SlD5R@)5p0B{IAr=m;IQ=?40Ws;{-z5(kNV{*DE~1 zIqwFpwfEc2JHWJ_=2p|d#{V2P@OhoT5fY+k3ZvDQHXDich$TgWq$-||#bpu*9PZ^O z(eHilqO@?r)uS{$#!FJf_SilVbCS^L7?6dk;b9G=rtdSB=<^IoNhMGq1>&C%nA>=- zG5Sr#tZEDpWP}Z*uD?C1oyjBg`MqifD7G-yJ+?!}@cVUGRb0lZIi=|AGa}zOfP_`? zmb|nBwe$bJw9w0%)v?8~GQZ+&oui>q)SIt)#6(!3qbN16mj9xtfW2yt_123WswkFA zvo2+^eEIb!o1>rsuRzYlvuEcMs#-7%Be=v!U1$gT`SFmZbafNs%L2V;OpO))O)~tY zXm$e2n0;l$F22hhmc`u*c?*_lue=oGjHiA$4*+yEin!9H-BDx|*6h~V3#;5k5e_r|v%)sp#ocC) zFY~m*a^nK`zXriRCW}z7KI-+NiEmgdA@T#hRq8>>tRDBRSwQ`sr1&sKc zVUmmDE2oeI$7g;~GbqFcqW_jM|66xQ?w|INVlKR$EJ7lhPrudZX`h_agv>Z6c;NQ? z2LwA-jWU(~a6%BgJX=ZE5UoWzh2m8ciaB)QEz`kL+2^6g%=2D|)E^(3Hg5kmf* zv!Wd8z^0Cfgy(tv!DzxAbrM$>&-O#Zk9E5Oqj6)rbyz3V!285qbR?CnS_HXSYWDSd z%gLJiFcQRvW)jD`$~e31B;vm@+kcI|{##^2z_6rX=ai1C(3<{KcVWOVp}=Bu+Go~Y z;K%2xq8MF47=C~MZf!7f%h~IarNC+C!{;9y4PyA&kTD-wDywNf6gK(&k6Xzs11Qd! zy=<`Dk8xv*xfv+Wd3?0AyKX&lzJ)D4PUv~lkgku;8Y^ita1=QIs;0BZpqI&uruU|o z5c5YDCtGF_EMM;*pOeJjWV(@){bz+Bs9%BpqlYqf+yXC!cf#Z~hqT1taqM;NPgWd% zO`+8_*tcA>shk>OF;hL@BHN~2r&;tpC#E)68zm4~$x|91Z zW#QF8ViGdk-@RWrqwfg)fm#jjGLz%#;(Gp%>HxmVweN<)m2y^x>G@rypxlRF%lE?t zosy9Io3@AN#w&2)*;#SFC0RZiO{YO(3Ay|O@EJzP)>&gX@o239O0lIztv1|--sk;O ziVK60cxn7-ZExf=WTrJt2C4SnsdSt+G9kw^|7d#tb?xwW6$bH1RisMx*afklfmY{4 z!DU2#3bGP}5}%Ook}AD^!sYbz?9Q-Ikquf8mH5v@D5%GgX1Mm~F~{O}JL+%`C;x(z zG-8-`PXDtGX1nSksfiS}=NDf`y|I0pe$_VtB6x}ts~V@S(n7yZVKvqjMiG@vL$1HZ$fx|%ksXz09G0P&T21C0#@@lfu=N+K0p|1X+Cj6^@ z{&iP@8Qz2YrLwCHtroK2Y7jo4d8H|S{k}HC`254O9%8v6|Vl);>YGG;hcc& zA*YHJfe;;>4|}1Ev8g0D%#z=u1B#Y9B0L1K>yPXrCm=)%Zl+4(h)$HR`rh{WTA`|1 zv;wlz%YJ#W6PJ`Giel!+yxMOE`Z|?@4OjoTxoQWsPb|yQg-&lAow(aJ`+TaKieD{3 zH@g}%y+5&4_e|%3WQV6q;rXz9Lyvp*1a_1!Qcpm6HK`S7c0p9DTD=L!`;?$%M&$vf z){GwsLof}IHdme-hwK-Gq8qxcX*T%d_2@VuOP>JZ_M>&G%Lc$j;0RXEymGE)TA5Jz zJF)WDy^ml;(r$}@?zHHbYD#eX!X@%>`R2?=lm^vxHb?itl(yR!f)5xbM*+aN7S}7t zL`o$T zO0pdnw#IhK?7vB;3Ek5@8uXd4WIM+gs;1-PA++A*-PSx*z;yP8n_}R)wkIaX;WX<| zY(+k-fdt9WFt3|G-pr5mND=iBGICfnv%FGBIpzY2i$$~X4nPZ9U3lzam>}pb!0d}@ zcbem0wsE9cLMRW?HX3|ryIDLWYxF(4_|hhdfsCokKJ==B<8Bl12>?EugGPW}hKrI1 zO)hE(&}M>805?tcEcwF9z}_eyqg`rE z5TI*2Tn3yL+a~K|(@ue`0p5llQN$@ppe`}&sn>2@E@$~W6hU@Us`8=2@+H#K$|7w`JMX)tsMgxUfaSyar7GJSsU#10@*AhGrtZkGev z)w?nb3tvDWTvfjsLe3&}9f_^;D_xN@awax_uZIi@7g%?ocCrUMbJ*`6NzumyG9#ON zXEJmmXVMStsKS&63(&IPSXl&hxIMM<9Vy7H)cVV*`Cm2cznhD4e-Nu7Or|wGkuXeA zUcQ%HUpt-z?ZZ=Dbcb*~ZvC&XL#)P9!M#tRM8a=8ybdhb%6Q2_Q)%beVKWcFl)@7x z#nXtNd68o56~W(oyhv38czPB#82HUs1F3IU05%}n?V>ivd%9_f5X==6-)BjEqkp3G zsQU;9fMTVw1)<63bbSkUl`RL(F`oJjP}Q{TH!0G{n$R=!}u9OM;dp2=KR!^6r>r|Q9x z!XQTzqxVkYacduyXXzZL+W4i>zH3$8Mn5D*2j7`x!(+NBJG}?)83U|DU*H@4;>Hnk zmXl;8?{WZa8OLko5xP)0>6}v@yI&Pud^$)Kjo~ByOxnwA*n7R7fK_QSsp2iYc7Dx6 z0Eac!xy`)37ak+IIsw`KpxH0LnVAwrZZODlhNR31(@SDZLXnRmdCBP&6srCso55d1=B@hoY=c=2LrMom9_^Ig(ASJYUF%T!e zp357pAF{d|h=q}Lf!LU=Ka?bj>$90Na`~K!N`@1q_bGi0pe1ukbJvfLhOAC{S+K3`#X-fIY ze%ex0_lReMhOB+z2`lVP3^CJFkLRU8m=g|U5N}(>e~hBiEb^9lH;uQDiqL{iG2#U` zvn6fvORv_Ze_%O4L1k7a4bkprKRMXRGaetQLP2fT@=$-bSFVEGZP<)Q$ z4+ke#Xm{`r#XMk$MM8TYPOjDKFwZdz75yx~k;VdMPOPW_0(qEKAEj%gx3)liT9tIK zZh?T!PRmEdegQkn2W6xyU^~gu$m-SvYj9UBcU$A%K$ZYkp4bL0j^J1Tq8rwWRuZPyUB)pFgR3XZy~Nl~T-QJ)Khzt-dLvk(2BklkN<7m#{T zWkVld*l{qeuvylph?lYd%mn?XRSBM18k>-z2bzP{9c?=S*R2BkAsRR=uCRL>a{#h` zqCug#V`lZ<+Imm2Siofki6pGq+Vkk((vk&q4+X7FlhY7HpyFMGZ5iif?zMZ!?3_hkAZ z;39<(?-8D7$)=K-^eVD2tP3KKt_zgMO}_}ydRcu~@iCM<;^s5f?GsPMSRhFK1fNzZ zd6Gb)LHG_HAp8OgM(h%IE=({0mczPe7^V}FLnCm5z{mnhP_)^jyoJF)pS( zAm*FS0&Tx=)^Vx4Xn5f;<;fdH$uVRSb!asVhU@V5+)xEe=y--@%>|dVVHl!BTkj$O zmFmSNI7#+fdOzcPbp|1+^o0 zp5{DwO^4-n`B!av_Ex0!V2az(wj=4Mb`n3=!DCCW6Bw4-L{<;BTFK%`qOIjp@D5VK z$U)e&%0UP#+LQ{}Lm5ss_#0~u-!bf1ks|cyj3t7O+mk+%KmhQ?E z#A;gn_1NmCx-d)!HBEX+J321mp}ALiVx0u5EtgC4%!FfLPoTs;B<~u^|CFd?(_((& ztG%Clo(HikU6mf-WlAO8bY2PNdBR&j0tKhr*@<6%9bIjxo=nE)uis>bpNsYWb(n;T z0gA98XsBVzjx-%BFQ-nlfIMAA#zge*dLp130(ahVb)|+BNhnv_=NnK4V!yqGZ~xfa58cisE~pCiQ7n@1>NDa z&r7;1?n1(96l2CO>J@5!PBp(#f+4X4h6rendDNt>Xt_uf-5Z#1#Ee?e`=a;6y zRLDL7TZ!O#MV&W^Je_X?(KxQi23DuXh~Jd0c%5b}LnEc*wO<6&1nxLRnW@Z1xePvE z;ufZv(G2KNvGf=DggL+?9MA{NnHgt{Jo?cm>|-${FQ-m}m|J!;UPWbd5#Qj0;y;?Q zr)dMDwJ<<2j0kS%?V}}K7^CgCsm2v*nLSb+cGP3zyKMm2(T&KPSt2#$v4HOkfww7U z4`2RN;n^S)IS|7R*Bc8VJEdp3_nA7>jli@!*shy4nb)}>CJ~BS%V-MG(xdl-ZYX;9 zkz#ofC0{0P$1`^As~j|W4d~O*9LxR4e_|$+s@M-W?d+x;$`6?6GxO0D zxzWT7tKWZVS{y#WS#Ee2sL0;765z2AN4rmzuwCPIGrzo4#2BJ@6KdMT=VxYI-`mcp zeCoUg&lxaycjEOJjAk#R0f99n_$^k7YB-m5f*UPER* zXiQqz2Nm`E-xVr<%{2j}K{ioKBAQrZ!2I#he|7dn>0-)&!_uSkDp%;}d}4*aGc-^0B44_)0Vvols*P$^*@w0-WJ==H+n8;)1BjAxl z19m!G;XCFJP-E>@073c_JJ{XXtXWSy;2iS(H}u&VgrdC{pe zsO^MEbE_Rc!Voi*PLPM5zu(2^wmh>h9`2<@Eqv6q2)+Pv758( z1%H}z0`NJ9wh?c+SCmC2l@RvLGvOef6jr@YplUC+QDYQ-6$YZ&j0nb8EPk5i{#lB( z6p!n1LRvxx%qq;yYD(j%*y**$kBmkgLUXp(od|_u2d2GN03>-`f@V%!F6UDKKpl>4 zcSnj0ZlW2x7}e#a=CzSej3+6b)<7LAz;f;i^eUjfcZ`5PwQ|tH)&LyyUT**ER`zA{ zmT1~fronFFN5&K_=EeQ`NNPdnX-Xpd^1)Snit_Oi1gbBlE@AMbj2L>eM)@g+CO|L+ zt6In4F>DU%!VbXrp58N!$Em38;T5EmVA4e!*q#&h`s-pLXZlp(c44D|HbjkG1nI8O zYv&JuiGBiD|25IBa)p!g6l7=5y_<>oqfBD*0tqd~BxW77TI(mY-5%S=`aG2xl_q}5 z`jtB(=ZNz7p~AQpM!Ts7Jy$Ayf&w1ZYALx=HcF0{aQw?0@A^(;c`=sq3rW9+-MoTKanA`236g^vXS@;a}MsP5tT(l_Q30ot|N)>m|BAy$zBFV+);M!sIH zITpL&)T7ZT<4QY$c;cI18jGAnjD$0UN{e;Gu1>!viC?F+cva49)sS-z7tj=Ey#655 zspb@BfCP%?N+KBtsM{Dly*zaUw0BH&n{RqqlLY8$Ee7%eRwDLRU!z^j8-mqbsv7?ol%!({x(tOC(U!_7sa*575w6=Ab#(2L~3CsBL_wr4u0t^}5m|4NpDh1|fEH zi%H!|b0J61)Kn{r1{sCDM=J1lhi;|+cU*5AMzIHMEL+J>Vt zmH=<6syx>KBk*`$C>2Y~D0cLYu|4G98DW1#Z2dp(<4u|>o9VidYToS`Sh3) zGu=1SfFsvL?j=zvH*%v~2Jo|wf(SHp6niT|%bwdlV*e9FsDZo*8N(Y8OtqR<*oEk` z&@g5?1vF_M>nfhj+t&`mAsHR5&a@b7!9P^+aJnFo4*2y4F@a#5N0YcMc#5yfC7NW< zC2>4rM-SwlBIV8h;f}&Lm5-=MEfE`;Tcr&7U#)qA@#Ym526iE>(KA4IC?uwpF)w{? zSYc-@y5v{!N1j3g>?h$Gmw8GyL!Mt2&lFecx6+t!4ixwshIXHV9{}H~@p-qalj=`T z{)^Nz6qY}Kc|CIE^sSszCdNI;1&k?iMv%Ly)~a~0d`&?BxVXcTAA8{+RxV(1|0^Y} z`SB~dSrUuWFrhHIFe%u#eoQ&yHa8Xs>Fyhn5)O?&LdbMgb2L?{^zrs}VH!&$8ls9rU<`-q<5W6~Pnuea zK>F8gc0}F?gV3?cMRnS2Ql0o)K{CE)W48dKD2H^PdB|EmE?NkNyyF@>m8HnuxA6%L zQ0zo#f^P0JTD-oPOisJReKh_?msx| zcxk;?++keOHX^}LwC}E2I$NNRufE%8MwN&IJCLk$VI?$8*vTnuEuO@lMQXh?$EA$1 zd@m;1TUU2ghD_UgvW1bN{fAaU)~pd%e+UnjG`M;fPcqG=e{jf;D z(~18`$k0lUEt{#=+UbY1H1!Qv6Z(aNR9@1RrBzQZm?0Onho#|p-Opzq|MYOfvoe6T zQzkQ9LW!`9+3ed#=kfTB96J^@*1eaplr5)}snt2RB64TQbVIVBJ2UJN7W|=N?#=*& zGnh=C)r^W6sf%hm>nHH8mA&+y5v-g}Elr9kTA%x7wy#6VdUSHa3vm_3Vz(M*z^;e; zvbX_K9&>=cvE|8H^2d)OOX@G%G^?}8YEEVQJ|qwgsn&M;h`2jg1v&nfG4x@kPwWDI zR*M*<#UjS@)X_N2E*IwQV+8kQ^Nnaiv7 zn7+h~|m-Tr>QcFY-bJTEyhN{5TN}z1r07C{Gy&nhxOLLdK*!BgQuKkAf zXaO9z_D9(O#F)JoPuxwb#`B^m@)qD%T7aUO5wAB*)&}`3i1=xZ$mW;U^H1<(H>w|% zKIc#BO5mY$;i@TLhh8MVlsDc4+Irw*xTD}jz)f&>KFwI%b8}*p_&0Ulzd%T{a9sT) zJjHsQ$PFV|8glfFA@1vf7_r|d9B%%1*bv8<4FFaz<8M8Vg5L+mYc7<^L@C_bvP;2~ z+L~na;MqLh_8zW}JwCs=R5-t(R|_5-(wK#JD1>C6DbxFK%|3;yuySt$a_V_0r~k@C zqeDvYvCVPFt#D9%BT^Aw6z(#K%%*uw^+cwT2NQYh`*e$}SH zI6a5EVy%ERL?Xlh=qWd2e9j7{n>Tr1sjT7(01QyH$cX}d2|VHDOUo~Z_3x~K#Uf*t zF`{~~AojssQ(A%i2xtg&#UJW?nF4o;r>y>ErPz`Mr@jX%a`6sg<5V|_khL0sKy z!ITvANpPFNp zU;l#qIDdi4-@;_{SoxC~Ay$hg4;vRDo`)@8YRQX1`;B`QKxkeIJMg6n5;A=Ol2jDl z9s0BGx;Qq5BLU%V%VSqAj(4S|xj%ogLMQn%ms7*Rw#^2jYGlGgCpP^GpiYn|wofX7 zGiF(sKe!tpt?mKxiiQXj8*`UuA<(AT{W(LrF08_wg>t{7hneHTMuw(ZGTt=G+C2Kwv?C{;ZG25)~GAJK~&jw->q&$K%r&kCMa zh9(|wa1I0t3S(5K1f}%%)fhAi{KI?s7uiuv>7T?N2PI4qhZL@X6l-*5(bC~zWnu7) z!3*XYSnSjJ@{s!Kwd60&^lVtKQiFazu%J|1lGw%NckrJ4R z^2r0a^0%0Z-PB=yP7L2>TgVB}f~Z)+j=pl*DbUDmo@tU^42;fX5k@DAA#Qw_W{s6l zqHUJpHz`*RB9eWgj6um(+da;t))<{qO?2VhFeZMMhf~)j3r3B?q4%2C!WQ1=tYhKt z2l?ffIn>W?s<}4)zmNqv+E+Pbn8)3yhoFH z;m(v1MdF``>7*udEb~%PP3+pL&&%j!*F;y>Pczbtr!$2?79NX2!G@NEtb z028O*0z3CT5N7F9rfmF;@^k$=eH3F=g>W$LjN5vY!*UphVe#2(T1oq=j>j4<-H%@% zJg4tU(aoS=cQI1Hi%?y{*P`m6Ja7tCi7PCXc`;x$QSSq|oEmNT3^C!cjnf&OF!ppM z0%u$m^ZLK(P5;#jroKnkYn0)xu_&t$Q5u8}b+ldr;_1ez zx{KMDIUh~VKYVtY;JBEv#JK>J7NH5ZK#bHa9EsPL!2SCCS+K*G5x9(?yO&pr68a^R zgyE!|y)pKSuZ(w41i{;|V?08YPAgnPI+$}_uTRLYKTBFTkRF~_3B=gNLI*#dD-P0+ zf?U(J`YxlMYuD^;%vf{r)?S4uen*>;pdI}J98G)Nd4V+`sAvJWwm~!$Y!{a19iUKW z!RE$ZfGTZY2(@_D56=O-VYUJIs6Z$zt$Ao!6ax2F>QW5Nkyz|~tE=>Opd#(4t_`fx z0;MvZa}vJ@IdXiRRvZ6E@6TE zkf^QZO!$N}DnHv#;8Ckvm~Azy8Pf0W7uMUO+Ayu*1z{1bS=CwXAPjd<1dIzL)s=ju?=YAgmM_sNi;z`8d>Vcy`>Y+hE z;)Y>bikKq&dQv+x09Ho=7O?;VG2cH6Tcj>inXQnbH|JEK8YWnE&O;MpFgD-jwLo41 zqSr#(fU<8C%sCO@r<+@lBu_L_uF&)cjD-nFJF;ErkS@-WasG=Rwk{nZPH-wbEtn7k)bs(ulFBSy8s9*ev;lDYZ?wVWyBHRfZ58so z5$_d4v-3508!P0wl34uQTuQrTKfFd*O>rVqJb_usBq1C*bkl@ryf=@MScsdpKn(RB z#vRUOCc+bMAuhjm+jV+Q97SM^ZuU=*wX7Mhgjo^J z9Q3)`Tud^Xa-lbH;HkFyyN6Pox{XfkG3aEM+ z-pGsytYZ?t!BtRerCI1|HHPlv*7HLLyu6$VIt9tRLXDml1F5?a{V`&Q8Q*>uuWG;4 zjv=jmv&IEY;}oD92*?v74ASR>7Q5tqk?wW^RW=h$y_8Sag^cj=3xkavsjoNKEKL6oSxF^*)f3 z%83o4F`AqaIFrx@hDjC&$XOrvag)N>uk#|h3QW3GV!~8+CCU9Q`5Hk!=d?qwKI09g zqUWGw+6R$V;D}wB5$9?EqK)&*c}MZuiQw=A&h7DW?x#K2LjEsUF8&M=!M8yVVi?Mk zr;$3QLclio7E|iY-WEJ8cVoOe{+h8*jryIZ^geF&r$7WTSuYuG_nv6zYO8agt&(Ad zsYyf=P-K4-i9fR?p)Tu4JdKFLcq*nXa3K0~Wb7;XV@r1(EQY$Z=*C%@ur7s=Rq`4$ zB5g?S7j(AH9Y|y}WtRdTHfr3_8F)~~TK*Pd$eo@irK#03oiRU%=h zDz!>=nbFz@aLXp%)LDI0|83SQFv*QLpg0AJ@M=HKB$h`5oZ76ZXcqOs1IALk-Nga; z`c9=J)-1~8oy8*nz2Kqrx!-PotWy#nBhJlw>!lD|>i;|!Ou_SZBA zc*fm?Jd+15ki?vk7&U|ktbQTVYwFa*$HIIWBx)x-^M5V@6Y2iye5#FvU{9rb*$7?j zdVv(NZRn8aueubDL9=wFM(lK}ImWEFc!Ul>*Mhkk(cB$BpU|OWE_l6>z_m!V8;Oa1 zkKX<>kr;`;c@^syog{@*JE8}W+e+{ozUGqrcEUt#Qiui=R08yVW}%od*5|Jf&EqM! zpxOO>%B;C%c3~3+xXzS_Ea}YNLx2F{mj~7lf?|>}GHc9U>}h1{7Y}W7L4&59uH+F0 z7I7+RkA^{ZpjHy!L(k=ptWqxPQT*@6dhg?Cc&4(i7`qfWVHEwC4259z{8xW&cRj%R zNr8Oa8)>Xwe3$hn8>2wd%@3g6Hy0L|;JsM#bd-&S2ergI5l39n48k!0Kq!0W3aA|! z#jh1->o`gTu%)jsOs_hD1i5N}n`k;ZhbyeU%Tu=iqP*_Sgq&NeuR`esgoSI?q|i-a zfegpiubw9Ga9(%@gLf}awmAf*u?X>;F%R`#c|@&&Tg`^D1=PpCrKx{wuy|zw0o6dD zcAx_r%vG}WmxJSaw81-S?rlKEx4{+T9%-j%uS1&hJ$-E(5KX;}vO}l-ZU|_F0kP$C zG(}OVI20ba+(Ob_Z3~)qW$W|W)2&P=m{qC=V4r);9MT1j3At*WIECNm^y%heX_boR zpY#W3WTc(had>#f)`!4X^#)2}Y)VrdkuX*M#i!EqULp-7;(+d1iULS<2QH=8{Dg0I zU%KHx_R}nYaHQ{zUIoRPBp-*K_u$@)7F+V<`SsFe&F0H_UAz1L}pW zs(n=Tl5-k)8I>|EC>8ms;VCHB=4VPeovXCE_pUa|DI$&=g3_fn4v^f=)yqq;>M@>> zOd2Yy}eAqxT^?4riNWz#~d0|c~0zV z^=d8~qp|NZAV{EV5yU0zO6!RHUas|1h(E_ykpgowm!mL1DDH9>0*Zd)|z`b7=5;;WE)c^;VwZNq~EDHBGMhy?y2(w z8|3bnIr|jC`5#_R0d!SYJAHC*?krH2TIA%{qMm;2|5<$}=vy{HdIMk zO)fV}tmhuA{3<;xKfpaG{3N&Yuy)==;z~r>7H8|idRf}r8Hs((d1g(>7E2{9cL*L| z7Ig=v?*UF+qcDlMtYiq62`?dBXh~9L4gCfjOTcaqq5bvh;h0mioDhANAd;-J#dBi$C8 zl}TvsK>X(_YK9E{sRn{xo1SW5Gu2tt_>BY#7bmN2WtgkMV<0_8yo&aZ^=2z7x z*66vxQQUTR*32)HpC#DmM|B$-z5UetX|}@-f?_Zthx#bLe>9iEPKzs=$T2{H8e}z1YVt@EIYFJo+gkZ1-I~aa%HQaSQxlE053$Mh~X3S9FrB@OJwW+@KQ5Ej@}RN z_BYsRIJjL9YzutqWy+3#+|SjNK|o9aO5Qr?xg^8hGDBY1S0jsLK%w#vP{tCUco$Ma z2W<|ZA}-W(sS5g@__X_|E9CX_d`&w_UIb#a-CHHGTMt?a$NOj-%qSQ=Izw#%BWksj z3-+ev3G*aD5m}V{zL}MuWaftMM>ClW6QftGZakJr_U#72PcW6{I;4ps@es{8)y(An z(`!XWex<)c;esL~tCW~==V#i2x4(EMBD)s$>LNyZCMQ9fC&;>DaqR`W?#UNNTdFFV;Kff*L^WMbWH0uZ(-3uUewT9%TdFGWTQR znxIDtBb#~rxilMQyC140SkTO$vv4zP`+c=ObrZGA?fWZOt&)t z&VQ8ozbwV&Ad#&#vzSHV%Pqr8P+H7?nh*c_IU^mC`~~3v?C3q2;D1=+{#}8{Xbb_qFl2r-$(W<*)t!_-`@pgm0wVRa_rF+&l8Be}JnwP*FuL z-{ib`xpF#NPz(=ycmDJkiD8Wvh&qyL1RaV#Q);?U5W?z3~9eftz!A$AXAGw?95G3Jz)eR+X z354Kc*jDd@pVaUBQ&^!YH`#wuQF?}O%B?kg4E{K1Ju3Ue!U@b^e&zCgoSo7VO!Qph zsOU$XJoh`!0^euP?E*+8j$T16jeZJ^961%HNs4artJdjl5tCG*TdvZm;L-KJ==b9i zH>%gzuf9b(-<(cOXOFc@iZlsE=56*sHc#4bt}t&L(ZmUskt>$=5H$`}XTo{~Aqf?W#Kpn&6cH;)l zn|(F#IkgzC$;zXmJHVM9AK}{c;oS6NY8XVhby${-o_q<7`jKGU<+M{5!gk8T^e@RFxi`MvjL z^2ld;>n?uC)E{uwbjHKhh?PS~{V;z#)j4ErtClBN>m_j$U;Bt<>eQ`m74&vv)W=x@ zCd-zJUc(tnb&e8wg!I+}xqGV7Oh5T9h+4Y2DzOD~)J~4*bLM%N!~#3rKxJ-Q7ja8h zoe-$T0(-T=2Xj=-tty|n)_ixEj>iT`ici5@q_@sgHJ#zCI@fga-fQ&Iiw-kQ_nFfz zcBKJ^wL|u1ZXzyqorJBZBr}%$}JeVy99e9s_=zPPDvz~xlx%GD`B|Sv13DlkxVhHI^kZOr|kNaesb;W z!HBpDpj#g@4)Sc#F3J$*7bEn%G(}~&)el!O54prn`b7g^-#DnGs)9ll-NZ!{ z+xQ!qgKE`JPO8X;=k4f7ynWx1K^W;&F(uleHQ{!*@q7&eA5#RG2tx5(UclbaW(?k% z`%+B~UiTOzkOc1iV94C&oji{nU}ks~vz-Jt5Q$nGr&63qUw}+MxKShnLgGd}q+qhN z{4yayGe0t6guB$Wkf*A(`>=MpZBGr+`?<bgA{4Vv zf0!<^RMS;X?;xbfiDh1I*S0VZf@(9jAlwKl5EPL<(#tn;;mH3$qmWx18H8Mb=2z$; zPDp+{OJw=j>+sU&Q2e$_yt(s9bLV{PulZtytZ8!ka=OsC?>Jq0wiRj(eqk}G+uiZ# zP7=PdMyP0Wxk0GwZ1KLoOMHI=%46^*E;U)xo0d5pQrK|=~hUGT;go`sTp2& zTw|;d1c0L~4=4i#d*+1bQJ){_o8YHc&td(Z)R9b{HiV5AwkL}33K?c?y!}O-Rv0Hi zW>TB%_E(PxfVxNWzGw1qF}(N*MEqpZ^llPHhdAux!p;2Id+Gad2`8xR z9B`etzRkIp3OM=ye5YX$7Z@8F|E3I9tE~#2Ph3&{l#&lj$>O(XK562Z(gxaRM=sJz zGEQZ`L|Co`b=~N%^@-gOH^Bju>=I~>?H>kj*K(;Cuv8SR95ln|Vy(atu2mF*5`!OR zmrq5PPe&haM>%e4IOdBjj43@BqHJzgsqgdZ_GL{rWcBV0w+#D$+(2k5J#ITwXi@^# z441_vlMPkZN4pQM?Fju$)owa(ZQdb1<6$MCuXv)B+TAJg$|QK8K=~!k^-n&(3-$(E zjt`;Z&mD<&!B*S@uG~;k3f1;JV4kjdk%l4V?HO#EaUSqvx z$^yA6=@dstm%6@Hm2S8E(X(CtJ;;uxW8rcJL{|?DWyYpLqv(9}*x*!M3LUb>rAnw< zhLS*mu*MpsS%d#LvsYKR4qN$E)_FG2K|Ki(y-oS$Ji8cSpd0foQvBgm9FAOn`UXeR zrE6rgjZLgYe%Y!^Lu{AeV}fFOgu*j9Ia+!9tuj;8^TP4r!qN11k?Gur+mXZD5yQbk z)7u`?&w0Li#2$IB?kTQ$D1!>+qS>$5IDFUneb)JXPx-5VSdetDfi1})NBrJD_}#tg zo_W^sdv)watUTm zG=@ijSz~(lY5Dgjc#j4UV0Y|g7me|~m>N$UPV}#6CLMfT`#-ob-e9`(>8pBH^y<>V~tdU@ck@+^MsC{XsP?jsOrO&e&6=68$wm${| zH}~^d#w9}AnR44i+!XmNL+|dsl^l|GO1MF#+zJ#J`w_0*dthI?f7vOWUdCIP33e|fIwL>A1<99yC@0R5o(L}s_m(3y4k0&p^r(U-F zeBMew6w$gJf%im$`;c;Jl~NBN@I;Iqc)vxd>Sg7MUQ@|m~pqJOT5vVguM%&zTT zNMeW3mIeFOGIwhF8<{&q>LbL;>-0+IyY@_}_Su9ro#U6ykOdM+V46PwW?#`r$N`XW zbW3&JFW+cf7r(Zk?*YGumv4uUV&P&C)_IQmHID3I&O+m|PH43Y!$!gNj{^;oEU$*m28(H% zoS+bQimx@rCT?LQLs{@_2(?K2iZ{mP)L)*&@3YcAY&za=T8dv-Uc3E1+rhXzTVvW` zW0RHh+BG%C(QA5wc33^2jdk4v+^6MB@8zw6K=|Q%H!tpcX5W;373e2s=Mnbp$<&@S zTCbzM2Dnd(UeXNFwwtE5n=pBiPGVB)NPREa?GUEE2omm?ZY0itL*Iv+wvoMNY>%JZ zk;F=6BVY#7udh&CEX@faPnHA`Nv?(4PK6Je`Nj`J3#T)8C~=UqKYG${4XjmB56X!= zN`;;nkAm&=&R%_+dS_{uRHaGx6PwB&9Ta+mDjtK_q#e<)4@Zd z+mnGJ?^f>>bIug4LjuZuvFQ37yZWpj%{~O$I6VND zs}x(s2LzRKNI&m;ipG8(dh|ZQ({-MlWJPpZ|Is=!oYKdDp2&&}sXyg%zi-F6_0!I~ zlwN{)B%Ur6CD;8kUAfJ?c^BCF6oLk~Nta zNx%{%V<1JcBxzsjhQ+=V0VFP?)twZ;bZXm)*TnetZ~yiY#F#)d0h(r6kHD4OQmulM#+zCQG!4~56|rC<7`z3#?)>eMOw75R!- zYFa_J%zMf&uay0a$8kA{=GwH~E=#fP4fS3l^TL z$t0X8HA;_s0Z^(Hd|u&ZbEDp?4QUJYgCkd)zfctL zVpzZEZkxY{Jmeu*-s`>I%V#562_Xt`Vqp~U!0adgSmB4Y0rsN?&If<+2fx{yy_tmM zAdkv_1jowqgN@NU#*Pue1^vPoYn$4~AXY?9dHqX*i&sdLXi#=2Rcm_T7k=UU+~+>8 z@+z-VtM{>PQR`wy)(`x^4|FW_s7Mb&Xe<*p8diAG0J6>dgaYf__236T*oMrjr)D7p z8--j3q|=BD|MXA)^n7Wqb6w(w$eF|r3CsmC$>qyn(vherdv&qcS{-h<;f4?SkPq>; z=SYMftq>VSpa_4;r+kVaTixFO{ons+Rjf-BAG((5{G3~Rs|nSg_=%sW35%3qm81o$ zpY)_B`RG8?XMW~qwvMlhJ+_$r)&&mAkka1XS3qotO(T5w$l2~YJc^jv(siNDtGU8n zR;|Ev*NIs1d8|5_h3=BZgjEW&)r@S@0zN>gRu&@o&=37kSH^Gg7H?r9I)Hv-leqEy zz2E!2h0Dn5f;F)l>GwnWNX>yn$|2x|0V}*#Ua;x+Cx7xMb&W8xS*Lho0XTE!%#Z*0 zk4MjCN{#Nr&d>YVb5!^YQ@cpAA#v=nA-Rb79M&FjQ*fEsedF_~pZckK>aaovJ^L%d z-bfMZ8*jYPk^$4=g;=i*mJ2FP*;3SPk+gyFF@@#8RL1c^aLfVSvAetbpa(t3UansWWU#mCCXm1TyT5b0 z0n^OPFZ5jY*b-ousJqSivK`)CnA*ET^`iTiarp8t|8lSG!H%(RV|@%_lj+fQq?OU> zm3$+D9KuT;)1zg=s|Bk@D~QciO?SQPU2QLI!sQaBvK-poo8RLA>xTV=^d zebh(Ub^yu-8!|U0a)<%g)h_2;2ja5un6@A#nXl#Dn0&=oe1)vq&r~uKVpQOgf@;~O zmY^m(wfu%D%Mr5y5+x+SV+8xD*oZIn)2>cbdetxth>`pZ+bG9zuESnjJ_Pu*Px~}h zdE7{UwIToUAOBHa73bzb!BLub)}t{JKoc@ygweke%u?a=V}CG0cZ^Oa8H}tPIEi5& z2?lUzyy>Q!3JdpPI9RxNeS*Bx9lT(3LzowBdMs}_;)itzYe)xDPt00Jw~7ri|-F6l=&02ygsDc}NhO@qEBz}Rc<5po$G z&7wB(&N)nFu$W|wW6uK~@BjugzCjUu8B9776?F~E1=%6cw*Gk9&%$OBS=XC@&-t9s zkt9bFg!2kXb!1>~>3uZxk}vs^<$%z!MBEfKPI>^rbT$BJExhN|avTT_UKeBwkKBnc z)&3X^W5xiOKzQ1$(rxl^2aQov5WIZ0G~3?z>p}k@RhqycYWvRPQ)QZW{N+&k9mFb3-2dF z@cM;bwR5}rQ1kNvCLKwWkP5Zh=P*_c=3PXPFV2z@KwwSm<0N)*qS&mXn#b7^Hb(>#`mc4q=5%QBi`IB$H`DQ03ce36? zWZ39jfb@3v0WS?aI*r1Zk?f8j8_obKTz2?q$VM(9YCmFdDTzkyK#E-eY0nI&3Gd!y z4oXc;zX3+5=?j^?sH_9P3ph%fQb{GjT*_E{>D5|OVh80gV*(=rCutn&NFOix%*ne- zn+%V>N;0kZrHoCX57i{1WYYJhbsl<+;L<%;C6h$K-fSf^Bno1$H(jg{ z)+ayt$&!0SY7c+~CUs@7#FfpE0HPqlj42~wN=l5Y*||sTc^|Qt{U{SfKH;M#9KNOl zr2XIz{vb-lJ8kUff9%J8tg&|h*HzUdIHMB_kszNp0=UqW5-ywj4WV>}QM4vRz)J~_ zN^=(EdF3A;f_z~)leob6N^U}WRC^3KK9(O~32Tjlx+535N%RGfM=m??s7)g+U66=< zeyP@>F6<*i<5w^pZ~V*zF^o7Mqaw*LfMx&hn^#(i9rRdy_2$uc`nI}-$MOwib(sNpjn z8JG~qBp9!M1cz{M*HfX^8u2BqGrq}$Mg~)TaKGdovRnWVa#$m&NOkpy%wgq3pZ#WC zuO%|?^y$+8bEYE=!2L+myCXV{@ck8)oW-tyNkg1FLZMuoJ^y#E%uol6SpDB6pI1U& z7X)De1fQ324?&D1z^je=F-1@z*$^~8M6-weWIc8|ZMVQXKb_!|>rq}52iau{Ap)iX zbD)8*XTiS1>LV;<{#?$<@_OA`PWg*_I`_@r{LTHvnR2Omo%k4HMk?wKX3I}Ta!T#i zwZFJzsr5LVYJ__92Y0ahIL@^N_C7-Cr5ZefECCX!}u_F!!AgIZF=l~++{M|rD+GN%=81c@sF#<>S6^V$~i~0uaw{{=#5g#E- z{NP15DAem4HL|(~3{XaQYC;ozfYu%(cID<3jVZ?ENW?Pijo(vV20KS1E(*$?|){^eiZ4>+&8?mFLD zC4aTH129J;#^p8C8@hfv?li`<4egUclDY7Ihf?g`u-&&r;`a6mh3N z<>ebiKf|N9tnl&#qge(Ek4!ZBtC$o(aDam-N?*sZ$9|L+7(L4>2Bb0bWQc96R=?jf zi}?=FiGn>uAz+oibc2x%B>P*K4#3thcfkH~1TvyD!AM0K31L6ObWKe(Hl_OLiA;g= zgPrVD!N|w00K}_$+E_M>qxC6kO zZDbBWe25YRl7(Fy?9y19x+EX~V3ae4KVnk>N{Yy2n@&L^wQvbeMzRWx0E(NQHD*m& zh=(-9;%QFv&X(mmRF89HOlv675ZgDazCo4Y-e>aKNHO6U)FuQpO2o-G=hzAu7lL5C zplM)=Rbhz?7pK%7!{laUFf|o~CCRy72(PA8(H$<+#b25Yb+tlDGeK}#Mwt+RR!PNu zdEOtRVTm9#=O89ZsHL&B;MA#8Y7+>wW+jd0P+NMYxW)ExJe)yi~(pE%v#CJafu}C{7s-* zL2SurZZwv$D;U+B5)d$FC8}N~%KQw25iAs7O?n)?-a+HhYECQy32c)i+I9;%YxFWM zoh6424V{(7#A4J_MlO2k;jwoEj6q<~`ID0pL+ToprDBgR$_MTUqd0SiOrdsY$AU~) z3=IPl9+_^EqJ*z{k^S7)BStj6OcFYy9%c5h!QY^*>sgK+rBk~c=qYL?qAN=fbglv1c;k(+s2eHysPE*zT%|~s3ci_^v|8l`SPz#us+OjR7!TxBJ6_{>Y$FIo3ySLnC8wj9$qz(s!NgkwU)jU=K$uGyn(Ea;D)~ zlwb^Gon-<Akpt|F@^_3tj0LYdIXN_ ziX_oG2_X8W9%I0C0Mo!w>ndHMpb4m_HF>Oq>naie^>G5#2#iMK!~*G5j{pYE4L97N z6@UxI;3N=6TF__|B|I|_AXBg_a#}hfwq&Z&BV)F|)#}Q?D=)uB!;T(fHscxs!A5wG z3?RWHVi-*x!5Ic70aDE*kN#er@21S%30LDan0>@$qmxWzCJ|rF7_@j5WRT`i*FeD@ z;nACMTuTAMLt2&n1s0npFfw|hv=*8{U~Jcna3j_%U|^dW5sN@^U~0lmh2mfr;*op= zz7X@rDIp289q;9Ndd;vtMYH4Y5j`%yIPl{=(e~}D&%KI?p6c&(Fw*a&%*F6jn^Ul0 z_lC!ZVbrn2BVAhkPOqOWKq{OvpOWuPOIH|_evPGPonD4OGPSy8Gne7Yh61v+wdF4e z3I$?T=>no6@NtbVay?oG{O~GwY)X6u$DJMv%lo{~`-neSN$l;n-wyUYd5ib5W#Jby zz{LkZh`*)(WncDXxF~fo7ZRCwUp^Af+WG}j1g^jSdR)2;j998m61?e+UFi4%_KP7~ zMiV;U?by&1On?uhNeDx7wvjyKV23m!1UYA`;rA;DoC?GT;HM=riffEdiwL`dG);Rf z4#*hBQvzW|ck2|1VNHTWBV+vmW?XLH2@xf=$he0BJmxWvv9Xh5K95QlX^8cRei0c+ z7;+S1zw(OKpV?Zny%iJ=p0l;qQ^>-{A#aYmB?`Qn%VelDR?B8IdnVI)Ap za38TNW~SNz8Q}26Q_W5Rw@ty@eh5-`$i|(Wo#LjcMDjQiiA*H}d;w>wk=vZ&EPDHZ ze)L2dGWG-5;LM2z!IJyjeu zwEGL(YBw)7eB|ppWLf20M3-|#lp(eHTRB}rI4*wWSAHcen&6cbG1Z8JtVk8V$9B`2 z9JN^KLAeC}=#T#Bc-6bRyK+yUENYUIJ)DvuFpdhi(2KSjG2z_rH#jg#og%@Sjd39} z>W%d1GmjjbW51Yi3awM{7I}{Xf*3%)XaKU%fy!j=kr0{5Jg_nDy{>%=2?=-D@Fl$A zfZZZkXQ8Cj2x2hdM43;_7~Y!Vf>B7Yz9Xv__~NCDUiR4kgq8e43RW!f9q%QNh`~@q zPBefXlTVHXi@_PAhF6=x#L(3PdKf}1UNCx%4Gch|$S8wCKq5;fSVv?8L3AEP-Y4d!;s}Tcg*wjk`L+USEd)q)1FFR`IEHCL?$=P65gm~OnU=L z<)r@}hxsl71E3K@D;h=EO~Q5pvMw4m$%+dXXAMyWLmJdZ3Kk!W`A`Cq z95swhFR#&9PlXhV3*t5>_7dg+^FQJ`8jDS7zi$GY(Cl7nQX8I-MENnGzna7gm#y`w znERyyb)sRAA|?|h8g82N`C`^Z;JU{nIo{lt0BB?s!>}O0h28#KCU~@{B}aq-Fxdb| z3(4uy8UalCh5*2%CVW~GlQXbUIe??0~EVXvLBjIjD)9?di>5omUM8 zQ;geC66{6H(GFVfxe=TDIBRzcb}M1JG0qJ(lDWpz98|N@NFq}APQW$dA6 z#@5!BS&aSt>G&ws%*gn6gPLSUEs;&h=2%`$Fpb8S}ePy@YwV*7_pY4~KG#rZT7xiOiFcPoYwqAbn1zE%-z$5b* zY-ixs0}{=8i7a}eZ079VFiKq}QL0H7=uk^bWJUopFdNEV$$&*JSxWm~U_2^l6gy@Y zi*B9twU^$atvHw3-?T5G%YKVqMmod_l)h$aLd+Fj@~~4cTdR;6JMzbFKo7ccV76@{ zOX$+G)>{7Bw|TVn5lL5y{en-1*K6pdt5o+VIb^$CyYqk@28{+X$p8khCty8p`(e1o zR7Iy48fqmu4QD50GRs(GjARl5fFd{4gp--fhIjw~KmbWZK~xg}Mz4@#$v2(aeVT$q z60yMAK8i46-GSIGhRug%8%#b*Z#i@gJS{NPWh8=u;WZV6gdu7?A~R$AEhH|yPN90o z`B0a^I6+Age;^Q}lAUV%R8t>;I@b#pK_*io=|TH8Fk)LGJqTcxow1FkTVrIAVl(u#olEx-_G>Fw!79CpJv(OA+-Zon0uzj9 zs}-?YlQHf_z+e|-PLXU5cY3!Iu*F1%OG>CoAX$cBG`m8YdyR zU3^fH97eK-r$nh0j}}1$ii-onVUUpz&Z&&732a_M%qn_ipoI)<;j>fsXVf&2AmFCZ z97N`*z(IwbA(7Ee=F*D)HCnlEB#OB`i8W4-$VM=+rU;2=k3DE?}#!=7N2MIiT+ z1I{&eiJ^SP)mwK2nmo3InZR`xy_Cvc(j)*&Y!&d5tj^#)kC5b~noNZjku<;|MNjIg z1vXEyTWw7#7{U9MQ>RX~508teeDT;o_NWL2j8&jf!+wLy7}R9DtGi`20#>OlPh0~V zs@5Yf{IH9;)Y8ORX0r#S+7#h(2S)x_Vib7CV_~#6=CYZV3&_yeNBKrck}UjRc8zxr zj_zQswo>*gHYdc|g~IgT@wkDK-e~L%%++QOHwa`}Xx#KkKdT9)jeU62luxk`8xh8r z4EJ`>{*zeoVeJ7St*?nMSjceai;rG#51%yL|iAqJ++b1qPXmpzO$NEuK`tT+>k9TI_QoXv#Ls6g^v(7a&N zN|b!Fl(!ZzLti)rjE(xjpJnpI2*$1^N{yY;f{8bc9(f@RBpB_s2aMNbzS1ep4`v%K z@QZ}lNn?*5Oe{bgWTq*_bxAbtbZ9{!uM9&MG34VaRm_=W5@^xPDqM)E_6Uf#6&$9} zWi}d&M2V_BhMYI`jZ4F{6OIcRV$wt)Egm%)xz=0@nGQ9|e5yaR0~Ew4XIYQHkzJAK z5?v_PYW*bQFtQ)4!|FL*1z8;u8Tz^tTp>h{BG4p7JjSAx9OpwAACj4#Y=WIeZoBO^ z6S1iw8@isfO(X;(E^cGYJiHzu5d&PZsM#seBZ#tP6~=&+Pvqo6(y%BjG}Oa@o#hNI zk0{ldo&70|1tlaJGTl&o_=kVEEw4wDuPMf9+2}G;H&)%3+y7zo<77uWykJ@G;9XM+ZEUDU1`cDSOpg(?oZuJaRA4 zl?vE?-Bl~Sh>K*fo96n?j?gs6M|kfm@G>CD6hShEYN8ao2PYq^pxuM%65i_Qv`MnL zz~yUpLJETT(9#u&j`MQc1e!bEHrQT+%>rIDIoHvMMomiw5bhet{FNDa9Te?sOm^l< zeeF8guMLIWUWWnp&4`y~o|{Bkt+}AbozMUlY_D9H=T)lvnr`k*^z>3^=A{}LjREiq z46#;6!2xSB^@{_g@Zul>@cK>{*k>Tnt@aQIQ2Jh!aCu_aB@dl&>{&T0u-Kh9lHcbQ-p#Z5x8O>J*o_n~BAH3Fj4?1!{{u{u(rb-lnxh7yb1P7Lbx8BNf?DYa-0rX-J3nZ&_ie=1akoM&w&_K>kBgMzJj%scwoJieC*(B*Z=T*yw zs|q!NhIR*LwQUEx_e0?T!x0rFY3I_pxScfa~l#wm3h*` zvAjTGhmp75y?94q2Vih1kczGsaIMI=Pawl;y1=@CAja;RfCLv7&c^m+ZeiF)GNf?| zCLCG)4u;2`Mld0nQH@e;<({ceKDgfzb72%x^ACS9q7+Z#h)n$`=5iY>L}(li4NL$^ zGlSiRO{C1sT4X8pNIp^F4HJ7Tk7i3V9+nI?OLU9ptSiDJ#|RbeSnb^^=#3MjK1F0VOcj4D-TrOeZ@jBe5 z0&I&TCV-J1;KTw5*CYH*Ur0JcTqEEkK461B{N6!FrU}+2gVRi>_fX1N0PPfS{GzB$yu@AxdWI+=&P^ zE}h%E0keTUsQC~hyvW!9fF&n80Sc4210a8R2WCvQrqSAz1H7X&5t) z*w!5^S0K?SV!|&H^1_nsB*qvtFlJhN3B$x44YRGA@PdY(UIWb`85xac55t^_LM<3n5Kz{fshPJ~@;I=2$F@gwfq5Lvt6{1Ssri z3?jD@eWe9YCR4GPkcFK`l$u&i*KFpFM>C0&f=An@zHBv_xY58a8?F8x$#E!URc}pp zcX!PUblEH!Q(|uHD|ESh#7H3X=#`ngFy-n#LNw)a!-qOU)sC*2oE#(C3WyySY3SDr zIfTGhdo(M0i-|?34Nt-sUGl7FnUiHcIm(64hvZ}=E`b#;ivpxAx_pMTA3z}Zl2C}G zr45N_AdrE*umLm;<_q=;v%M>oIV3nBc&$}pz?i9==C~sV(4a?Sx8Dbh7|mMfO%EquU^mt zvKFPnHP=SnOc5*W#vQ#(vJwG&yevQUFD~-{$s{lXF?*CA(US(oB8;x&vfuU?W{+RJ zFC+xJ;UI;&NkK*eIq>SZav7chO5Z|4Iv_jdFe3uQ0FM}FS&zWEt>JpZVKMqftaYkb zj~=QU>V8cIDzQCkDg)}9x;7zb^cmR5L8H|DkO1SyEr8*Q%*z>@X=||UyGJaf8Jqxf zmb73=Wh8+n0;>%G#B$Wx#`uIJM8RUi2Sc4%4}C%aS|L#?04e~GYO}LDY!`KgmfbVh zuy^Uu)`wwe)ZOf|$L`b3A}t9wotc^ukZD{IPMt}LZ~#vYLV#j_Y26B6OcG9-$k2ku zcCC;4JaU1wXy`I=C%oSXaNhtvb?THRb4M3UnX2Z91(HCo>DnjJ=<&uIZ^VKZ7cw=Z zm-FT?k5)LN7iDrRBqGeOY=$gi%!t6cXh&t&D;m8C?5s|LY0>&bI2Gn8W`OmYQ?Y
R zKQ{AG9(ZqA!RyAG}!8;h<-<$myE_hE9@Ujs>p5s ziO{$u&OE<2vwS!W`oX*;iD`GyoVl*4FzL^)97q)u&7TL1AGq>vPi*IzF~?yb6;QH0 z&dOW1eYo@~f|%*`nXAf$p-u;w(Fgv|kr1H3FSpoFR7g&653h4gnyTGjhjBz4%A{Ty z?k9^JOgVPd+uxszS+$IulK_#UC0vt*|F&4;BT{Q`4>BBHtoN~pj^c>R>(vT=goxS2 z{e^rE>%5PmwDX+>G72QZfbj@Q_~+Z|Gq?GTlrBM##?8f9c|9}?1Jg(7Lf9>+6{$^w zL=>5jQYDx!!jTAG2{7f z@)QGmLxdc(U3+0%3;MN7cX2PypmK4Q@E~<#qx-@u-`ghB4zgM#nYNuM{Lj^4zD9M> z=lUipm8rsPmnkSdw{q@LIV|P3`3YXF=pKo#4c}gbLhqjNA^2veW%g2WfE1mfYJG_jE^bpOGrg<7YQ_moBRxQx)YLlWCOX zqdvBKD~iQ2IqVO7dKxJW5xU;KoAF9dx-AV%=opKl^zq)5=XL1~f#bZ?x)dk5k$je? zT}Dk5kCmahHWI4O80Ea8BNijL4zq&^NFjwN@cwk2KdqJ2-;vp6TRuTrJ80Ylf=%d% z{zz4SNr`Ddp2jZ}8_D!R1ZXJ;JtXI|7>OJXoJ+gw|8%~Ot!jCMed>2NSd1*&dK-K7 z)YD}%I6GxiH6>8;g<}(hPmwM-%|j_n8ofk9JDkJwVPJZVGilGU*jMBkzJiONdymSt7Xrg zbRFYbp3RLK7f|atTI6o49pfgL76!k`Q6^@&K0sa0tXLC}aQZ&(tt#~IIHf=cLSn))awSihlLeQg!MY$W~~Tid?FuYJHG^O3yid*sM42TBY*d%Xs$!LjdvPQLB51*EHH)!2)#U-mgmj`*oeRMbO4`Af0*lf9#C(4o^MQuf`_p-#{Pd9> zO{NR2=y1luQ3Uh(Jn2_|1B82YMC;`!vp>oO)P*4ZT%Qgx+srJPtaDha|yZym&=*6vn7Hr~|g(#RM zY45%pHb61fs2aX zJ>OsNb-A-LH%XcYkCTfnNk9@%m5n*?K(+GU?>*mKZtRMIbE(7vz09nvhWit=Vp?hY zNpbxB8)k=u_W(GOYE&it*gPObrXvU)LSzeFn65KaBvDPRK|3Npp{-YR-aR_I?iA-M zkGs@6kcv$vO|-BFM0S#PEHgZ)>|}nuHG>G9Yq=}Wih+toP6}CVCtN7#)u_D6YgnnM!SJF zv=PQxq-dCJy_dsWL`~dWQ0No&0|EZ*x~e1-S7UCZXt&_etkz*7DL9V^8)1|pRevr| zN?ziL>K$i&1+59}9VAF68cpOCQb{OCM(c_l8o+29s}^Q}o0bxqR_Ra1EVKU@n$8ZQ zC#cU4o^63}>`jtKgLr1OrzQ*DlWLEh+u==R+wZmJod<{NS0@xCAN~v{ z+z7bfu)4!g_PW(4X*dG+*Ru48`2F$G+zrBQvKaVfy6(bkG#<0JyI*8oar(079f@S02U?IiB2$Z*08e3ajNhb`g*QnV2i{Mw`C3@m3Ub@p1Ed6RgebDnIPgi;IhC zO;S|b35EByQ~AZ;C2Ts4Q%$VS5EGut9r!6__R>9T&<6yDEm&QZ>gH=P=4)Zv#e-4o zm05T=kS)e{LyH#49p-nu$N1U-Yq36pFbi?%SF!`Lg&;%mH6_TPrUZyRgmb?Yg2z=i zE0cdp+LXh8*Pf22gjQp+D;N{QjJ|s#T@0Kp7!22KXWhNEe{{(KHdb=R$RhQ&%eOMV zHb&2WmixC=A&4^Gi#I&-arykr#%$SAeI`d!$Je!~(pVg_1_TvK87XzhS2$vjiD_NU z3*(Nzm)%Xq`z7cj@B($G*II=i|8V>h@0BPF!?P|j zFJ6Dc2|@Y@U}~a_1Q3d?yYc@!LTQ%+5yM<@=xDx1(K>r87plqOe3)g+u+8l%Vv+V7 zJHxD@u3#=fT z|0^T@keP*>_}8%6F7FPXH*d+ullo}FrNpSmF?ilA9My($x*QP)BL%b?#HoB{7>YwE z@0w#7Kw5<>$@6Us5Tc_szuFz7neP8oY%%$WtDv4{cW^4sdZ#Aa5FXqn@ZHOVF1QrD(rDyTX9tTw^j%_#aOdJ5DRfv zUTEd|M#%$L7L5u$f<Im5IN)-Fvbvo%|Sv) zCqH;a-x&6uw9P5|{~JEZq5gUZC2!Y-EfRJ*bpNlxW{Pbac6}I~S}q?vSTkR|JI`HY zEc9MFsnC938~_XsOV^=pZwYesgVFQ?_$b%0yNNTG4&5F+iZNiy8=SuX*hu0^hkn%I zXxQxu5oP;uQ zU3AN=|B;lwb)Zj9hab!m&VEH;j_3+mKUQhAFxqg$2kOvKQz;?o!vRot`uBQin}BAh zq1<_H9}|KRj;U)p6PV<{5L)wnH+n=|(Q#qb z3DF8g|M&ysjVnIrREP=*_$_Qx)sb*3PBmRp^N*^ge1HCSvo{4k+KX0*}szr6Zi&zO36be zg7~k|an}(KsG#NwwAXXg*Ov4@>EvGWaFDzBS~1e2{3BF-7IsyyRg+W;1O>06zHECW zIH0MFlVD;~;-|9QYkXJHf|3Ey`gY4_w1qg@z(K?4dtIVMQ}?iG^k1kMwbp5pSf-Mj zPK%97zTD(?B#x#Yi$@;5IJxnPk*Pnvz7wl*5*W9n9Tw7s&Owjs&!}U{A9I>NTn(k3 z%C+g|!M`DDSx)!s4Epe=I2jP2?Ya&)XMoWJ6IHsrZTV>(HlkL}WhF&Aug}~G+m=rY z1B>I~lIb9Yd1mNR3bHA?2p?P%AkHV+>Wr)Zpewej&uit*BZgZaXUnmU&J-bmaKASo z8`elJsoT(XBDH00qu(b1r9oz!=c!hJ5?;1m!jQ?b9F7Y$G)f-V5PbaiW9!3W!sfK` z$c2N-(qLimR%cs*mkExwl=XXm#(td8=)m+O&jagaQdTSUe7;_aQw&xH`^{RSp4Ro} z)hP$|8X{7W8tc=C{#)Vx_=cXIt1icsp=o59c)5W0Tr?M>`#?^|Pn?*SjmH0wvO1gi z7y0FGQ=k_#uh5~GGHQ*b)#Q3PN8}Hs-9&s`!eC@#X_{`{s7zK##&=I9fJDn*tDJFV zu3!~qT5t=zr2Zp~Ac=6)EFdbwn|X%gD%WP`*>Jk_2VHQdEN{#pCTV1h!Ny z{3Py@+o|#T8fzds+fcKbmzsM1M%lE+~( zqR|*91Ob-H-vP~JU}HY1=e>-a`RTMoIgS}#;q|_<`&rFywU)*>BoW}m3-Ooi_|!ZF z{g)5eyXgz;g>61Bp|cDIbMldXcfI4db<<*Y!_t&`Yrz1eT-nWv1TtMNC!`2n&={n{ zPTt^p^C3Ad-^Y>rh$-dXGg>{Aw6NKjHS}Tt{xrNz596rM`ThF>wit0Bq@5;$fw;9GnjOX)+yla+E2-f&NAE=^=iah4naxg z?BITak}I_aWp6-fT&~qiRSBh>Kr77&Hox+bN$HXhCoWPx(Lt>`Zlw{GM9tl$nL^g zgMMDe*{(qR=+m^ANNFrA8JU@5LTF&_pYPAdh)pSUdeh51!Jd~*=udg=$J)-|Ze^j} zUOz#}Uh^~m^yzcFIfl*3{a8M{)Y+3#1-={cXR#J|q*{ev#g9L|aDA66bw>Y{DV#da zoK-tYg`A7YG=d=Uh|-^Y&EOnvB=Czv%-HX%Xc3R3WKelTVdXex=VwLWi(chd&QuRe zIBycQw8xcDYB!q4+eW6w>CRH<_kR4P!$uX*!uaQN*;gJqmW-dOYX01IoHPF^uqhRS zlmh}4^25maf8Ds4BMV0D4DYR1J^?`kY?>JrnM+E^+?XjaM=UC%=L!8mi%PBbiz8w{ zUq9tRewlBs+uo^ESrx!JB7Q0$;)=kFBg~T+Hb3#oOpuPh;FHiKK_p|saDO9ZQ)%%T zG^R!QtLXRpSi!Y0b2>_*QL4p~Dk(`K!XQ>J;!z@!YS_XnP)~EkSA|=sG^O*Pqpt!I zk2#q9aUPSMSMfh~PoJFlNL+~Di)`B*kvef*I<0R;C;DX722~g{6zpM$`Up3dcw+86 zQ09I=vEFgNjo{%)SnIWJ^`dBg=M$w_LzI`FIV`Z^6Q@7*mQ#+KJmd0G1egDb z&etUcbtMGZ)m5rCN0}Y%73zrMu5m|=Hzbbw9CjB&WRPvo2X+a^EuYU;iT(C)I5`@Z zxz04Z4#vjpUqsFS{p0`=eX5%DB`AuhP%yV7w6G!%g%VP3jr_^P3ZU4RC<8@8f@QB? zVNj`5gh}!C`<7dug2z>+o^BDyC(Q+sb#09$DRRXAPs-tjzR;9-O3Se#14Z*sG6B4~g-bkR-c7-6oFXf2ve zWO6bw>>hI{KcP^3Xcm@Y6K&~ke^@A-7f87*V9@Zn?2>YQsaPG%Qftzz=u?*KKK`Sv z0~prcJNlF67-a2C^AO)}C7n&BmYt^O;(ToHHSxq_(Szvbm|`TEpW!jXD`Fd4s%X{R z#6Z7(&2iM5p*ZVwmp_3Pu9#RA7${7EreHFD6bo!_!eE+R-I~Rk2$@ zpqC+o0uuPTZVNnEyR1!qFTI{6<-%uAW}U` zut*%gtNPu(Nb~iE>#J*0W-7IIS*r}w^|(-IS2cB0iPg6(i>DHAHH6Hn+Pu8U^9>q+ zGcP38#8I)T%CrUZ-PuiyT$0%BJJv}2RzMlrIcq@sq*fN_i+~Ay2h5JjaXEAxVcn3@ zS5Dpko`EwpuBF;hj_2WiI)Tjy7ea6yK>-xI80_9G;RYUW#m^8hrF(;EmX|CO6j%08 z5^73koow4UhbhIParTJzxV|dY&Rt6JKn7QZEj0+ED~o~8+&ybInEZ9qmjvIBxLDy! znz5&CR&c@w==yNUT$_Hk(CW;^g<8}vPmeyUoX59w;>bE0c;)9DZY<GAQyYd3|M#Q&NnQ9T}gT+cf>dnI=J-BY&k6HXFd=yLC;lh$DGAx}BE?y)%3gfZDR z>_LY|l9*f`rm*;8>0d-i0qSl=lu+~nF1KB2IS<|Xd51#;3T5~X!3#jh%gE@5 zSZVjalgu{(S7oCkl>a9~9}D8a`PZbP-n^m0d;*hJW+?IgO6Gi3lVUb8qkYmG8s={n zY-ID(%ycK2LVq#^kRLF>w!pay3zf9p7WnTE4(0}`z%MM({X2fZaFK|SGrKj}9r(I; zBGwJo!QNHSd5tK*N|-GCF<6CaPIF>3jrB_la(&sbjJ4>ma$URyk$*~SgZLMb5rWI= z=x^xqLJUd?>PcLSW;LD22lcNqUSAnN^p7h?eS^(TQ|SEd<7Ql9#nZ26OZ71&jLmh% zUz{jH7x`XyByJ!qAVIy;i2{lB=j7Vvq-)9sNH3eCxMVm@Z>TovYOXt-9QD3LDveQ1s%hVE309smPOu|(n+IU*CC-WL(vu)-br zphv(kCK#v{V?U?Vc*DWE`$Id1gYkY2Bs>`6;#08-FFx%59U+wWHP- zCm}kbA}WEYAss-VE6ju!ydr&AJA7I{b)53g^W~BjQcVjLF<+w-my$ zOHIcW=V#v-^w7t$G(lsgsx3ytbt~wbTC%dLZ>{@%ff_}e2=8c$&)<`t=)5p6O)KBqG`KJ^c0Hfu4g%H## z?6Tc>vl%rr^F!D@J{MQVFq^dw;T;u*r|YoDP!kgZw#vIOUj6anPd@TdDQ>e`Mu@!f z`caTU>%mkLm+Caadr;PLXv5Gp<0rPMkkYi#fTw4Js|6hd4vS?f>=r9WZJv>HRoazU}^UNT4wLo2EE4F$SC0 z@7KX+G+d2O?P8Op6ipA~y2mE^G|fm`gAcAjyYLeUminH4o_Wa2X>a@4y;9HPx|mir zFAtUXXH9Na>vF9SL~3e^YSotZ_gBgrYIK!F3o)I8=OKN^C@^9DoucjXNxV?EpRg^3-oWg5Vb?s&G)JMh}$?kjtqa(Mnru|0F2+S z<{_%aa#1H@h7O8$<-|#ZJCE7d-(VYg}vy2;E4Yf@el^)R<)XGIcj&Xr0NIJ5;`posp`A{f6>PrFcKkg<9l)7#a zgiy)YKftj)GS+5eCv@gJO{;&{0es(Hara@=sI$O(BTqHvQI^8Ib`FT&Gr#}3&=TKQ+ zpe?F5cZ7j;YhB6EL3@5aFktbgORbO)$)NV6$cN*Uqz2-eFnjk|@HTdW`4f`!-ssff zbU8U@w#&}UjoxR{zCCzt2(h3+fnaFKR>R-+$V-@aytVDE}3gH$nob>5PIw-2F7Fn!?Ry z8`N8oY}_TiPEZNtIEP$9<^HDWe6i3UZxH*ZUMC3pgDTJork0UUrCw=`{&}5#q~XTS z%4Kn7SKUFwGPiW}QE*&rcb#cT)}sa5mhG0NGCgxLDEQ?B^qPsueA>(O^DFJNhmHgv z_|jN2e9x8ecep`3a9uI572rR#@x9@^4A9PzwTK9q?cKJ6`tOU?d;Ef48q?;5 zOX>`(CXz^6YsB4yKLc43fC66RgVt|_qT-Yzt*TbiKc74BF11wZ)N~F{c7@dCc+0k5 z=iW0pqk@dIM5`}53(0m0860b)5(UxASrc=EK^Z#Dy-1->gV+)rHSGDX4339{CWAFF zwb6Y>g|#s=;PV4ac;+G#c0FO$j8nmwgFuD!b9#X!h^7?QA`|m(f553kpVj&aC18P3 z?P|HSui6+$pRQZ=2uF4+?trYj@PUvhteI^|4Xb9>Nv-aZMLDXNhl`#N!G~$Y~UO z?}ynpRkVn(${oosQW+QZk5*d8 z|197aEMpM!2aOxbZaC?9ZTD(PzTAWslvc`+rWPT2LU(!y1T;w1)lX@c%+bsgEbABM zsdQQF6RiyLD{a}my+&f**s#`_WM3T%Jq{~M^beoRNBSfjEF@NV{tdRi-XU9-N3xhb8&VOty4}r<265HJp4V$ z=a3&_Z(5UPr8Mv``QFd_Ef5%)?0%Ir)-t7$UN=u`aM~GdV^?FN&u+mL{nE@D{8)T_EP2KWXC@ojlV>60-0W>uMm1yK(TEOK*0GCjJVR1d zowym{5}rZBqu+skZs|TQ*DI)waBQ#7X)DcmW3n~631bM~-w=QIP}z!8>o5GG9Bb|0TEwgmGpkPjsW@`Pl>mf8s! z`F#4b3nT*R!{E;vO0|_B<9l=EvY$N_HOLg*8Be3O#ozO&Wrm|g1RJe~nl=(ctHomR z5VgE*C8O$sqBgZU$K$#$ne@_0Xiy;+pNjpvapNW!*(!WUD)VD*w(d`3jxf&>_=(2D zN{YY1_3r1wE3zm+y|HfdkBus6!}3K|i)ra`4U0|KU}Yvv04+flGot-$NBhc&CNhND z*XeL%kx-rlF00-lkl+QD1HmVJR$pT;(@7V5M;_&PidgYHT z1p<@ZAmcfa3YXwqYljxxf^!kIOK7@9E}@}@Mrk<%*IL}t%8bl}KmYUF|Nr4<0$I|B zVAkoASu!zt6`a+`jvEZLlICyzP(+}#srQ&Ji+RMfdDQi)0q_DBr9j>p+4Py{K-fG zsaj6$56~UeT6SHi*o?LQ_pisfV!1LMurfwuzCZ97*&Ob4SW9ginAs5FoxLDA(;Y;{UMKkh@xbM}8e zhskObG)_@I2_v$p_<}-?o}+}JC<|Rtk!R+G3jQ-$B~`XD>r#K(z*y)pB~O@iYy2TV z_1QDP5o{QwPsNht=|TSiQ{>p~iNcoQbEYTpEr1$y9zCUqeD@96fQE|fkYJbxd*3H;%4R`=4-in=7tqvc-&W>EnSny?5; zvT;_flh!P40}V65B_~nO8MIp@k(l;q3?0m$Krx&Cf~B;RC;|)?9d90!t@gE|;hxMu z;Jg}bavVLFNDte(bgqiuc&z50c2PgT@j5fUP87nFJ@`)Rei0~dNd{tf5EEncoO<)O zI#+7vJifP%9tiS8Sa6v-C)fn#lk|=|sAm$7B(#-n5T+izQOjkRh81!*RCR^Ts5uKxd(s z9(mHk4QX~LqLKe%Lvj%=A!R6wJI4nHBxz`A2#8H|`sn!Db zrp`5^Ri+G$!Mw92wcIJtpQspo?E4(VzveMM!cuUaBc0SmgRUivPv$QF3sf2_`u#wA zNLS#G=#`?K^LlG)BP@PG^&YT5;joDPuLV1luOPJ{tNY++u8=WGXeO_RL^_4?!ZmpX zyOBP&ofHenWj`W^_Mg%K*i3Y?wwO$6nEHXlwwi3m!Q)sAi5aE!zq4klFFcE{NA1=^ zDlX*YQ7cexh${p$2Z*wHG^1+9u1ls^m78c!a@g7SC#&u~Yf3#I<4IPs9G3Ponv>N@ zfEfz1Xi|e}N#`tw(G#v7_x@2N9I+q?rJ)>XTOfFYO{4Su}7KE zps3`jpsH|!Zm}p#`Nf8=5;LnF>w(iy*wlS~Yh!C3x6|KJQmcT1saRJPT{Y^5NFnF5 zqFpomcDe13X~mvG_R&0BYSmW(i^oW-7#!V2K+oh~*_;o+R;DH{nrk#!>&qi$u%Ma`D=lWc zMsrS2BN43OUUvc0MHTiV+oOlsqnKJ&16p=mx(^IEGSlu&*ll?sZS-D?t;ZDLV@E2N ziIbhHrF4~#Q|egkzUp-#GxNTt>qwHwnq~bQVx$>m$nNWX`TGK{#%uaz=B-bFEawQ% zc!P_RD`%aX`{n68=X>kheqhK-quGB<+Krh$#}dU}-PihB<383V-OSR)VZH^1HkfN? z;YQrzekXiN4F4BRXx4Ljf!l5IaSc9x4jzt$07SG8WtrfFiPCi&`2 z?W4j6&5jYO@NspRxEXo$B@DZ|oj}p|6Yfu=L?uV)4$aelm9kA;uX{YivwWPzNipz3 zP#iranTbocAf;M4y9P?)5D(qAPnmbI0VC7@&*iTo=m&B5>doZ*?*k93WusUzbK8)D zv~TzmM{XvK51IDq72wO8;#Ytc?tH$)fcMhc+-4!a)MEVD3lo+5we* zktRA%2*(6GoWdK4atx*c%^}RAG?3Vjhk^~b{{6Vg4sHhJun((z$l@Ir8>)4}N9VXl!A++1$lM!@ZV2mH@vR z??nsEbt)~s8rR=W57o?u>{(RsZtnG!^VYXj zS=<<_Ju_jAa1N;_^?hW_BI1?buG~j`bvL|wHb1!3f<)7{v;3-Vs^s?l@ zGHJH39f*+QiFZX^E(Q*(&qkv>!ovp2EK0S@rLH@v+tS?xj7CtF+mY5@ z^t^Z5G9&R?*NE{N;X8)Rn=z^`alK^sZ#uSKBEFxb_#U!c80Iq}B6gV9krIp=>uSxj zYS@t3b)b@-A4(a$e6&UXzQP^6xICJoY>&I;-+}DQ67{Scc*?IC`C(#*rXA+|JWRs7 z5E0pwlCI~8AfWp(dWPnQ-Zz|DZHRSaczEb4dX{cPwn zz6s93QM46+=UsZM^tI9G>p(Ua0fU!{D|9WI7eCI6J-RW%)96dQfoGyzEGyF}$xy7P z-EdQ{`EkI(l^MXriZ5I^j6y6B%c|07r|xxL4T^BR>3f8UUDJ6IB{TmEft92RPMw4R zOqo}onZR)p@Gs5GNKOks<-@4jRvb%^r@3DOfTh2*H+ zSQgS2p+XBH+^nqJeCz|fVv}Qz;^{fk?vnH6CZ{WNdAjqY-X?1Pa6dYo<^&5W$4nZb z*nS_Fqk7vK`D07h0r2^L?PLAzt_AQK8#+JzdZ^tl67iuMHp}tp7kR{3!=6`|K@$*1 zSSEYb*m?oUBh17Ji0D9*W@%3}nEpp8CB_BH(<10`iB9W7Gi=x(UDWxkejDwH=Wc!d z6>YVo)A*SM1F)T*+y2rQZLnm+5c2*vBu$oqeIVb{`u8{=0^i+fJ%#B{*Uco{JZB(H z1NVfz4xUkTh}Yy_P7jgHRZeCuR|V^n^HPmB4zz8VR!?*B@=^sHkOL0s6kI72d%`;P zYuso2Hs5Dth*+HL9LsFqr=2#*c21S0~or_-zqdybKs$;5n!*ULDxNA#ZzbuQl2;o zJ8^JE@T=nV`wW%qjN{11H18Sk_t*}XL4Ad=PsN0eyq~9X$PIY+!9`-SWM@EUWVL2R zlwmVf>3=X9I31C23^mLYvl(Vqn2PCnPs#d>OdNFBDo%|u^ea7(Am}Mxna(aT4xRecuWj`7W z04Wxd5=!F+(MdR26BLstxOq=kn-e?wn#PbUo25im^3)C{)HsUq7A=TYvtQ*ouM6W} z2~3z;o6zY+48ySA!8|JL4UHY1&$j(S-n)`V=s+5z*fMIX6!+px)v0x}PD}&S_X#Ph zwwoTDV=w0`qa@L6DVSSLR_M!kSuNJ9{Y{~jQNIzSGRQ)!T+#^L?Lf$GYKZMWM$l^L zbd%j8Q!}mWUft%~L+L)8b8C9wSoIb-59Zt?UmuA5Ss!Ykk%VsMeA)(%e-yA+@7;9JNTdDnJss{ zUNid^+xtxOY#(~x_TTw^B}NE7C(GZZR!5CaLw>6%nU_AL4vqO@+0VS3-Pit`5C{zY zW|kwbfYrKa5m1(UQXUx9g0BNtC`V%-`lx=#_BN)h+~t<9E4hKv^wxjaSlFTP1P!;+9YBHinxhHJemgz$D}3 z1_-B;ni<_i=j5D$z}HWk4!%AS`1B$Fn)1^)zUQI?hea2LuI$;F%pV}i`q#<`qtTt? zYns4Vhf3G<@9Tj1HbzK+Cz5?ai?3^JME{}L-bi-e~aAZ&&q*0~)w+uD3{YulFTF0b_u`*ql3qi%JvC{5ddHqNr;BQsmtBv~H@ zRy-rG>E|kJ?MZU3Lbs9VoN4{AoZEoINQiMpO^$hoNsE4C68kRXyV0n3xOTp?7OjCJ zd-_PH>9(-UQuugyjQsT++Kd`@v!@ddD`s@&7Ot=do?#+nw#f(XNc@p`X@(PkBl)b* zD(?J7%3ss9W4+0Tq0uZzz(d=5gdIDUq7>+)bih&M3GEkWOSJn+W7>Ei1dcR5?-ko| zhE>KU6IS$Jx()k_1UT65!)5UKDnoi#$eF37q$F(~gt;=)E~cT_M@En6J~HI+)c1ha zr^jKmROI6r^+%Z~#Nk52zlD!;Hdb`o-7ykHT9ZP$%uB3TRz)*HA@5>;EJFCsjRKcY z+_jA!(lQA@17xow+CN$+OV^YNce(M50jJhcrD9#>In`SZk;%JGfyL`62G~s)?x(hCO3S)X2p?Ft5%6mCaYmu>24i-6k6v$dICH)jZ+{|fe z$Z z3ot0FBclNAoHeuS4^Azi_uDv@o#kNW);d+w8H1#r)AI+!7%4o#Z-6!~0tqNPTM^e! ztV-PZO;$Dm+h4L76{;FHM=9B`=KB<;NQ3N%%K_CH4dfoz)-bp9zJ&#nQ;C25yx=b9}$t`hA=m~$18?yXkYwBe%mD_AAfaE{r@H zPgBZSntDnXabG#aq)4QzKl<|wSK!*S7;eXRUKhJ{ab%$k;G*q^o?HvGc3ep-z zl9_sNldfu7bPCN29h$|f5wwLF&c5=|9a3V7V%goj$pA)%j&XaUZh>94X>?bf9R6$8 zY?@k*@o1bb1Mtn9Dy+gR(KDr)(zT_*rXf^=O}E!*LB2>!O&B?AT)%3FGBl&zH*=Ab zBNUfo(V7QiTF$g(VPjr5%U*RGTdL{;o;S1Xgfa4z*I_AXTQqn;jD=`}IS}}aw>`IS z;K#X?yDe+&Y$?n2{#%*<|1kE}VNtf*8z?9uDBUGUcQ?{QcXz{3(%pzicMaX0(nxoA zcXxM(XWYLXpYCs;bFORtcwe{%o?7d^pOtUdhoOD-ouM4F1eE7$^R`?m>qul9z`R89 zpHG*|oOSqIyR$xEnkm9$Qd3+Bgz)aAWi;i_eq5W;<4|PD!-6P|b0Xci#4c>vgN143 z3voP2K2B8mT*jWVBafNh(q_td%xI`qgCz*l5L!DUcZAGDMQ^{6Q9X%sBfqrCiRo)V zn%a{ZQU-Yq^D?BEcgh3+m488Uqht2EO#b2za#^Db$v~PA_59{0L+YzFPG4`A*YFe` z+Fmb~o~_=s=7kuTN;gqC(*$JsAp_$hX`VjIPEhu0OH*ZO4PU~VsjDNasU&*Q5Y9%G z{q572vM|==}=2M84$!tmOum?q2Ei+=y(a50eczAMv78<4d0ud&GeOsljWZdy% z?_qmPcWKdDwKr8_adSkxmpYTsYCdMTHgXD-NPLL+$eRAnGVZqQinuKK0$nnQ{a@xU zNK`dps4E^KC_XFX`pN9wrRa*?6Zu=c^Mnu&l%XhAbB5 zN!r|{=;@w)j6n~U^i`m#AAnnq^jYsqdIM1+Hm>5k{6%m)HS+5Rg+`)QgfxeR+2iO8 z*wFb2zzT&ickG1FNxyz}>>L9bsyiEsYhQUe*gF&l5E2DlkfORL5vNH-Dw;$WJSIT@ z$fBj0WOkpB--K!mrP7prE03cH|gb`MH2tr}lCrk=juhf0=OBiCKVi*5suRdVItr^O_i zg~Dq>uYoO7PEc4d?E$T6Pu)3gS_4lnKD)`chzmb^O$gHWKg_dvZ16pu#qH&W!katw z(26{t6(nN6G*C2pS~YTUUv71a)#u%ABHvDFRXOfvH}kb_>xNg)E;H-cLpauiSuO0H zLnW)Czt*ZMb@4prnRytN2nQjeBU$6A~N|jE{BFs#C)wkz9Ih@SkbqH6E>D}3m01T&{P~oe(gPv^{InqI5f~lAij5~`Q$-%>7k7$B6U1%@n_QDi0iW7 zq4AH+4Mw^MSJ-6HEOqJ?qlG|-&U+Pi9DPkdGmfSI)YG`h=@09&^RB6#)JH^LkR}7Z z^@JtGrV@gGZ3=^gJ2~x^pm!V^2udn(>(5N{=(80 z+~P9jJJwYaezDN^psvRd>!as`&bJYc#@5UCds_DIY7f4Fty(;=G@t6@lYJjRMAmW; zk}_7dmvSe4qweuE8)?3sO+dokEA9`}eCE#>fjW6fXhc>L(eO$f{&r3Y#6CWu1xX2e zF@8{|8|TBZb>}k4vPxdLXaub(hArvl-!lU}lo*DhZ#`7(wo(0MSwWTg)lUtByL7&` zh737SLt_N@5P~L~#9u};#?eN3p&!ET11Y1%-^x#3h}5{b?<%6qN4Rg^Aq>e*DV7WC z>;OL0D<6EQ8`28c%S|o3anZ?+d<=6X+PqA*0_Pr#N~il!#6DmQZwG(vMX3#C0q$r+ z3AT-yESE&QeYPcFO$wBQ67@hq?`yXm1ZQM%x3jHvHu{o7Bkw8x$X-qBg@2th-D@gW zRL~Y5fb8!uzLr0hD{!Iuoep_RIDd2qlFyW=qtEz^CA~X^Y^hdF%qJiHU5qG1=Eojl zl$qo&c)Ej!cnGT9>-n!Y?Iq^(!&}=Tb z`5o9qK*iSZ$&W`?@uU=y>7P>BTJJX)F*C*Hw;e8l{*6@E5P2HSgeqNY;V@#lF6j^51t@YCExue~5MY#R=UXER%f9rJ!5xg6$%97jPb zpjxQ%!RU4DBRFeoy=%BJ6>e2gRNX`TFwMOIQ*y;yrcKA74X_ot!>z2qAFxsG$?c`b z4UB!m+e1rY$fFPxq>e-E27ysl`~(l-{0WFW{eyRk&*7(6F>@}bl#9PG#l+gRq_0!R zD%9TM2foz92-o)4n6+I!o#21$+U^kgm+dn@luOXMe z+sdN7E3=y3kfd;5?(%mN{n|)jH)q(BX|+k-lCnWa5OPRIO;ss7xU@@HGojp&8wjP# zqhK(!GnbF*X&IDncvP4GIWzE5DR@& zC4UjI%3wJ7WbxILES#O}?W?&>2GsqOrv`M1DlYep92gkvvz*0#5z5kL;B|wQ6Cq+1 z?2Uz)NH&SE*?seMVS9x7;M{siF#Lh7G zVY;`7U2dm*2U?Knj$`G+-;i*Wz%!+%SOh|8E*LGiqjHqOoVoVw<_hkZrQ+vHI2b9{ zoalO9CrgQfsbX6C^E&lxVJvyjCaA}l;YjGU-PJc4Og>5;^DRPL#5Ff=1Piv7yX#Lc z7a(^8CpJ7QX;uvd|8oM_0N2b4t2F%9ZY0D!3=8jrbSuqOzDl$5pR3L1R6I*twoJ}2 zSk*2Wv_CzvCP4UfJ>{f4G&8h9Z^iFke7ki$wv*jI9U{nDVzFF9V7jLqQnJ$PbSQ3V z#MZ8+-7kJ$>R{Lovv)7!H0cZv8{LK_mlarq&oTA51XshGzuX{GfzKZw#@D}yb-6n8 zStqw{Ij*Ib}Bn z;MR==a&am$S>~oEld(!Os~_JoqZ)OOio?~WCA&Ls{W(Bf%L8WYa{zD+h;5LuIgBBw6NY} z?s6a&t8A-g3543yHs8|4b?+`>UEc1(WT<60uZzj6d}<#c`Rz`s=n6}hyGn~};OUlh zXAZbg@Jv;gh+`8i^D7Spi9;&()Jg%#2soDjV1`lqnv9tnBj@)MVpq&r%&EKRzF8S@ zd#)+m29yV(X~xqfP9gNy_ZDneQwbepr%RPu#LSw1Q5|r9;AXA!=x>uT&9686+KqQ#my`cnyA$d7pxf#=9oV@ote<@D?t9; z{=(|oF@%?fj8;bFSv6zb)7W7lbn<&>^2}CFP`+%Ns=6>wwkn+CS%b4EAF`8z-=#*R zJK2t~%xJ#jMHV}Yv!^4UND3t0i1gCpy?}|(qPK9_tc4v<$|>XGJ!5|7_-0={xIu7w zTBO&Zpq624>)<K}kXOBGJh_T9Z`$ z+f9WrX$y~$6b)-Qa~|9i(>4(nR+74 zleA&@=SPP)tA6sRh$GAcrNZi=;OXbmCnMJn(4)8Dyv4ktXe*4WcbsNgb;`R__zW2= zmsJ}^2=|W5G{eD1hbon{9fq8Xx_5rM1@KJr$;2x3wK>V!(r#FqP_DHoOGh~7AN#mgXuxjwG=h2VOSRA>skL3o1$dPu=epu zUE~YnSA)S{T250pzBrvMiH9uKS@U%tH`JS?@|Q^;EB-7=hk2{_3hV~|_oq1JOKWn; z#rwA9UqK5#lOb*A1e!HO?rx4<<4J8*!fSV!@*iM{Xzl1@-h6gzI^Q#O-#J(DRdxsL z#h{}?Sm9ulAO~_J^11P2Zs}U!tl?~O%K2cEbtS#m##sF{V_tPd69O)5b~8rVx#*^q zDO!2)A&Kns>pI*3oLu-;w@zbetP+zq4f$2apPiAvA)@1;Ny@>=&Z{BO9-d}NBJ3uj zH!Nml){l`V)*@?TEW1<@38(?r4)GL!R>06n6!bK7wZ^*3SLQyM=j(pMqg;5@yy#?P zVrCpg>z=<$MpaI!0-crv~?f(SlKhW5+!2jsG%3haW&gIORX-Ul2?S7a< zC(1LiuYB@&beu?1aqI(^U`UBqsQ<`Rc(c>4jpUMP2-Cxxl-(PaH-a*GeXz61mZ3<+ zBjQC(*VFRA$M3LGNPXJrZvCU#RXfW#cIM@C7b1sI2)=7SFeL<=$yD6HQ9wiZQA_7 zI3yH)Ulrh9sEUxRh34&YH39{C4>{O8xGlwj@E_~NKcjg59nSO5pdLjL2-|4nl+f40 z@Uc1U__DWeHLN4z4yy|Y6s&3dwOJAtR2NJNZwW(0LdBAwfCeV9Vr1SkU9Bau>_+Ev zG9}gia{ZA?BF_i`n=-{bMmvTiLuBte&5kv8&}J@bCgc*m5`rfb(r0FHu1y=mGFrR%+=77!LLN6Y6E-(RW0SDMd`|Mum1nOJ zBS>a3#=x0X&sLn9pc#x&Ej|#B%>QKm8b7)sAeHR%wORiqbWyX)eRnSd^sRZjT`ktm zCn0Du>GzObpHS6WplELCR_!^Mc7ltmXp=i}07j_XY8l6`f)&x>RN z^;Zy}mu(TLUkLeP0NM^i`rcz5h>+$l2K_|}BCJO)EI1Oyd2s60!w55+QTl~ zd&3c2ii!nA7C87z{2O_^66TVxzhD!G9F*wzvOi)2fSukq(PZGb%u*HamRT#}$*e-e zq~V#2S#ng?y6>c(q>Pgxkgsecm!XKLm?}OlvI&Uw!3%Bh;>(4LYD{6d;_gfu?xZ8Y z@+MEJlksQRCk1GX#xuY9q%nrIEy6fGQ>brnA_PH&SL6&yI^ynju>uA9;PXI^K1j#( z*62St@NYo)0`~JNSqOtHFltXn*>{& zL{VXAZn(LvZg6s`%(z7sG7&|Osm?H*6mLUT(}rpTjhhYbwNj@v_HOZbE=5GNSNaS{ z8T#2(Bv0bTh=$3~Q=Eb$9C|29DkOn12g4GrJ-X-Ju+|V)?E%Md?WcAgg4Q#SWkFq11_NLmJh|$LCh}_)lAc^Lh zfRc^`;R5FiCISoi`LAQ4LO7Pq3$uq+yxWY%rH!T`r5!#sL-xS=1NTnLR|y!B>W6>K z@%s9(9@;&C25>2VNEE=spg6!9&Pl>zBx>E8mJL&K#!hkVRnb|iHh(0^n=%eJ8;;Q(-Kd;42U@0#70nf%`#vEv&rQ zXKYK1b+(Y4rVuz^b0_^bsk^A_?pIn5F%Ks^j%KB? z6;%yP`;c>}(t@dD!_hcXl?sBLawASd=#n_rLw*X@4NV1xEX@fu0YtPaS7v?2K4X+( zS*|yB*C5V^>Y3T;Z#>{JT7bD{t zpLI60y`sFINo{tdYvav>*Fzk1f5IMzGL(y32LK!24)2y@M@iTw%L&Fml8Ir!^`avt zgM~ydt!XNNyy%$ww=dr5K*?d$=fhR7Ip{)zb^;Yj&x>;P0RzHbN0OnIc^MOMtZ3@Y zS3<*xG^7e{`LViC@dNC>e+FI@&L752ty{N51KbWJ`8kD&I!iDU&l(w zWA=lPszpoSbJ3+^5X?`EK~2J;v|x=ij7-Qr8`#0gS7js%Z$3=hEq}GFCi#O<9S1T_ zVcO`9+BbRM+6?c7a(xjh+o^wcp{`_86MiR*vm6_F*ke)&HWTx8*M)p?VtHpsgS5a? zfN3fBIl7GCh{jgUJI^-|?j>`dGJ30-K*8`P_d={H5n0Yh+rr^Bvly287~Y0j^oQ_= zxX>GwGw?eeo9TwgA@dpcEA4eCf|p=6p7LdT1?cq^FYQ0K%>N6`N}+k>W`~SBDtsXT z9b_VZHLZv~Y?V(=f0gKI-i)my?NeTQJ;RdnR|axdhXfN9g)6t8P#=d=M8XdgON(8= zT%ZdkOvDQ}gx8EytWZF8#DX4(dpC$SfHS0l-PbJ$5X*j#KHbr!@A9znDU6~AI2G26 zfYwSB+sS15hS)mEPlGX1Yw!EJ0cjleV3WPB_s`My8?;GdHrqi7@>ct%Qwo{|_38n5 zK!2fPvYxXHl>;_OC$h@%ONHE`If<MunG_H>;Xzk+aR%vM3AMLmuhMh!Bo49kf7>*=}3S6=g zK^X=meV$g|F%MTLDi}OSt(B7{rkI+$Dljn5-NBib?8dCe>rYwzTnR%54j>C|S5c@6 zP1IeJO`LJj9d zc0uS{!FgIb2C``-C{hv>JzaEA}o6GDn&q`Jcr&PC`H zEw~^AAJ>Lmg}PhH^oT}u2vAqCj4FUmljxk%7mEw6iIeuM`+#ncFQ1>urb3tYpc_;* zp~Z@MWklQkJ&nRn71GaS~NCZQvL{dw_(L$^Q{_ny6q^-9IE@93H^#BFxH=A+=fB+` zn@IFg&V%eK?=q%mKqM%JM~2GFHeV&}_!Tvyn?GLk{Mcfkao*2?!id^UYOIkKXZ6+h z+~5`sL(UEZT+;!Hk_Wpw^izIR#w`@0lG)%OpKZJ2q|yYK|0X5iRU0MoY1fNvpu{n^ zqG+Kg88yO{?Zp(N-}yFmc+JE>vZC=VkHt5KLNYzvdNJv*_}LbNT;LZy_LB-8rJq5$ zv!M1Pyj`DwXH%EiQ#eY%MbMdgVg3a1I9jJZSHbPN-uc{8pPPAOH~QcXMgxBc@U%$2 z9%!-NjAxV5U_Vf*{B7Q?@Ondq%kIiF>#)~QEkn>guwfz%edru?*DF~3+|#zf^44(Ay~c zZYm;pp3XW+bbda}>N<=Ib539m=C9A@{dVMRh+a(_bt~|bVyHN`lN9AJb%w6uo3m=c zOHON)hkC^6U{@MYq7GD=ld_2A^uBo$A5lv-wOT%uHPCze^2v&>fm-KCS6Y<% z0)Pz3+@!i+M2pZ7DBq3aDn;p8)S)QuJ||p5+q*W?P&w8&n!~Fx9ekjV?Lk!}fSNo& z(}q|g9$c}i&MhnOVOc_MlEP?GQ0nWs@~ zKGAw$x3IDDc+4|;I^1bLO^Wra{Ae&3DT*&z{z;nJn25gQJoFJqaG!UfCW^&@yIM%Z z{_02x>z_C7?7jBZ>kDOSlp}%LUS?+iBJuOdVqHYm5Qz^&ioF~kb!LP7UzCjxxZjxStRFRg0kv_-2*-8GE%wX$p_=vBC;vOg*c*Q^UwP+e5rDVfcGEd zL*VOJSc1Dl3%?>2S4&J~Y5CPkXz&}jI5eOpU&-BIkt>WLe`D11A`~GkR{1EMs1-ID zq$jv+s__!BXoEy4L4$Te7Od#vDv*Z={UPdwV24WtOsKG_qH+Y{jnHpBu*_xr-Ge1`eWr zHo~f3v)a*={2S;q z_?$|71PTtH;A&($^v;2H>H<&{H=xntV%%rtfr(UkTN7s)iwjGF%QU1tRI=WR*CZ7M z-TAav1Inz?%KJ-HW{RTwtsp_z7L?LKEPo*BbwO@RdnR6nlr|TK={0rCg*8+J0<|Pt zVL4$T$c%HGZN3*rNgWNGNo5TmV-FHD7%e5K3hWm&Th5+v%gmQtn_VH(ETo?H7)WWj zeiL#E;;I-uRp|th*)YMXo$w?&954R9Y;907uOPc**zmG#i5iK9iR>$LU%7esV&yvo z168^b8enpj#7S9|=N4dnO$!B)*_6ONv27_gBDWpIjY3vmZt8ySr@!o1^y0fOAL`tK zYtu1ME4;T>G9e~G5H~uEL7sWXBK*ne=m)qtoXz7zd%Vx2LPJFXP#w{k@@7N8~(;!Qqk!(0eU-dB6vw z(VN=3=3{WZki(hMY%^{>sW=N{L%Y6S8sH|JwKzBuzCo(xFF`Z^CVgt~P){+t(>9=G z(Z7h;FGDT5FvOeKmkej3q`X%Hgj{ptSZdbAvBpAw;tD*YTX;kCbAb{-uaMJ35q zN=!@h0yj$SGiWvg@PfRbdcB*#X$TU!zd*@p9%!U6kVY%4Yh-B0q?h$P*UMXLzdNE7 zSSTz8hu}4tyi#p8Nruq;r&9lghUOpj>YZ6rfVboSHFL3`Si0>q6_moDXzva$utq;c z{CS3m_<<`{!RiEDwdU?T!rSBd;hM5Ay5B!KDT}`N%7yywFOY_9DGak8jo+eB3&5htX^k8okK<8o!zwDCC9h}u# zF8cR`77zW3w*6zIVCjK{j5yIIC0|%(`Q5h6@eK3PCM#$L;KEtLDvvhSz?buxl7$&r zK+${q?7i-tAP@8{klua%v(68!)zp{()5;Ke9~?{3Z4{pM)C5;!Q$zAV&z3od~W`nqSQo8If3TK8VUZm#kjtcQ-* z{WjkffKNy!77a_06yB=!`377sx{^)iHA0o|HSc4>eFWbNM|N1%UV!&q!1sm&s*O>i zCPo1VuJga0=IdY6gg2S@{BxSLupfWyoT+4qE^jfQ!W6iQ5&2Ef3{21n7Lv9*f%`+7 zYvx>}V;^O#Us2`}*+;t^RrDqZCbFPV z;-X?!4OtGPw_bi~jXAKm!RWsPcj;kegDIfMo|AiDaRYDd0%*vB0iqdVoGkyaC4TKu zu^#Dh1NTOzPN@}~uD}FsY+=k#yT8kOTTc@%>6qu%gCDReoF^@h@tUS^uqR{v@w7F) zy}de`k%5)v-cfAckJMeG9SgUkZ5LelH91Tn%C*S*O4R!*lzF7hKMe0ivgmQ_?+)Pq z0=|59$k@RtA$hKG(QzPi01p?PutiS)s1S$5gMO#KhH^Iw3m(&e z)9FgH(k>G7&FcM#Gn4f7>Rs4s22oG1D!0cSLrq`rU4N~EWI_7NRlf7_f-rhhE+|Er z3EV-N+(B3m8-g&Z=Y@0Bbp(LO(?w{0?!Nk{lNZXO8cO?}G{_$$OkqpO z(Nx*)K5^)iv;ib@vpwQ5`i)zY@7&p@)Xjlh+CT`FRM;E!iWS2Y&=GQk`UxiP#5~s{ zjdXh98W@KbI-{YGQJ`zYwq8D*y7Zp@d^u%CW&z_CsLo=smiC$6 zb_4J`cK<`-)uOE=2>t!BHq9(W4JyckL}2V9Pt$A71yooqdXDK*(}%%Lgwoo z3$72*WvDIyzu|=QPLd%Da8b+lnNm%@`+nIK+ujcxL$uRNSN2b76QQkaBF)C@>#3id zUzS&&mwjHpg-OtWnf5+=>-}Xz61De}ZTRmC{*Oyzi5S?nqq=40nE!`_sJw!_QbJR7 z`#Kb~l4ypqSGZ-gKQsilucSsP0j$OlO}yebj&h6=i*o#|&w7~%gHx9%!qV;E-d@uQ zj`3xjG~*{BIctkF(8w|PpubV(x5C|Yzmv>3qQUp`l8=v-)vs=8%v4;ChjxDVZu7y0 z$wz&Li22P9T27SX>UEj(@qE?p_n!~nv3of0|9NPWAO)j)Zkqb-;O`Qy z=Z7S=A3Ks^wS%wCj9RmhBy85t%g&V8P>=uecniJ{Q~DV@#*XtnPx5s$2`i%98OiSZ z!{hdmoc~I+zr4IuQ|JPLoR=Lj%Q$izuT)VuE#kWjBX~d~lL;&r%xt3Jc!a4V^HqIT zMN{(>UZ$^hr!*>oykJE!$w?e-UB8WS5UsvQ{4fLUn1}iYJ^b}rz5(mPw2ST?ye#nh zUiid`KxoI%WjaR}1YAb@;sn>?EEG75#q#MsAE+|VGWLi@;6Jy$<+TA<&S%tSZa89Z zIfu{D8k$JsVWaqd1WxJkNA$kyL}Sdm0e7XoeXn!&^>UOi-Twrv8{JQv<9+H6XTSH- zLR%=UNBT6eh*F|lfMNRV=S*EoT(0+N)8$C$fhV{ik+Q0`AA`r^aUI`T+iS{mh1ei8 zD|=Z83v5JUU)wBsgzgG#e zc+94xFM+H$%5_*0K;V_oSt3Y_QgY`GMlg><%~gp}m`)e50Lnsxm=Uv(hVL%j)JR&) z{@RUB%U~uOtMEttGI7%n@wLSbZKnys+$$bE_#LKQKdF~W_I z4AWcVz7h%29l*po)_8R)GTa;Cyj0_iHdc%>uA%^Kr5;&9A4_a2zJ)>h))ZFHej)pAS@>@(oO3ZT2vMquk(}f< zBH7!;cx)C}MsWL(hDuwXPJZ}q`&885ZG=oBmWeUcCb*_yM-0jtPOXLp+$Y*QScA(e zvL%-p_T5#9s#DvWdXA>?cr+RXGEAGDgV|F7xrYq7V}7{oU1R^bREpt0jDzJCuy{W! zy2(Efgb%Xmy#+fc!rIR1v$ioZHyOdn8jMgEi=SJLkLD^iW3LBeIMbP?tonXeF5p-) z>{r}dK){Yri14OVJFE7t^WaiGMRHF;PC^9~eI*4a)-IOr1*382=+crF!euUtUVHx=1r-OZH19ve@$e?*)7~u@SA}9qHr_`Xdp^; zv`(;|4y0zLDHuA_0~m~iZkm@vi}sm5{Oq~ITT5z zWiSN}oH|vZY7Oj++166EKK@Z{;u3msFhNCwYJK+y9{4KN!ZZ*ER{_UvQTyf7|=#iTs0h z{PR^W4yk;pQ9`D=(dc>$ey|9_1>0I-0x!Tqvk&yif<2 zVq`As<;W}1Pur;F#J}_eB-DX(f=Mnt$UPnXh`Lptmsd1fo|(;UP}k1CB5I9+SzHg4 zUPDa~?EVKJoLQjuol<%rnhc&2k2(oi{mZn}^48~Ru7d*A!|Obhmf*W*L=qjF&$MXU zXs+rir*nj~UOiG@gz-(G>v^HHQ9a)q!dh7YTyl$!cdM`!B?(exdS&}fZ*vt%I5taF zs6QplvX82BNHOaoQtwPP692Yz`{_U4cZ08s-|=Bi z3jeVU;1Jx81gYE1EMIqFHS;I?320NDJZ;;^Y~I1#)n8G5IU)le9vQKZtKN5gJJcpG zS4P`|&E!z)L+F4nl8MPOWyWr?t-z1u-YyATktU4AATP|cGSFc~CZP5%lyAOnY;s^_Pw6+-<8yjLsrHcP-{aeqjz%XclczrKC+ zyor(3RG~usld-O;leeixJV$AjZHmc-HZrc^B1`@$HzZZ9fQv%?AC4U&@OdQYXn|I->O92I48FIf^D(guf_BE^X&;3Ir|OlUL;7%KhM%N zIJLjQESW-R(EaTQg8zOxgl@-WBdo5Z?7Z{oc0@Pw|0xg}%~zbN8(T1$Q=ajbZ3sh9 zAlzS5wKC~CPiT8b7MVWYbQ}pT*x)g2I@)6E!uw){74?h?n? zT3bI6*im8>Ripd6YWd&F6YR#OxlL1QaHZM}G>hE!D~kT^>HAs%wWeo1k%}s9Zh(x8 zT6F)%xOJa&lTA{#t&)XQ4}VG3g&)je=-}ll zPk$cM4+!wXl=JlJ708yaW|Z$@LMgr=9c6qoFQhd%&)e*)lMr8{anFt`K|Im!!?58= z;-OkIVbZd44q_F`h7-hGSBwZaq^EngQsZ)NckIWf)paN(upb()7TD+(G^HxIMWI)d zcvD$uH>J%6Q!F~QVeLz;QZ11z_D%X&U|0RE@+mrAZ$+H72~gv>$FJSC1)uX2n6hb{ zlI_tu<~X9#qOnQZWRvUnxaVU5J&c^XzxA2Tg<|&a+#dfv_5^s3R39S| zFBNxSK4hL;PV8RCp)#yE;#`@&Ulcn~u74vx?lBaaxpw>Jq#ZM4bx>%pG+f?#cMDf8kcAdk6T_jtdx_d_< zW1|!{t^%ciOq1xYzf+^9Fj28VPqFB<8YEYFkmDN|p?#n*7tm4NMFd4+=2Nen`!tL& zdq2Ouzy$YIL@oBZ=m1&4@k_f9bdK8<+%<8hhahm?J)3ES?@lr-*-X34c=ztEadd&pNB zh&NWXI`nw1^kL?Nt)WxQ3MGP*$Q}yL8givN-+J%Nf`9lHqwrZly=*HxGo5%#IWFwL zPQmM`FT2TwXcwwc!Y{7QPBC5Zm<|wrbYP)UM!k7qeeYK27Qb@AO;Zu-B%!V8^O(3%(1Ql(IW-3VTbyEwOTf_oqKX`Nu)p53dSTjC2RytgEWu1?9!P&}q)Q1e@#w zW@tTw>!FZ5T`)2g0dtY(sUGM3_ZGCi2V! z;6Skl6p?zEEQI&l!#?~5TJ%h`GnwIjS98%N9QDdqIH6ssa}vaF2o69(;tyd|{r%A& zI#3YCeUOv0Idj$&5&Ho(DC8%5hiquV z3{(EUu>g){7%6Tjp5|B1)x|+y(hlP_i?lNpB0Ddi74*gxyE*xB@xvV{mI$t_5oqqqfRM$N+AZv;;m0M(NGeLgN1s&Z^II2+AHtw(*BG8q z_XH8}d*D_Sdu&*&Y8Iago4Qwn5#LTBz!tHH$*Z?S!WZu+Nm5dN1*KsHBckW@Onmcy zA}Wv)iX`;AEI})k!H*v(!yzgZJ7>q25SSqwg_8@T6B4PX-mdGXQKnhnlbwl3i7lhn z93so`OG6XhnysN)?2F8=B<7$?4&@jXuq%j=){*fV= zn&d#3!7(rnM3X6lTl&pYos!v}&-Q+LJ_a)On0n+2G$x@*BfA#U5#&PT-(U~yrit)< zumA}Q)JUlA{(#FKnGsJc;8?ec8l`IKBk`vSE!S02bAL$-@*r2Y$U7rj=Gk`f@3bxM zRz}5q{be%%e|u3iQrlG2Qp!RySq^!@T)U7aJ+P>WYC?9Oc0n5VMqEjFfL{6)&e{NA ze;AG4?RiI-0a=TToDQ9p@f;9lQBl^GsS(7U6?;=9cE9o;LiXQWO^+1mY;Psb21C_` zz#JRVlOJ(+97+LW?DeyR7GdE~B*wu+b+430tON8Fga70<>We~ICnorQN&Qj8D@QO% z%w!L0JlX*!%)x>NhjK8bQ$7=oL7|QxbNt7s8;wT8kSA$ep6o@|=iHrmviDiRNubQ0 zW%FWI6f#8>go#Z5M&XQ7JlcXNkM(2&M_Ou?kWBw{hS~DINCaSpus(VM8gf!suM3C4 z&Dy$Y%e;|~PFVN{%x60lldCK&y@V{*>FrFCh;_lp+yOyGiX6Fv)^nIq7B0U+6}IUw z03h|dJXe|7b!Gs)tVCMxfziHoVvo7L;KES2K%HIGtZd*C7fqe1gqdq!%zHAw_{&3) zM)b~-7|Z?WNuZpid4q%~0k#>gAyNGtN6`CVR>v!Pw8B6(GvxG_{?ez|G)&6X52+R9 zGcandQ^_}C_WG|YXf-RsE*p`8Wd!EAavQv|KIMMHo>kq6m7cL26fPB=tQ;6TBkB_& zC{3J1|2F;Z8>f=}IG23)YX5TAHceF^Nm-{FIR{N|Lx*xzsGL^~Ffu9>E|nN4ir_s^ z(E&to85F3FofXyxS5GGIwT+4^K9Eq)0V)rbJ~bloFK2UQ>cr1DMvy(^t|>Q%vgy2e z$F_>9B!v6_@b!+-kw)#hZfu(!+qP}nwr!_lyOWO1j_q`8+qRvY>b3T__CDjBvH!oL z>K!#_%?Ib9%1Y`5KzjJCd_KuJ1r*o=fAdMbrJ z&@-~H)R`W#-H==yDBXX7T55~*o-DWWw*^>5_1xWOTMV}jB*S-w>Qk#wDz}`pUs>@j}co z7fTAc5>*5?8aR?H?T_)D;+!&-#c=xmg;t{DIUO{Vw7KHe*$m=N-(Lyp(S}8jejRFT zfMN8DmQ=Y5WM;&rvuSA>kNLY{9+0-~nmek?a)>ewoA)UQu*qxeFBn zYYVwMQ^hd+c9|asmd)UFazqP42STc3Sf|0W+epd_vg|49)}AXV!7RTC*p6_9W)*#B zy{GehQumTJVU6OgiA2@>#?2WuSFM_K~ zR0N}nl)-4YP;nv^sM&S!t3?T2RYdvA4G!0Om7-Gb#_8TdlV>UysN8f5JxP?p^Vw1{ zD^8A~l;A6GRN_2n>8zLgOEKK^M)gnc#uh|3nf!HHv9{LJPzYu;3p26*91`3+qRSOW zh^gydQYUlx2Pae;sWb^!AC(26jkw`DdhVv&mLgzgswRVePP3bnT6!rH&$*8AuPyx! zFf&&j{V2mmIJ+kv&?QlSwNh_Z^HQwte7R3CQ#}=rP3X`ZpyA68)j&@D>~Ghu&B|p} z)b18P6zCA&PVFcAE`#64P%%%BH48LtSmdinhDoxK_zQdI)rjYmF8@HGorr3ugjHS1 zkgC(=XQuQJbm7|qjodCOW0olp=9x7qRBK5<6=iZe#i)XY+vaJR39CatR%uMc=q=>m zX+u?!Mg zP>0y1FwpHQt*U>JV+%)6soppv1XpkX8*Qe3`oDHs03)#hhne2=26Yq?X`kJq&f$TpHX`y z2JbzH>`V5R@&4E80j|_p2JMx&lfemGWMv66q3vdrE@HR20xulj_0KmYTiv;zksi7p zS6S>PQaW-=S867xe#IaXX3t!b-^i$R2TJ9L;nx6_C?1E7L>;nw?eS&Cp67%KHUt1fFPo@uT%Dvz1Opt>8Ls| zKzGD>)C{wLlU#SsCM1S9Pj={#0p(!}tI1P?6qx}e^)6iS+I%0MnMO?W;{O;r{r_#X z^-7c1;T>*aP5qUPD>NsgNu0la?g6;^_&+t|e(quRyyOEowzg@%D`|eZ%324nIhWin zhWYZs8(D)jTiz#&!)2r_PRU{-@Ro3hvb1>w8QmErYOpyvVM=p23rS48Lg&!_pEu%5 zrX6fP%(EB=q(w8bWoH?{!DwM{G3-U(Ye~eENytBwSPvRj1%D<4oN73u!;gw1m)p0N zKFD{K89=;R_1QW3Bj3{G-ri+Z;xb8w`L$E|+sDqb z7Otx}88T|`H_1NV{JhQa@r>N^E`LG6Vs;T%$O1o3%nTOe7}^*N=Ur%KM)}v`@bM3< znoafq%jMbIXp;;E^h&Z&k}u^acp>Fm6*c@=4fc7lNtHfaw!;yb( zdq4E%bg`X7?(_Wc--9eyLL>MX*SY5BSZwhSkZgPaiO!9&oAeOlX^ zZRtFi(=ccUo-fa+XS1w8T>z>QC#fu*?6aIu9z>I@Nb84i%}Ixnoq6oRfg~#8=KAss zMwyr2PXv!KnJ_88st8_Kd`Se}|2-a5F~CGaN%44RoNNNIn?zH5!6SD2_5JLk(8%Hb zcOchy61HY_DRXg=#d0Y17rIJie|F&4xrMtx_xtYOlNGo0OV9J@W`5_!m*?xxlij}y z685hH+pqE0zN78Na$vPug=8uFg4+6IN-kO`1?Vdaz{@{xk_DWp}Af2As1w5zky7Jy1C z&@I>n{2BkMbUGCcLy3pp9}3z})|;$Grf%jCN|A?ji&I;+rkf`IHa zg)aM$0t+(DIx@N8jI9)jQWCnW2f^T5B-5yw(kbse(`Y+Xy?mhb9(F5>7Sh8)JWwbV z)UxdA9O6?&>-j!~y!JGlz6w#`bvihiP<2@^aa-2?*l5w~`J}cjNB0pMDd}6SBE)wf z9F7uro#@%{eeQxhk_hRmTffwAuX$|8>u>W5b!$w59j*_&gj>Ae7znAY(M*L$8v17+O4ze%KH0M^YK5en5eb^cq{;v(hRV> zyf0?(JB>Ek4{@8*Z>jaAKb==!|N49{AL7`4zv{RK(A1W0EeT|0jv z1^4U6>Hi59BsOWsK8q^3>i`tb_rGU1J+`|5{I=iw696a4ao>I@;n4FsOl^WkKPUW$ zApIiL)d;&5Zx+F*l98xw?^@VJ&^Q8f4dJL5&w-a`&_Htru;V@ zTfc7m}UGjKP(|JQx)_XYrC{cZqYHQwlIU*CFL zUC&n+K0nEbLJ5gT33Z^5K#1zEp@2n^L|0W)l$4Z2g_IJI!6DF0k>PlNL7+ng`;&4m zbSO^cxljGPo}atFr_1Nt=HsV08UJR!3iRl(a=2B>=Wx4OdtFyw^ObsX=ZEq7-mnYr zXbCxd`%EH8e;UjO@~b;vexgwdOAE|je=s0}1m=o+8nR*^fnY-=?^qw;42A`Gv8F1c zn4E9ZIHPM!!xZ$+!x8r>d!M0`HxJvi_5DiWa)dt=0|J5x>A)JF=o zM4blG;g+AJJCxg+L~kK$s649my|^?J;3+WwyFk+W(JK(hIB#FDDMbh`OL_s!E(%dd zCv>^symY~7Xk!y&h2Fs28wD(7xM()aBaBX#zhn;z6itpCr#_aM#Tw}IAS2RSDtjF% zEH-*X6U)m=36qoIJ}Pj64N;OoiQ?1Uh-wd=E|TC%3^lR+D>{ue^o?|Yj4@0EgDt$k z%l4_$IA!j&YL-R0<1q8dANhJ@R$>Qg4J>G#wTxU$m=t6|2nrP}n3ePQDP|GmO0w{S zjASR5jPulOzoB8H0Z!#FW@n0QSq_+m!&!7j^kmC086~j1dc&_7^eAr8N#$ktWqPy+9mjERcdCqnLT1mlp3p>(bLuzmMUN}}Z1(h8)^2fl+*r^U44WBl zCRRLY_r|aS`aJ%3lx1eRQY2Q2@hjq<-r)FHePZe(&Nb^HoG`}b5#ZSU-hJ zK|gRy7l?J*R+^%Vo#RQFGey-)FqyoTi-=vq+Q5|MVglGG;ahy#Rqqd}H;YLyl~1Wi zAapae#JP1-#(Dn^b)3=;D5{bXV@@8V5&`p5Z*C8y2N;pvjlq}}+%W}#CB@u`oQ`$!hiC-0z%!2vQetzVQ~Md*TOvlM`h>HO$sqN!FZ%X`cSxvO!38-5-qD z8dLRW0KFW-rW$m&9;C)Ew`dkqQ5-SBT#C9hEB18xO~x&0UDv)1geCultM2`)QME#4 zv6K*F28#0SycCs=QETTS!yNPF7>o#hk)5-Zx^X+!8I9Nz7-dG6)U>%k7tMy8+WgGP z+}1}nBbWy;NehAif&b8hzj<#39NPatXjn}8g`Z$Q?t$(BXaKbE)3CtT?Kt^OAP`x0 z-mHdpZ^J*naPSc`E!O#li(i)?0Pg)GNd59|0(_~!O?j3><$keS)$n&(6?eyc(Y5Zh zbU)G|sVlPrwf66D^t?_2(p;ZB%dM>4JhfiMVrBjrR4;(m1mNeYv^QTe+LA7f=}2QL0rgWT z{Qifi7K&E~2Q%h!xg3q9(#*m|GVa3g&Ab3)FHZkLs>>vs>aIqqn-z=`aYTMgo$xeT zXh@Wvfq{Yt&{>`!cb5m%STN)_ARL-!SaFMC9fP5L4G?_Yq^QUf=~d21e?~oEBnBf{ z63+Arh=8{9Q(6Rk8ciNx28reb(oXr7IRO+v^+mEd+-j(9Q*T3$>aVOE9JyXcndPd| zsQDEA5t|IhBZpo2fT zI^NoA#(cS1pH2?q$MXgJ?!TE#{_1&b4n22fjihpW3n*NKQG;T9L$c;6{G^Wf`-rTF zg!08$h7JYC>4Au{e2+s#2A0k|owj$4p`wN=XbD713A!_ELcuG@dGJSt5CIEY6yik) z=m+Qr6e~BC>A~Z^bU^IXlh)SnjB@#-96 z(#mO=&&q0T%~=muU$0Yl#u#Nq^HD87-|f$(Rj7~eb6AyrJQUz!S3*D~D}h<0 z02@@$G*tb0kIAbq!q?ecfiH`%uY14PC9fYv?)n&jSZ}!YT|h@T7M3qc;M-p@;;iL& zn&V$QE$M4p>U)i2iN^^$*H+X#O9{_llA;3(=LW_7OJb9tUS^BIza)|oajSzN;bCxi|TcBuC zUV4y3#mRyQXn@wd-{in`w)^=tJG9G=MAWY_s(5!Z%MTx<#A0?F(=XYQU3&CeiQ{A; zW<61>lT#M^XyX>z!em2oJmx5@dS5JZV#YSdqpgRi$msDCBx~VZ^5w*7vZDD(9Ap3V zMza#{%yl?~_*#aTZm^yrE}%CwqpahVobhpH@#YZ{wNq_TzLITA7Idk1#v-#RtX1zp z=FgAJ1LPSMGA0B6UL1dBfuiN2d}cJoHeWyD6j`LKB5p?W?>Xh!7}vDfd<&D~nrfDa zIt1oBq|`_flhu>*z^s_481KUas}fuS@|{RQxCbR*#%~7$?Wg1T^RvsSPCZH*ceoJ_ zM5@N8X}1)#JM4}kBFvuGSEN79RL6rkKt{{jIR4iqbr<&KJFAmkz-ll7b}5W5?( z{w>ScLCe6PaW7Y6SSk3+anA!8q=uLNu8TS07rAqFRn>tt){26_Cu?tpfOfLT=VXf4Jw zY{d<~$O`q647XY`0^gI)3QfpkRw-U>Vq)+<= z(7A?FAe)JCPqc*rb{0CUZJu;}9YDaSANlbU*c(KP5ljfOD{sFpONLx_i6a2pW1ly zl`Q2U6bAzGgnOg}34&OQo8)l9a$bz`-mW+}2(~i-)DgZ-dBjZqC*TlbTvqjQn zLrPFf1AvNy47vFMR()9;7Hs?b>lyf0TNrLjSBSzq4^sprV4>bfPCcA7b;L)ZpKaxQVwEoy-MM{}C<%9uG0?=tY{y$z58eY=rpv5v1q+qiov^#LK5uJ7aq;R~$r@?sh= zV;n&xeTP6}TYt8_#vPgDRpC}|PKa+MilLG-f&me5bgjbL!*(9ZH6?xlRj_93ix_~q z@cK?)i}k|h1Qvud#8o^uX-F67F(rvp9##Sx}qzsI-Qc}Q&Tjj&ZT_E<126IFt zM5c^w`?X*c7;8S>{y`D2dX!qSpQ_HUu}c=ucMu5u4u>s3y~JT_r=b1ILz|$g#>Z#0 zVoG+c*#vE&zQ5kWJpu%*AMAPW9om#A{(CaI#TI8YN0uyj)&UVzL}C==4BZas_W?K& zioGW7e)*DQO?JfmU*e^zs7tA2E3J6t;#bp*F~O$382B+#iz}RivY!aDa2u&WCOZ>0 z(*hhq3zK?|rN>vSSgPMvtKa#eZqNi;s}2uV#Tjd`>$IC4mXeT=L=^E@6+ zD-mS8x<47Hz<5r9v@aPk1VKu%-aaP}1oG&g{Rbr5z8{yeq|Gd=pMTf-!3)7C#f>XO zN12ToKbmwEzcmDOQ=?=s*>46a&@hnn~DMeJnZV$e!LmAWm3D}_q$ zw8cK>INjS_ZGSOY9m1|A;4)=2H|s5e;!!LmSi{Fs7SOqZ5VX$t%uzN*b2^sTiG!-r zcL$m|10zTCQu0jG6vQ!)WsZs{8Bqk`h|(cRw|H``zpk1zaEqiKs&g^^*bj0gO5iTy zP;;hQnb${6UXx3gaUc#e<~t@PTWUBg!%_yAR_fkRSx5OCl4?GIms` zDNscf4Vp`Iy#HmGG;#H-vm3;D3=L^x6RqHH!UMmKzj7=b-#kxP=L1OO1o42pD`m}s zS%~rjIWX4<%U)tX(Ln()7S@tDoAeX*LE<$qzLtNEiq8=DJhs#8p`i>!hH*GZ5eq z2p4_+eoE*=h2N}C3xzh%=ndFfssx^1_fyyPQ*PH#H z*P@=!w7EshB89zkD^@t@d3W=0s3M9SuIroqJC*UW#`_1W1{2x$E(-I)e*cxm_ll;fWKIsWnB z6758BL+Z#dLRnJdc@AC-DhH;9&*$B^ga8>JlS6#7n?_5W`V-jCeYey_A^RATR+oqn z#De|dPk$)1A>9f7laAuuBx~BRKe8WvQtkyGutMidva*g&WfBr3Xr8q+FabY6tK}U$ zSEbc}W*_>a%KK;{lV8V{J4mb?N6PjjW0~%Km?8J)-{&%Q>MsrM_;AiR4^j8lYr)x3@4}!?*X<((#B=4wH#>b5WMqUn zN2nS_+sjx~NHp-^YPpCA^3ZN)s_zUX_7vYCMgDZHbzHJDiQ}n6dIlYOOCgX@+tP zad^<(?pQkf3K9vfp14V=Y$gbNS^&qkFAGFB(uT`4pJ%SRiKRW;OX`&r)E7J;NE=?; z1cnJ8H|hV$>}yP49ll)DFqq`vqqVF;q#e;M|9yBkh^nxV%R<-hO5UH^s|*B>EPZ=8 z3@9<=Lq$sj=&xQcNum#L0l(KPKxtMwI5Unie{a6t&`*P!n_uHe`i3kJJxKW93BmtL z_P<*pCe*J4rp&LMc82vcZ&P}s=^=Y>#>Pxw;?aD$D{6I+62|9a6gr>J5IH{?Yy@)= zPPHy|>kx#%dyNM%6q><+h`>96sDQZ8WX1N36U#V3*{wkur5I1dfJKKlj7(r|Aa6nv zGlP?iZ6z>}z$37((&4d-1x>-;km)5!6e`WL4uJ?1GX7-IhS%E{1QJ!@ zr{Tl8dxCw7+S@`0slbIFJ4=MNgM9(VuuQ-g65l2KrGTsF7=%O+@UvFZBU(~&{G-jo znf89)V*at$=*NJI%fo7W8~eNqeZG;nlr#Tx`ZYZ{zh!`GdhG<50 zmkc9PlGL#N!uO`<#c$rBOopP#U+kjUIA5KK>fM^lvWG&0-k)vh#@DOj)C81P(9-(xZe zHQ8Qdh73pH8pLLRv8(m4& z1)KtZJwS2i)b((hjRIF0@VI_77@Xf^-{YQdy@N0&yR(p^BM9 z0{zItB<6B%X2{os>v*7|8#S7z>7pYcw7_g&&W&krm1Y>vW&f4&yT&F^`Sj+upzHz- z#LS(3*(u1SNwn2p!oQcl(eHGzvp*se`N@#hR?Y{&B~CW;J*iPB(DPHllc#9}>3{ut z2uxJ6W*sS#{{c!ZT{}a-ymA<@Mg$v181+Ho68|(Fif&_i%G3x7o2V=ey+p$nwcAuO z(j%*eBL*p*t0?)$AOjkM`Vw7IHTac3uyr^cgf$)6A*DF8C{iClLPuc?NoF@()h!@X z7qS)~U}?XNFASy1YTd?MQPOd;hscvR8T~zM7!?7AmIcyZv8qT^S_`v}X;nrY_)R6k z#*BK(V9XpVN>`t{iE1cj!Nh!;AuHs)Vz%{*&yq5OXu{YPh~evoXL}wiQ`ItuB(q$t zyMz(rTbV4mg}3E>M{=0N;MdoHRnu^qW(b{4nc1a@R*P&cpR1)?Sn4@~2E%qIs?@|= zB!Qh4QLauSxmD_5)a&#zr-GixgcDmm0_}cyb=$A%`5WrBxNaSlF&}{sV3o4G}lood6rbR&h9IULpXMvyyfsw{F zS!U87keHLjP+icUfDQ;GlJLCXZT?dWVT1^D=9~%}RiorQ(+dO}puO{1o(?rb>qP9G zR3j~kY*(n?MCTYXdRm%`kRi9&n4+s3K?otYRKK!gO;te>=mpwZp~P?sc2Oa;K|~%m zLCEb$E@*pKysf4u$_h(fgk(uI(yl&Op53zT&*L3vd%HK}=@5>E(0j82KVJ%m_y(dR zRRrZ+;DQ6o*LI`&r3=ltEcst6wv$^5OkQ3i#=V~rX>F~*-Tj7MMV1({==6Jff-EPO zj`lI~McJx-%H}0uh=8j3foJpV!7r|_Lsmd-+J(t z({G?aMvhiAzrY|6_6>Fr*~ze)GJ=9cq0mgDRTOLL{TLKO13A!uWhk3;5&YPS!OlR2 zr1m8iViRD=c)mJw(*~wCDwGO6U(kCBgRKSs5+L>6x+}+p@fmA0 z^5RP$z^MO?bfJ+9F30E447PS+EV(@5++N13g|(2ruaZGcNT69N$cXMo%?1m#_5XFh z7DZPs+LKhr@+V+aFC9Nlhn}k)Wd!y_UXg;SyBO=p-XflWR+>_g${5CXV7@`sT9Ab$ z)%00NvKL>>M;&Dl2}9gu=2w|yKelf0MTCL|blm_69TLWI9m?=rN*vNI5mUeo84VF< zE?Cp5SbhesNE(}jU6ia*$Peg0u8JVj%^yzPS%LxJ#q2v|5Xe66e>&^PmgfF| zyfosvccR3T*j7th;mU+&=SrvGJ{T7@5gjQi?7=~7N(514lu1K$c& zG>P>Xg+&>}skt!7L}H*fpAj>t?qCxtkkZzEZ=rY2-*>a73?{bm&ZSgTdK&z z8Y(1=b5c*341oe>H6Y|3PAyB&r}_X9Wo`|8I}ns7Sg$~ADaAPZ>h zX9rw(S>_Z3zw7%*+$Ug(oVA;Ms$$#4$%X5I;t}lUsycStpCocN5AqDqHs&_vr=X z_AEmbKvJ;j4j6Y?6nH&P9Sp`u-qPI(;fTe2`Jv9ZA$8SnVZcL znVAP|U$aE_BZMzAo}(C{i0Bd#P|S5z*^c;I|Bv+TTN!jk{qDyYdDB8SX8pT6!5}>!p))k*T-u&7GAn!wt>c6YW~*S zqcTa`2FT!@RO6k7cke7CXWD-q|<`5oTZ~9bT^WMd=#QK%jFGja+rvKlgorXg9k#_GzU3L z0RwZN7A5E1f;u9=6xB-Yu7}AmbfrMGtC0h(V1ZX)5&yv^GI(EGeyBPeCJ5M&xEfVv z&UHb8{?}W}%?BpWq6g<5e&r0AAC@9D^mQmH-|3sFo2&1?*Mf*-=)3`OX?w;2;oCbc zBJZc8|MUxDOV{BAKIt#pB6Q7ptrRPHekdhuK`WxyJC`J?f%8#$a|2KO@0*U6lzE1tLNIm;Zt`+!r`{)aS)q10ToUY*6dG9F#_IsJHkpPRm>0H@qzjhc2X!TwX zf76S-r1N#?y1vQ(xrmVAFL*eUVMvk7dh~TORo{q@!&mnKD-ow#H=gx4)}M1ZYRz+h zKd!LVgNCM}yAQ%%W!<2!*=lvzoU@*_CWHEC_p^dLmcV0?61XeS{RNb)H<~E}^*FM- zoBnyM{}4XsbjlP0ufV{{_<~=AK0})acorF*BjG5|ME1mXM zdz$aI*GY)0F_J|<8oG9X>*^^R#066aiE1=c0oG{bZh7BXf-ZSoFST*haJ8rIloOV{ zDu*ff%=(bz3`I zOTayXxlotHQzidXp5N3D185Xbc?Jd~WsnQ-WUxQ(sv7Icg{oBU;i88Duc#vZwm-nM zVD5wkiJ64X(Yyu@*4(*7u?624WVkVFHrw6b-EVTvqVCp=1{}ET4b4PttrtV=$LfhX zj;CRNZQhlgZ#b@<)MP*QOUPZZxw*NRYPKK# z)OGOkDJ~8!h(0>`*jWzQ-Qg@kk2kjVR2l7|$JuQFosciI1Wv*J^0skU$wO>bTwZef z4i9n(S`SiT(^5q+rM+ec@Q7`$&2d6w?yUecSUy>J|--stb~HPT)D zt;dxoclP=C3zxkaGu*7j!1L|4iHSzuI zbF-VPM#>UZ`uBA#hCf8_Yas8c`rK_QJDD?>W7O_qy)<$Qg2yg)0cB@oTs zuZL%6W`c@v(A1YGBepkJ9xfz)p|~t>yUs@bghP?hrFFFo$~Yg9+{rdki9WA;7{$Y= zQqz8(`};A#(5d!=Zp(923RSxLu7L2+mlv2#^~0+kBp=RQtyofxi<8?Xk)&ovPD4;^ zJ=Z5di&`0H%01iVE|K>4Lz3@a(AB4yYKByoWSFITez*MB%)k z)J#0+|I7mT&sq7xgR|oi=V#|fr^dXwu;i(xmU?9n@+Yx>{{e7&!IVsvs6$K6rWEOw zYnGn9C+e66>kpwVn`S?w0X$n@|?S1xfwc|MtQnAI^#-xNIUMZnklZ9H$%n!K=i_bHN-Fn zAQ-}erYm(uj*8yY%1V(55!tNR%uS5s!B39Z)%Eyp3rf@(1~AhrDSH@~Mrsg}RoSV( zV?h`VUr-rgk4!0tjGH2+&m2g`z@Q_`CL9S1Y#$3KG<;P9pw-xHPIm{RFA0QUwK|jq zF3HMW6y;lQwr<@bRj0bpjys+10gpGBc!z~-)E*PG2DUNk?++EI02fg{3U z?XzD(MiAh6G&sAQ?L-RJS-9`wwch+zq6f(KtVQ>Nu&s(Sm6{iy`Omh;~~ z^N2tC)1qHEOR6qEjh@cZj~J|>)SAM^?sn&7iNa%DK9`oQ23_+2p`ZXH~@D@jzt~BzbqkgzbMA!j+4H3RCScoM3(%d!MyeTb&sw)xuZpUbcp)<4R)D`a}MJ%FqYo^942)mjDEoQ9h-9_c=>qDsi)C|W~+mr$DaF9r(*dN?_Hx-fU=J|b&cj}rQuotut ztbM=&Vq^yqx*!M^a0Z7b#d^l<*kXze9rNb`bh`7qzdqJ2k(ozXY1g|q^h6?5&G3$1bNZUUXXAjMd95uRa*853WJhd6rA`@uZP1Wa5KXbKG>(=o0Q;j9OrO@;hov*Cz4P zshabxYR@O@YBG8qL&W@et4zPz9&QaNMK}dHza)En=6xNj@pOyU*+Ngd7HoDnUr}7H zJGsn}*nTxTwIMIotyW-&C&EmP#2|Yh-2dCYJn1#mT;;b}`$OkfjBDuFt%9J$!nJI5 zrNQIv`}OxYcCrGuDVEw8!tdKcK?@I_+s#C22GowrFM)U*&Cl9nf{ogec&AjlY&o7H zzB$w*adVH^zLF1(0N>`mnkcRET!Du>6N6lNi=eiFv>dT;!fZ9c%#uCB&9!beJ5VXi z=szg@zmRQgvHV-xi}{XC2X>?C{ML%7^K7TS=(4U!En&kugU|cpL9j?o*x&ZK2B!j8{Pat64QgB=40ftXN;p>XsDM7E^vs=wY6H0Pd45Ag55)E={#*&Jd_rsV9 z905{P8c*;cF9hL^3pF3Ds@0lGK!)q31u*ZMFia;^bRSI#B!+6-iNw1YMpsY; zhKmWDTo_1jXhwa%HC3E9S94#m`eJqf`oW)V{QEaPD8O_5ZxV|Z0@V5rnj+!h!x zl%FxdQ01%rvQ`d&G zGQ$Xrd{1Pd_zY(2Y-HCtSu`Wya_z?OrSf-V;EBz-tFQH<^hByC_2tP)AUapqdwhD`IrTTqIDd~csdbdR<{nsVW$0IR zjlc^8AwEcV+wCYT2r#lURe^@Rlxfqk3tj~YzSM|u)38E_xf5>a|Bj1*Yol7r-d%+p`-xdn)jZwb2;5Q*PCmV$6itT$j~GoT}}gI?OKgmJx~d!(ae@bXx&GQN6Aw^-|Mt-Lflu zOPE)+k{AN1?dr1EzdFy_*e2V?K4}^{-p$>g6-A5&Z_&d z3}M+@4ZND>+S-UUquviz*kW`hKurF*z`gS7sHrT-psc03z{{|wb;^m$p2w=zM&I=K z#PL}i=ef$_uLB`P^cikX%U;(rFAgX3&~sA63Y3TgGz2id{8y=gIvwH3Wn|H)5T}^K z+%SO!X$JZ_jhwp2(*g^YJ+sX@PJ9$c&hrnx z$IAQ@et_%9o11B#;Q?Y5rwKjg)JvqD=+eOHd?Yf1WHE(PrMhNsB0@=t{0sZi-c)O@ z{Q>h{J6kE=vz~XEpL`zMDJR^R^itG9gW!8M8uzNEhR-U46X>`am){Dzs$F8XDGA6Ix;Yr)6|af)16XICQ$0(MgJ zE5>rT$OYpCL%OAxuA$OA)n9h%AIScdU<@k&AKYK(-ivE%-WW!U5KmVpcNr@t)&nLlYUM0I^jZQx8X;& zl?rTUxR7Q=R&;tAnfor_tOTIup7r_~ANz5e+*bY39m@(6$LFWBx=BI@kAlZPK|Jf1 z@PO4Bc^pP8+A2iw-YI-*R!mK&^S9M%sxu??(w}qxty%9PGFsNCL+i5k zUNZ@`Wd)zIWZKVULNDVB zsrWw34+q2(%G9V|7DAJ9x-}~`=0o=DR*hAVX&7IO7D^gSXtD7Z=~g&KOmHUTrazc_H0do`^?x`4(;M5vJZR$+v8oDOa?(d|p zI~p)ZO$q})#_KoVF|ttHyXVJ+m+-E(EhVM{yjNpO34AU!D;1bv0?_6#s#lPk?%g|^ zw!GZ-|9ZL#=GwZbwRVxs&zh&RrolT*n%4Lm&u%W`yHSpZYq@W2nfo0Ut6ip=NJo#> z+e+BEYfLn)xy<>Jv+QU0B;Z#xR83RHQ!<^4GS?)#L~*4vz>$V(&C}ZG_jMm#d8}1^ z79iJ>>j@0nNB$I^^s?|NDq&*ZDbQH@cI~ZefYkDHEQGUk(4tLR#%ubXs-_XJ+*n1y zWi9P8?SWjv?>2brIEGuDZpOkQ+cV&UJDC1R6$*N`E~&sakzz=UNpVs3(t%L%`x7@j zrLEh!Z>v>O-+41X`msBnHFOgnt!_t;%S);iu;L#w=$AovZ!G4LlrQ}3QpaEbO52ca zbG*;W35Qp#f~55U-UQDZE{IEItA2eNoa z+6nB=>pYg2d3<4^Iyz1<#7UK*SntL(vUoNukJPeh7`qj|DJu+nEE)(K)`LA^Nup87 zb3BQhTC5V#c|^emHXFJoFcVhkFime_DJ-ljn<&eIm`F;{Y~Ub(_n!y0Q7S=zZbJL1 zJnD_91OgGlo=q$JGd0Nwh!4Mrw~e^eS#VK1fdT!)+ppQvbBVK1GAv(j&D7E0#7~-d zY~x!u@6Vq^du5zl<I_+!p7U?&(F-bs&y}{8puz$CSbx{Kw zR?**96G;NO<$X(qwHNft*O>R2xqMWD0_xt<($!I7_jV$^L)IG@CEQLtugqiG%f_Nm zedwqS+Q`f5F-)+n7rl8i&k3h3PX*Zw%B!|G<@{gy$e>GDF*pSoTy z8xDtt-k7MNJ7p%z^?L+5n0%sMTtn@s`k+c=;7LJf#sh`OAWcv2CpdTEq?7r23=Ylf zT3rhXngRK4_N4Z&Gwl6}A}C|thr6TyhpBUlu5|0Rc2pJHwr$%Lt76->Q?YHcV%xTD z+g|aXz0djn^R@Qgt&4RrTN`7JF?;XNaZ1It3kUP^>!a;7rPf2Ho zmO!|s2Jjch(QkPohVNKzT^<&0)kx9sRjl&2V(Kot4TRn9(fX^Fx546o$=$l{ulf4n z4Tt0l4a}F`88=Qy)cd@tVg=SP&+0A=9>{Y){Ezp+U?PUPVbf~``h=0j$xQpcpb&e$ z(#NCU=T+jmqphTBj9;f|$8sGnEFL5l6HxBQdGYcC7F%ttG~2x1dn@I#cwzYjZe}*~ z^|rnXN52nnpW{k&Q|;!_rE?%+UxLgN@N}M-U`K=ds(ogh7eyG0R7~rgR)?vV-SfH{ zo>n0DPvBJ0ReM_$B2yWkaKd32KU$1R9(&_ae>W5pwK~q7d~~^fDsQcfcvuOZUmyFV z(zU@>YD2T3Om8jYso8u_E6h2A5TBKzFU;hlFI}lTLLH?xmc3;H%PV5lI^Cp(QAP9u z2P3a`r(--z2~C~@$8Z;dBl6+yj!LTt+LaXM8Y*pBN)GfqFDpu9i$E9!Au^oj$?YbN z0`zzM9~)BSUi((Bd#~kk+{EUrgOzEn9jH))w*k*t*u}ULtD~aja%y%Q3Zbh81yqW*i-x(24Z9!>8LFxLtb}h)wa-l?=%sv~BIoIYqIh zUzcOLV($$Q$yJlh`HlBh#$`Ft8sdSMV(anT93!o{pVQctY%u@a$kpq#xLu(C!VUU6c#(;iPJeK(W2#>KIAI9?aff2^N5xis zyX7rv5;F}Nwq0|nK|Mre%2>N!8{@WG{bRK8>u$1v+ecSd(h7qPj$+wG3h%hG_XPB75EDbm?)TC6gQ?rt*tan_~0%BKXi zv~zL!#_tsO;_I>Jj(4yB$dPhdK2TXWnH-h3fHvDvXP5hr^j3 zCePl!4xEvgSv@YLbYk4DScB(-ZMT_=j*M3ugg8Ii6dvs#M?~%T?Ok-mgyK z$R#L=YSlW{Bf8ls5BB6ng< z_&{c;a>-*M1ft0=6$-?%Jg-`fUg}=r1%iPJ69W2GJYt)k&zEh#m61j-9)2c=V@cTT zHeg0Y;sWjH^g0#sj!tyPXw*eIoY|P5x(Mh~rwi9exnQ6sCv*Fqy?BZX@)%0mX0gCb zs6We{Wb$+nT-WxXK{1;a5{oVe7!8WiXoDsS<>l8!N&dZLyx+6rGDpX7re+U3#@Z5L zrAC4sks)jOmbEM`EybTl9LysS#?hFtpqjv6t{?WG$6g&xbP4BB zsuddxc1wTt5%5zHBJCuMX|yIpw3=fwI4KNN#0y>Ikqb9xv)uJ(Erz3$QQGLTJO)(6 zm2c-@H03jOby%+Ab9lOk=*}DZ;rYI9WlZs*+Une0OAcWgc-wuuu{g|n)G3b(pjfy) zU&@xWxCX?K5Xekw>;5=~E;<}?4(pkh&4~eT-at+M>2&7=Kap5nlEd>ZqMbtzEsFVGuLyUHm+xvIcixnfbdP?EaF8?%Ek= zv)rxi-JwQ07nA&W(#R~f@r=k#4Kpc0_&8V~rt7VSDZ#KYg=J#>xiP{|XFGzP_1Yx+ zGusPzJOYUrLo1B#R0FlE<_LgPM*%_vxZHnzaE9+VzI3J}@9HXWl>NhM5*y;C zKaPpi_?jZVo}lx+dN}+OL!aSB^PY;0f#*|`!0cftN)6-W@fn7^TTCnTNYO3tbZ5|Q zjbc^|bbmM;!~8(2i4%e26S`him`KrnJ)Q%5}e$;7$^Zo!?3?d-H2Gi|N z-%_tGS1@Eg<#v`zGqIL}6ESO0^sr4DBN>rnNL%l(ZZ|!rjS;1)*IQ=V+Aay=DwazJBZzbm;{ih zz=u;!%eG~frlY>1!?a{i8@~G+CQcnJ!g8UdZXMr?qGsN8#L9}4)vRnTCN$OvD&)8- z!6xKa4e@OT=UH;tue-amDdjFst;dmJ&Cd>_ee@DM$5D}6Yza^IdiKwxqG6=u`$}`- zsvI_uPYU@VX3wO#62g$jU&Y;={#ubXl`_eX^zghfBfSv@b>2%i@=I8)Hf9S(5Y%HmU~&i__O;+TPQd&F(wf?26&os^_gXx%pn$HYk+!eTH%1iV zfV>)bQ;R5w3aX^oG=7AM^o-jFK_T0)rs&%(YrF2cWQc%x5fDf4 z4MJp~kP>TKdm(}*-4%gQ`VQtO{0c_{=h|Bj`sVc+Jjr}1jz2Ez#<-dZn9)K2ynwNK-^0o zG=Q6t!oiX`5XP_B`qsNWxqZGbAKa${CaAssrv{}Qx z2g3h4*H1*)5qRJ4KYWid@7Xp?ITq=afxwVXPtg6rg=`o4{HcmgC&QiTgT?8h_Cw9q zcc#$`c%V6-qJ`dV974qScZW(?X0!l4KM)=@SCBr?9apPQR$2VZ~#h11Jvav+XW4{3l9G>kC9@^ZvGj@dW} zTLY+MV3byX07#N>k@9DJkk{W>@*o*pBw$eiBGc}BeOJb0cu`K*vjyTZ3eoeOFc_9E zzDlXO9JT;R${GqZiRv#@<9}YzV=y()K2pNM zmEifontQxxyK7rRBT0&iRNvI#VvY}--o@~h zKy*{6veN9Elrdhs0ni**N^k!&P3F+Cu6{^p+WlE1u{BQ$(jOra@EEQkO6l}LI!{KS zA<34B2+FrFPWm@nmNIZ3XjD4(uUVMjNX1yV&`h2gcS21aBXx*31L!QlXEBs?bWvsI zOn4E?QZXIufo+kg_Sa5_rSI*=nhiH{`0C3Rtoe60wM0l6z zM32W0<1~q|B$*ieS=%uPO^?DV3ieU-VeqQCV$U!_<+_-b zBT|5Y@f~JXG&Hs>tqYaJU~uC_m-9}-X=(EFJv#L~C#h7`6Xk$qdF^Hlw|PFL=3`1B zm|KN$INz)5xB0K1Om((nElSES5LW6?!0g1DMXTi$oFb?U?)?36s?NKQBT7W~7uvS! zx;&n-CkNdsQv03XU8{MZVRIFYOi(Uu%v}o@U6AdzMrt?>q%SwOx!%Ior#dgom_w3V zXN8ul>=xUi-4Y~6Dkhu@4((Sq^0?!BM9xb$?4P{`%%D2Y9NYu)E%xn;(u zvU%SM6J%e%4N%Y(5GybL_^g~}#Hse6ejVSlbT=<4$?=K?{?7c4ThLN533qaKCd5NT z)s!+v#TtGgM|9xC{f(M`TB>~Z?LGstUev_sO9ni5J$H)Zxd(iY9!xJ775r!_a%)# z@);{q8L~%3-0!6#fql|`cLCIHWxVN zeoqUQw#({nTVtS38!(@RJn+Tm|#t3;b{^ zk0a79+BxeMVnh`v-h2{W@R)#C_;QX9ktPZv9zkz2plgN4VeFm+zITjo?y=o{{!u8d zNOh2hV#Qy0)8l-f6}y?;)p~ogwP4(GjN6@kkJH9c92+U4#JGSxrLCuEqXsum^1|9r z%pnG-yortT31Q>tXpGADS*I#LHQY^IBbM4B+QiCTTgi2nE=(kYOS3ek?oc7RaB{yw zP4=D11JSrw)+^qiK(&VX)Q#rKm>DxSQ3Nss>R5u&UD zb5I_GeTr4*DV48q&pCx3iyGWiXoznt4;*g2t&28`*2dQ65B5~VF4OO;^6eRXl}X#g z#VGu@jCq4h@ZU{#Lh_Z7kSn4;GcZ*(NQC4`<2C1V@y!F&&5##0B7iM|lV)=)(#U}I)Vkz~twtyD z;;!1TYYZ1xPnt}H^L_8_DakJVoRA^z`aT@{dUb_@ku4Y428!IW?nKcW|0 zWs75I`!6QpRlVF@cylZZ_r3dG7HWmdZCXzN-u-u-KOFi-Buvq*OnxF~32Pfi%Nweq zX|;t$+*N105^)XFH>E2LbSi&Ei12gxWkPVfYhqC6jSvWfqXU)w)qR@RnRCq4* z7VxPzBq8Ujni`gTT~v1lHx(hcY96s7nTSz ziOQpgR@LY!@PjEW;F%O_d2?OUSn;A(Kqjz|Z2B`4S;qf|~f>3F;kl*g1J ze(16Bj&SV-k1l^6gf$Bvt_HjvVBgDfzY8^tjn2`yLQwBJ%gs28TuG>0dDaE^{!=0U z=LX&5TMX>WuR&a{TD>H!_>#o=P5Qe{loEUiOAN7$uy$&yfz*rQ6Q=#E>ZdZd3Jae7 z0LiBm7%s5J!G=+oz`@7%PvpTN(610;-5oib#g5d_C8;Q$mO4Q>L?RAkvhnT2nBo0c zduYWIbV8xrG%Z9;PKwl$FdQMC?@0&jAf)JD|<77-A~l{^EUtc^cr6t85F|s*)v8 zQQ`F|B5@FY*=S;~^2ZjbL?)xjYPIe=Fe&-=kHPVDPBn|N(~dtlh(dXa>eq&@H5SWQ zy(_IC&%a8Oa{sM*>0>_1SLVX=pNlUnuio&d0I3_~w6b5P^ch@1jEAOz1T}Jl>C;d> zSc6bs^Esc1sM#@5iPA48>A9-iI?2)BhGz(-Dz}W|b&N~|z&FRRvpmXA^Z&vl0KUIL zJqaSdOi>Y7-hDMdt=6P~4jAa$-^Z)kcYJz?G7kqn-;Z3Nn;X1hY*t?syonDWidv!% zbz?!Ft`FDs*~nq&cZ9S8G$654@<1ZbmiU|mXe^tpsJZm_Ry&^WGTm#nm;xyYS?G!m z)`#3q{UnN+GpE^_H2#zg)VQHfg?v3s#k&Xr4dG0H*&#)Xpz<*M zMeqxfCi2cC4QOD2+zIKnmdX0jj1U#5xxS|BeXfmAiZB!GHFPqDI2flk5!SM5G<%*8 z0azaK|H#w^BFsy!atT+0R$0NZg0$&sCDa>0HaORPCNTtG{x_ zq#wVEdZHc}5Z_nP?$psASZpJrpDVIpx#MHhMCari{m&)xI?s8{a}Hs#42wyri*D$Y!qVZ`A7 zT3J0|xL4C_xLn=r`yV$qxr7kSB7Tg)Qvw@IIB^}8H(e^e?L70X#yS$A>bR^82nK`^ z1xgdTX9xo&O0)l3KrI=t(&?%tdz^u`WpQ#j+_0OY9IjhE>QG_O;^P`Qcu1O#B);7q zhmba8#-*k#<}FZ&j)ogONPd#Er95djRhuy>4Ofhev36d+PnIU-TBi%HJ)j$mbeaY# z-WQr{6o}}9$%erSelFe=sL&@P0y@4)eNdIVrk1cxgz=?Vwk<2+pQQ@5{mOK(jkHnj4`?9@ z^8B!)wzIW#9i!`=;;3_&Y@!Ips`n${9o;ZrTXd%02EG%}b2aqH1l7*Xw7<$h0+INu_lmz1MLZK&nR2$H?<-*3g+hRUvxH8P z!Uf)LL@-F?6Q!Xe__$@c-yZ=FC{lhhzdh`lC+7SDeuvl)0z%fcLQ=7+)PNS+$zT%x zin8?rRS}TX#<;ME@S|d67z~ftv+aCD`z)&U71icrztMA3gldQng({7wP?W%7WDy;l zsgn<|wsg!TxjWbW!b(Ius=;D5iL6ufZ-fdYiJhWFJRC~OZRWc*4ga3R@|FfsIZGq@ zlWN_Llz>Y-cihDDNkNRm_+9iH@P~!? zsX8(N83nt{KhAIUF*}f6+UP5Lp&qJUp%88`o!fvQAy%}1XQ}_aDd^1lt?F%4XM&J@ zZ&H(f`hBdtpaTTtK>~25A#;D;_15ROy=c`pMohlur$`M9Rc4^? zgQ?|V<&n4XG~(ntNb=voh*+^-te)9jNe%&WF$vz#nKjstiip%X)IfYIk2S$mOpV60#8=28Ats6&>%<{ znAoCNLQMR1L%X+SVWjQdGp|6l!j6>&tyFm~&<5D)$tLreLp#9+OFb5hZ60dc{0DD#cO*6RVY{MhuG9 zVh}#@ycx1l#U@{j>-B~1%+_$AiS59`ZXSQEgXiYCw2)aG8oZDk`y~e^g!%NNvzsUm zBIv~TqBLd%^26!M_y%-73u&{og@iL3{5W5H!o(~jk&j9+ZD;{8ZjU}bs%Mmyrz7bf z90-f4rZ>ruG4 zN~KVaP}bNX0Ht1|MN)7^XF3DVGNw~D->t%Iy!;a8*<*YaCS+qSWeYe4J_`6Ny~I5+ zXMDvyBn9l;3Ye1Jp-;~nh+8KXUi#j1JvV`Gz3X{Om1r*E(w#Sr;c3rtc#hulr@sXC zhm4-~MP+5>sN<;{VR~vEPSa!~B{a4CMlpu-N1>+lAJ<>PjOFk zD+aflhPspOH$LeHKL(<)cTRr1+dDwErL|(g+gGP8zoR9`XW1*RC)>M7#~jW*>_;q@ z;AfCukig$mOgEP*Il3*MdfD@eu-~*jt~N7)gi0aL==vC(kH)mnI%FP%ZIQKCd!VFt z)WDz*(*KZpu@-0syYXoQmpU0O2v)CV3+fA#{>~?C{0TiLD_kV}VkL0K{!k|*h5hjJ z5RCBu(l6i9v1WKJef)Dl&pR*b#sNh}>|T2VVc#(h?Pyo9%cZ#FU>I8QQi3_4hWsqN0g2vTQ750=LS^%YB7HE+ z=+c8q59q)2{I(`4gcdSRWU)G3-?-GV=!;(WYb`{qus@g^BiF16U z`KW?Grmq2-vCdxrox`7L)uv*l;o{DTL^wWx-BdV7=)G~X<8e@=SvIflF=xF?f`O#@ zxe^NDf7WT_=6;yNDMSDaKYo)Iu5-zQC0!OZiv|}TG8QS@kJ2*gU|%UZqjZT(=iq&} z6nb5_`|rFfnOz-UcLjqglx30MkX8Yl_I}LQtcb$;vR{Vmpt;<}g;%iWWLBZmV@%utpx1d^_Slr;uj~)?N zGRKiF;=b`_;N`}W{+P+~Zv8vGiY2$llx_n;19W5%u{OP3Qg?Wkw))tFV`n9_xKKR{ z=}j=@yyf&V3)_CSi_5hrY~xXteI8JKhmZ+wSN21Db6hCZBo(vX?2n&7hUAH&`zUiT zVWV*164O;Cw+U9BFI}jLP!;FLF+7dyxcIKjNkK;uCAhw0U^`~L>uw1x0RL~t|0n;` z7T8o({=^&q^VU|1Q%OC!QAiC*pa2327zjuJaX3We*k6M1g6e#xg6Mq?sxF&*OmW=7 z$LjmDNa%=gKfh>bf)ONtAVNYT^0&2BkIwb4w*|*`hf@Iwp*hJm-;kq1*{qW6ve#^` z<7}mFCFeDHI)&5#YO?>JCbFYF-G|x0NX! zPYsZgw8Q8f#lCA;Tx6Zb35>98#&qucp~a{=1>kCG6_v_8{y{{wxj#^$q%k5ZM2%(6 z3%m&(5(pxkxs_gs0LqO); z9QYoeImuxxUY)BVZZ}KYGOo8@n^f6(YrPC>ZIp$}>}JzSa`hNtnOTi%9SCL}v2ypB zTJ29t86jBi5c}sN*yjrt7dTEV@d27(k_)t!TN7$HOMF0?^i#5!7S>@yE?c9--FD#2 z&qpcNle4zZA>!yq)E1UpY@d^nR7||*h33y&yzCjJpMyC$yluWt1KU|XqrUtdheP~)D{EO! zD#{-wPxH^b@5}WuxX+)>nM^#!39723=Cb-o`w4xK7<>(}lSP(ib&@|yI_PoKEiruD zYa7+0u{b+YX?k;MEi2iK#$OZ#>WYr6?wSL1Hk6IYw9}~9O4=n&(19%(*!F92t73ug zC>*OtaKALU09*7TO9Otm3^cJFC@Kvb@e!9j|q3tUSwn#EU6a|GuVNf4pbp zxScP&8VxXXwRgsocq-u87UaQn5pkn^?S`0i&3GSXe3={t77nHVWo5Qh&-C6gmc6uM zs#+>$fd@yVq1Z6zgP)3^ecyDs4#58G2(6B}uCX+lQ?&A&3=`LV8*;>qo@mj`K=K>B zuNO1G-o^KX?bT?|7A|kGv678{_CBA!XLEA6VngHRdun3u_{7K--4H{Y^vw z^3PX84%QN)FFn`3^+?yC;$vwtmHF~m?VAO}|4KsMgN3~W<3s94(qS};^yFa#V}~9B z7*5CVICv&G*S6;2%llB4DRA&{9`vhR?xz3xV9 z^#%~c1_tAS?tqB7PqUugmbw^%Sb=$ztg7L>KXiVQ9Y^9rK3y;X{y{wG8N}yO6M(f* zNbnFbvVo=UVGJUIK8yMF4d&$$I)$5Mbv}+*TMm*ydl2?PFtOW2V98m*n`Qi=E@; zJ8oI&p_h(@oYk=>CvT9;W}d)Fjt@*-5JBQ6M+XtQPLG3T|F%R#@c)qQkz?c+hJ2Ia zJm?F%aDOR1Lyuy$Tmm%4Y5SCt@BF+~exw%r4qaqKs{c+4Pryf!b*UOI{t3J=WU9% z0Xm;)j`r-@J!8&I#*2jRDAj5xjkBZzNEbH#w4u5cYhg>0##>d4)5xTAI&QMgP%teZ zS3n#Ip#x^7Yi*p%)yb9EWViKErrR4irKQg;dO3nm{tf~C^$ z9FZm-Z8k+?J?`SV0QdUB>)UCtgyGQKW*831864=ZO}ES3^YlY{-65yqWx_gp)@Bzv z3ovlsV82g|r#=6BqQZ7OzhY&kBQ~(qm^#&V3GBr7*)HQhSpY`2ZLovCikl~+@Rx*; ze%bI{>hmq?tO6e5RR$@=S4gUy#MEBrp4D?SyLojNC*=1E#A3eFy4t2~u79zcTGp>` zc|Sp_=4i#d{$M|os9kTil4R^;#!oHEC@AT{Ap;{3>q$4mU}E(Wv?_>Aja&LjI45Oz zPg1NvakquIyxKwyQeP1Py!)sm4zE5l5j|~NnWP)E_CxZQuuKCM-YxRL8IMj_|&PAwLn`G zn^A~)XDXd`XWC28xgu_Y7)u{KYQkz1V=jr=?l3;*kz53uiz-~#eRafnNcMuxX6VW9 zq-&1j)L^lD+b0zjVzQkE^b*&qY!vY7X==2y&%ZA z3nCaOtf!M+XHA!j6~$S%>u{r~yG901ml3@Kcb6>Rvm5%&#oA7To9pkm5-uYH)3$8* z=WOpi`wJ*49!_wD*!fh&Sn%5ibEaQzI_OZOOcugjm)Q7iI{=UI2m>Jh1n7Ks>~$<< zdFqTwJge-;P}rwJRtJI`6XnVtK{Oazeu%Cvaz95@)+DurJ}MPMgtTj3?+uRN?W5)c==Jh+eC=uFbsry0 z7PRONV0cn)k9WPH@nh<6Yt%+Lo1={O3(-^9WAJLuKvkS92Fn$Gq2vDpr{l6Z0V+np6sx1Lp*roY;5ew_@#c^HK_H8E8^_Z3ik8!;H~?g{MgSoIR7 z6x$19%|kakp_l!(ZGu-=??WAEaQY6Q)|bjoTjAtwp@=HriGXAw&tz1+9^w z{ZC&x20LLf6`^_B2xU@E9JRviKl4y)vEDij!%Pa>eRdrTM}08%q-SGC7pc=}`u^Dz zQqK?PJgzG`rul;OLUzNpHEp;>ICCH}j4)Es^SrY8RmbF+bZGy!XGit9A26Frg$0Af z5=xj&LVmR6d2Q??Tb+d+d-M5rJ^U>{rZSP~=I?wTFl4aV;8^OCC#i<;0A-V?DO~Gt zcU*|^ec#9Tyrl?EDIqWA^<#joBlrErCgmQlwH^ymomkfnLtzVeEhvcR{dSPwQ?P5U zxa~>rwN6ZHNDMqf5C|jLsyAc~<`w@gvfc@e8Lg95KQE%<7%v!={ec7!zWHKi2%8=} zmdhTJ0lfPjzIO>ZHBS|WYi2m0wFiZF9eZ?g*yCvXyupEye;uQeXQ#g(YuNB=Zj~>L z(QN2xky$<0PN;t7^O~oXE>60U_JNX^65>Fg-u_eWFI+)2wCWx6r_P~hJJq^#%Nc9h`duh)32%Zg*2*Z_3&vVn3Y~%gLuU3L_ zOuWyA^{7b>uz7~yhHS`aSPInCx_gw!Se1@673}77)mN83jLyfHR(~^;3c~2L+9GS~*`Q)gR_{rynyaKkf7wb?)e~FRbGjA* zUW+Bkj984_n3>?I#`lhY+Zr`!&Dg{V=bI`TO7e&jA$h+N+j<`E6c#|!QZb+4f6Hwg z^-r*s#+q)!=5Iob#&tc*!fp4MQ07FX!u|0@8og$9dQGNu4S3p56VA%pq6EZ6H(ftn z?|%aYhov~Cq3}V{(~>x}A{WD;SnZmWEljqKZ`~=ePOw>l?jpfASMSdsx0Q^6e;<3Tm#<`!9zIi0Eh)(N*VU? z$N409pUk&sx{<3;0L}26;DS}D(}w}E(aV`@v2lLIMqF*mtzHN5vp0uu`cEfDhoyyxi6PJ9qP%{v zx94}>l$4+`nd%yMy}hf(i25`Q!bi5>r`G%&t#i)cPM}iWc|9wueLM;`h-GcJ`P#43 zs?Fy%6%Uu0qGSLY`rRuX%vq8kqs~1GFr;V0K^cdBEy_z{Waj?;37)?u%QAWQp;yy0{s_=SHcEfx{x?ElK~i5! z3Uq^`?wd!aIpoN^l|rnap#^pU3!++mX*ZYr@&QF0SXsPjG$t~?>mU*}6ID9fXv^iJ zCS~XvzlWpMR+WSW>S#N@Kk}Eu6hAgC&F>D7q;Djw0=w@@UH18${nqL~_r_VieVdlE^{|s!DJf&xW+7q!ww=jyIJ~U;Ix1xvxo;#5^*AAn zuoY3O#M`mlPoP7$tf@4*)R2Xu)rBD;y&ggc#1E)^!`q(OQa5)5((zm@GR2*YARh{} zq@0&DoIqW~f>o*`9~CvlA4gAVy{K53<8+O1-~R!J1!JNaf6)E;EI4IM=NpxV!%uV| zSlSBt3_<|%Cg7LT@cqWZcoE;gV}5riWK!60s^ejDc&IvYZQ;esNb|l_ zKPvJ2O%x_#8)n;iSvxznwldO%C|*W)cv#axaEb&Pgl9jNE4`K4cZYP&@j*hjpbc@9 z4or3^c&LG*V4}!8Mm7j!`fy__N#k~tgX{eIkroSSu>)g58 z6PNgpHYZ?2oX_F1aoOiY7~dc&c1NF$zVkuxSD(YTrhvIH(k&WBg`fbhx`@e8I1=4Y zpn^3G>9nCX?C0#Cl(FU7Oq<}}A2Y)6?q$fb9R8hG!^{qbOWElCiParHIAX&j-6bU3aQmV`W+9FteQx1L8|l`~+35u2#7vv`T6ve-^Bt_9dY)qr-^Zfl z-AM7lv*R++C_sNoIkhj)$AC@&swD!Jz0S;XKRsbqhBfkeE!_ezt^0m^>w+!aL~Q-a zScQP?v!1N=*r*B7Qr?y>)=)C#^eQJeC0;JPqu)s-$@f@K-psUZTjWi@gq6zcOnjuq zjlIZINTg~MeW;SLJ|igEhlPr%a(X?*+sKt8%5Gh0@w(ECuZGqIE4_pf`^MGK^P-La z@#P~fib0Y#o_B$=#X-htZziflwjHO;U)y=LC%0{XUmt|pivdBgoftyJQ(I-pea4n= zJDti2hK+1;(yO0qp8AWcKZGq}bY#uw6@VrdVIx;|1vszo&R&-6O6-NCk%^b{dYjIe zk2;AXE{9IZvgxD!LvZ_D*U1)>s&Su`G+D zmL#st(tVPRY#AtSIzOaJJ0gx&$*iHO4Vkw-}-PP%|Jk7jy``r$3;sU7teSv?6j%El3^>Z zI8O=$Mk$Gr(Y{2(=wek#)9bY730Pijd91#Os~eXRJh3UGXf&-b3>z5#AV^@?;xyAx z!iul`!-Q%cjB#Am{jnRul`tt`xr?BLq%p3`M&|pK^hG96FLGV4Fd=#O$~?sC+GJg6 zrBmY~Rhv(xJ<@r~I!wf(X!+2DOcqr+=E_vzFj*WmD0Dm{s6uPzkAs~u0ini0gVRPZ ztXPm-tP$>y(cVb??9`rB7V47O6w^S;@4b^Yr3@>c;BarP7Fi0VbRDm)!j-izwaGdO z&!fpjEjfq?V>gO$IsL@}A~wa|0l|WSt)Izmp4k^;q_L9*+x>C8?ETQOKU~s$D>ai{ zn$zh9YP{2O-}duIu1ht)w~4WfWgZc)@B9<}W~-A7N9B8fK)bh{qr-E3C5j&>=$8Fd zxdn&w9P2%sv)4;iQRWZ7L2PjJ@pD_vHXhynkp8%J{?{eP;T29Z>-eX1ldj$HG8%77 zm)uHB?AjSCs?@YpNhl74;^#Yi^YTY_+n4)|OSB%&Me*(i)&v}7d!*wti}JsmCBs|E z!%T#lN~xEm2m{(1GpSfF5-&lfDDCU(5vWq_=iPveV>3KU;V$J^lY6aruifVRuGe*P z${1~#elwFQ<$Cn$RkgqRpLd8am9}a217AKzchlWIV+S|kexnT?Pd$Xi4s7|$Wa?y; zQOUv0G?IM2&wbbYEl_I^o$s?eThEoA_?rv0opoec)7~fA$C+HKTk3@3RSFk}F^au` zhN^LN4sn>;X%id)+tjw?TTJxGD?Ztw_17t;tKdY7r)zBykP4 ze;^x7l=ct&u2&o1sJ0M5Z$=DiG~R_xo++Vp2R}+eig*gP-LxFJkC%lDC|=-bwC%~O zBjX<$UO(zNqAaG+a15iP;NYJId-YA=YRu%!Kf&ZyMU$$RZOA_Ep4d%L$CKH~6m`c!8eyHs-)+AfFn+pC)ul8X$}` zC>VxjxsL`64hIrHI|tKh4OeR(YS81wMy&qU#$GspD!cP-NJS?e1VGdEU|=sDfJk#|5bBdb6^% zcuae4?E}jpMO;@hN;nBUpoSPyWKKDO&n@(p(AvY6dp92kjJ?ef9O5qJ4_fO|zBOlf6#NquZa;HaY zy}JiN_!)S+EF%pJX&NmBtQp z@!v?8TnjN((x)@MVvxx@f2pA(n&H=W-WBiq9Jf~a(@eH_{k2E?|0sJ4ptzQGVHB4@ zkl^kR+}$N11b26r;O->226qnxcbCE4A;I0N{k`qMp-m%J*2TC;@Ftd*x_vme7su zc0P!Vbo{9uMeli@OtB*?Xg@d`omywD39{8kac~oI4XK2a1d^8jwO6*Qgnw@V*Xx7rucI*6J-OhHhV!X?cIeULb(p#%P`va zI>aI$gSA1}1gor{%9u^5)e8dm&ysQ;pVrKxCsn9%nVNo->?x`UR_nvA$mYK`au8=j z|NawCVZPEBlTs?*-&rW6#+3fRoK#Aicsz-msBv9-#``9*|Dl)$=H)5TI@RfNH(5Nw z^gYNFlhMOs&LU9m2S#0P4!grQV(d`%nyJZ#! zJ?ra|uZGJa%ZYE}lhe3A>lvo*X!3eeqq{Sx>Ff^s;qqP6)^j{AJ%d!3^ecHEif0Y~Q7{k4VN(Uf#)iPc{QQSt2fi%a^>J4B zUqihghch;SiArwn3unXF>c_EmqZKkQEe!8kw^J;W^pgFV5!`p&278YdmajS1TSpr@ zS@k(ZEseI9gNVG7q~9}K0LCOLCxppbbbVujmFtLZgzQEB}lYQoGY%rp8ExbMT=Z0gax2_1Y}C;D-(k9gsP3 zoFVMBo1((gGA)!Qlr2LHS{=IUCn5yxm1& z`gy)WIM4g3%x&j%_h=~az>Yz&Dmf-hh|T_GNruMFP6S4BDZ~q<75gE2q5M2%>=ej) zi=Q+wdo1LMFoBCh#OCj}3?y2E*A7}v+-I6kox5IhGBwinO-PWk<`x=W0Mgr!qbpb_ zW)Y|z{*j0;EN)5Fi)CQe{;S=7e;f>->!}_{M4F65(f*sBm;x`K_$?IaFpy563MIDc zi?-Ub!P4Uhm-gkc>XZh#d?Fl|J@i>w=GxX=;>;e#v@A2bnjqHFv#7??JhSP9iU+%2O z$_VU?bXTnSezk~m+i;~-6cS$ztm9{FR5cpZ&q!k>Vw25dx1s_$8mUIOF_$yXV-zz! zSAEs7c8#Hd@0J?3qDmQ_ov4W}kLWgmA~);mG0=RLAY0WS;>utxNpkn>aEr67`LZ03 z9E+@_M49W)xJ<#C0(R}TM(_U8u=>SXBVJw2S`xFKeG3?i?a~kZ@C9}=tyi7gx~;Xf zId;M{BqFey=ZfN!#JMKOL7%fMr&*aRd*~? zx1Aj6J#0}OHf7Xnk7@>9-+eokHT_vTN-xmTSX$a-XY8`wwo_7+R6Uq6sp!dFUl#*k z6!r;@yIm+b%ZXDyM(X@PR_8okm*0EAaA=_j=2LOOLWa4EV~fKt^qlY830r0|b-(e&k@)1TEuyL#;uT>&SWQ%{MSL^3-`LcFRg<67E+W{R{M zm%Bww^9*fHz~FHy?MRzf4jQmjW!rYIE&`N@7f438X%y~WDU~xtr#rom7LKZml9!~p z8|Kx$=&)I8qEoHp*XFHfRcG-~zwda*wFZVY$u5Pc({GBZ0vSh~i+TPbw`J5#C92~^nB7(%d%{Y5JdWWx`R`I4KOt6>3y2h9Lir))5obw9v#0ejCOlV`90!e>(!6Kdco7%oO$?k40ad# zCGVRY?ix!CjeHjESn~qv8WLZ9E3&DhRSkU^+?mTArvNDdRZT0wt6MwDx40`%!jAW+ zMsnGf;>wDq41(XjdTp4;eqO{LB{9h*lSjVhJZVFumF0?r4@WQ?9bCs~`@ zqs5)mF+&nYoMH8;P$tcEr5sz0BM3L3U(mJ$?TLF9JAXrQ&UKx!W-;%y)bODmF*Q@*fpspeceh!wNE);2GL4fHHg2d#;6E;jL{T9262 zVbUh@*HsBY4l(sM^JI=r*L)Ii89bE*5!TLN4qBAI{cgs0_i;Y17Fp9Zwq4*Jug*+% z+6#uugtVQlMPgiit^t%|JQuhLa*|Zs%i~%{VO0T9*B^Cv6xO#H6ttGe(S=Lr-IVcx z6ROfNaIRJNUk?-X7BA#&JNURv_dQ==As{$?!nhi9%){cVDF8jF2RfQk>w`dzIaPIf z5ewdt+zxI>O-U^epH&~9D8Ab=aq;9eC4+07jhXqJ8;eiP3a^>SYeG3u1X ztRiEa_ViC0J67;!BtPiweebBgy-6P4wc;P(FYuZ@+CSRQ%UPG@UAAMmfF&C<$dwt+ zgISF%Q6y@@q}@~vJ2jg7GuvbX)nyEtGqRnuG`|ZYwz*g00-8h&_TB(LJon^FI|Bz9 z8V5!^r`PThP>5iz zsi_�b?BweUtMZdGomOHher|Itng#3w!v;gV_+va-NW}+h>P&McBm@3nN=8?sYvK zdK8CeFOI1z4KeFHc8e0Gc96EK307IKaQ2zq0J)<;g9vKdQQd-d+mkaRc5}tIQ+&nc zPSe=ULxZy%_xI7hHWq;Yx55Yh7AI6zhF?ZBp-}PrxjjrLy(y{95z4j_MTSEf*gC^^ z!Ru~Unq@^2DSHNKM7ZmVL}?iv9rAVY?sEB-_)Z4QM_cHJLsLOXdk*!!o}u=9UZV)` zx99r$ptMqLa-Z5Mvn-G>UI1(%*-F+`!nNMT(L!)17AQ||+ zeAmkyKu_OTT6=dbRaVNNjUQw5a`K?-vJN;jcvX;()cCpTdjgor5tJy5M#oxiY^c61 zr0{Sk?JN{e-3=PdWLZq%$h{V)7~p&!3-EcFR}6Hv##TBUHfpBE^1X?hUt>!%y^78B#_MII=9DQNGx!$yC!D@Y5OVx{K;p+lVO~X~V z-4~G8Qsc{AIgu}<+nWDAr;82lT%)t?3=CO_!*k+IiJLa!3< ziktB_UE{J`f##<`y_eCCBc|z|J?4F{$E>YJ>t{@OSsSfHc{mFz?*pSo zQ@&aeP1R=Sc`dCPz>adCW)(|1As?u*e{;}w&~Fw`*&G`PYq1z<_utGOh|DIi40JBF zk#a#Zj6&p~C?_zLu}^{G=wqW0eD3ZkPv&zpvmVMaP;F}eG5_`xH`{D>z%_!2%tPOe z78D9go^ye(=*5CKaIY~-2GPQ3d9_RuY?@@%c!ADkMAMblMw6IT)#tW!1JNVdcPg(~ zOB&J#*Iykz1{_FN@+h|)rWM)<9Ka@0hd`OZ_D#*W!X~UNvH0gR{l(iC5Qy@0mNRe;B5_T0b2fdx@*Taw&MRsrOLj zl`PI%&RyH*4uq(?a}P!$-Jf%_bAzrb4C%vu+VPu~P0ms{)29%E$;&{X^$&;tk%;!M%R?e(Gz^K&y1w06qaGfyi1iLVY z(5E?|txHJ=eOGlV9)h;=ofY%UXpp@wvwnYD+r_NB_Ce4D7cb`|@mcTHVx(S7%U0al z4iUe*n7)JJ4v$G|S*g^%Rpid3F{xRrGsoN+lH{sB4~>tT0K$+T@u$$n z6wk6eGolWDlMm1u<;=*B)Kn49+pC)|UXSNtGD%D=4W;>p4$z6=o=DM|2VF@`!f#yYGkIk_N>pzH8%Bt_|XVQ$savciU`N+}HNHleecAk(YG z7GK=e9cklPVD6jPT=>&79O{1*-HytieH1Lr^Om!=bf|n9MvJ0wS>(W6&}`z29H~dL z)a3tWxwN{NKW}B}mEFp)GOcE|F$wia1C{9DwQ_xOjzoW@$3}Sm80aAa&9W)Q{)jz} zdqF+@!0OnV-&|}YO7J41x>rekCSN*+0Ez2~K#7Z?TrKBbXeJEAf(~aS@1DXkR@X>g5M<^zJX|(?r_U* z2GSZW1N~!6b)QtGKsw{ho0C!X+>30u^CO*JUjwwEz|73YO)4&OvOwJGy*bp=lBJNo zT+|$N7SCAlhC_-Nf7kb`V(r^QC6S#6!9<%vFozjHdk;d{H{sNX&2LF~NDbf5^-y}F zSTpM}c8#{YuSCRl)rUhFSXJ81mh#YGgr}h-^FpQX|BQ{9R?(aNpf7EYso+*QHZVcbpxzV(~JLyB`;gu4;v}}KwdbP>BrA|#a4!Yfr@Bm|}Ps54* z)Qb6X2oPeV1F_WlWAhiG`s?b!jyv5R#TTA92`-CveU0FjE0VrlK{8)HjP=EMF%qhI zYN$fg2rY9Equk`ckUVBBqHXY?cu-Syb>NiZ(GPqsxwB^_yYT%&aZQ&6L~N-9&fsfw z!o5fcJyt7FC|P1om?zqRw5o?#Fw?Z=i|Wa)me~&9XH>HHwv_kEVPK3;wspT6T+Z4m zpo#i!Y7IVG8IywF^y+Mz=t(hwW72W2s2c{#dQ;mHUR+sz_ULLKIXx7(lkNaBTyM;G z)3@!kdi6-`PQzF|{a5l9F$80*&>qf-vHM){rep$LM@Dj_4ig3^%CqfOJ@iDc4OYXz z=>z4|JWbCW$eFn<7phLBwI>N=V{%uy4|ZPV~!KU!VW!T{Schz6%uP-HbCisWi?+@TH07> zw_eZ@t1%o{a#u}AhKa^A&8ed>7z+YLtg@I7p<&%_0Ek_Fa!et?Lj;kmXo#-aBI?#cxm|^*dA}2zZ z_(|4eOf!}dv$uvVTrkI6A$=>@A(9wPI#)^<-h-|XOM!oj^91zmP%}F&G@vLz0SAeJ zMJw_x;sr@8Lck@YYP9UOQH+bvS|Wm?sB6%wLt@%f;06s4mXaP*Z!qXTC85Gs53&v= zvR@l3>NUvXi;W2#5Dz$$iY_4b3C_pPZA{@mhKORy;go*3+z@ekn_W0M%M{8N>&Q~V z;#rO*C`-4eRc9|P+LcB2zD9~LR%Ex9qgT)I0^=15Om?TfVL>F(lTe}Ky`1{vIO6FZ zy+3JQ7@~NdZ(teJh=~4L0~00dr^2^uO;+cZF$Be&H8Pn@he>1>Z|MhY1Gv54a=S4T zC*HH7$B}ieq#PS9cShjVe-b>~4N=sjQ&fm1yQhC;Fw8*ARyu=OEXgJ096q}_)-cRd>2l@f`+#0IK#gnJA2(*T{z zlrvz*MpX5Dwi-{f-kSS7xUx^CV7uQyG~zA24WAs%lzA-T;;EGSEMCF=w?z)9WMs#S zCGuBC%GjfPkS*9KaJEkF_HKPK?0621ox$6IA-RG0+(uLkk& z9*^@n4F|qNLW#vOg{+gv+ElQ$(8=RrR=!IXS6BL|fl1?BBx7P&c)cFN4&~^3eofrx zwN~Fr`Q5oZpLc%*zhLnF(#I%0$SZi; zfgH00B&_($2r+*M^Hb(^ja-~1AJ&=19!}+qdPSOc!A&KL!Fok*y(hEI0I}8R(c(d= zT+R@XG=1Szy;QQ~Al5hGhoP%z0)E~df|jhEyrHbahrEF%k+fHZfr>0K48#v?e>`+)LL{ z7ndfj)!B{_>Fh97$xnZ9bI9*KwlkFuIU?Ncx&$0=S%%pYEnrZcKwY1j4$1yZdfkRD z;F?H_%ouPB&2&t$N=04Wa6#4+Pp-4L7#Ieb`D7AYp}Txny5^O2zDXFFhyalh&>Vpk z3uKsLQVrX-$A_-KmBngNJm#P>Vl_7@hr=D}LU_2oIa-n0l7C-b6FUz-pgc^x%CB9~@AYoU^_at3u!Vc78#7b<> z6~b>jS&g%-;zsF2%}2->qB*RbEru{gq6|o217kE|h?BBQNQ_zciuSC$Be`CQ)icG? zB&!AcNTHGu;X{wS5&RJuu%9YrJmAKF+=*Qkte(gw)2ilx;YCbd20u?TSAVr(vV`shke(?| z(@~Y#IwJcK`KON*TT+6_AIl!w>gfh+ls2ngya&u}GUj*3_-ml-DTi&397%>$HNx&* zT^J3NFg4^b$@gF0yEl`9*upW!u2J> zv=bTIR%V)&^H`!GfTJoxdsOAmmPouazx!AnpZuA)jwLD8c8|-Qq$|5SMX(q_d`9M) z&^;qn`}OW_)qDV2w?@MnVkDQxBF*h;Rr_5J5=&m=H1yaeI09|O(?L(|ItgR|@Uf-r(Rr_&{*nYuaZ zADT0^FVywp9PeYrtc{|ZuXGzcW7N&^ky&q9fm08|$+@Nr;$Y&rKOvE!yIbTL^Z1ZJ zlQBq_NT8=4Pw@FHs_!qYK2LgTTr$#=k^fCBdklcOpH{Fc^KfRvm;|Fd{a*?2Ts>_ z%jaku&*Z}Kid>X8LgkyYhZaIKva>^?jm&F0l9z{P6Q3AaKZUk~lUAh0?t*Q-tp}7z z$TXo9UFR0q+wsm(0!U~Y-5Z+BlhBjvdlN>yPlQEKU-7t_@+Qy9y(TBIqUes+iNeGc z=bjXtFTa2m3<+@Mqi_|^%}HPRd?@8jB>wJ}=<|*)WT$nipf9SYnNd>y%_|WopofK9 z2EQBjRv`5qG33RJM~K23UUwy?8lAe&vqmqg$-+5fx+0Ae%$+`ZE~=~T{r zTNi)0@H4{T;m%&nt3_eF9P?C{JzV0(%Y1zUlk#5nond(6E4;YzSLSPkDg5s{0y(sG zG_1NzDyWZ`c-PHqMCoU%p~fau-3-OreEepT__e9QyFPX~k2M8VJmuJ36`Xe7ki&o3 zEJcj#Pc4h`q{rwtUPNpsBw(f6>J4Nwv6CBO)Me^&cS@65cGVkFh2#n^0t0BD@2n_YOze+S@#Ft?mj4y`_Ju=a zgNX5fokvym0+h6CJaD{G%h)RxW#%(vq0|$AtWAVOxMU%Z+{{ z$5=Q>^($pA2pc?qpmfxkZF2!Hlt8(MPJ{7{jhq|m=Q{+++h?=6YZ4V$4Dr9ysDPWv z5`1IpHDkHG;ST>Gmg32OCN#rkA{=Qp44k|lGQIinKShv#NBM=wvL;#~kAjOq!1Ep= zz;rbT63i_vSLBm7abp=ab?|uQaR(P-B?kp|i#aXbZFm29!+rv!tD!b`^Q^~LSKW~E zDMCu!j~ph+2vBBcIkLGKq#5j)3H}qleZ! zW7FVZCE1c^fSLXYzhCcu0%YyJ8mc-QT7v?a4n*i#s_$Dl9%k{Vr-W|b+ozIZHz`SKi8EWRowYi=7?l4W^B1i<$i1v@i%)SP%(NXop z(G6P-%|pIkl8$+*wVVn6c6-cNcTV~90z|8>J>*AtE#-C&JV3Pohf%Of^XVkhp2AM9wZdAbT$9M z+h2=zaFW$GOPt(7%EcEZp&0i<|uP@tK)N!NHpd}JTKY&({m>~)$Jy;8~b}<#h zuZ@Rou;&jbq?x-_Pp0t=#HD-wL(u;N-+urB#m`$<=#^L-S9p;>aF7n-@dt?e7Qa^h zDX!FMOUd$d*xZ&u#eakSw>$nt18u0|yUWkcV#-8+U=K>~+4ZIh;<^or6a3Yqr3T}M zCo8|ouMCGj!Serpzb5lr{xWr)YuWhb7a~5(5RcBL$tXZS(9$!HqhnQM`cpz{|meRebJ{?a38!~)of4{#MYHQsQc)L z`_MHK8&^^J-`LUaMeHW=}88YIYf^+QoiCR4x*$mluz z_}vABd6>{T&D94|#k5+_tl1jPeI3n=p`W0W(7noKKRTA-9YDGKKe7x70-+P zpT%=P@(Jh~7(yCy#;68!gLa|(-(Yah50LZIGfMI2e_hB^f-h}}67msipl(TB*~a$8 zofPGlh~l<`>Cy#Y=BR$6Qf^?V^}h9zW}}0Z`;img&5ET>V;*mZF9lCjbhqaQ-pXu~ z6K#)XwaEyO7NaUJ@3&Q?T>O^A!{u-mH4N2wNsWN<%iq%UpLfVK4tMBj>5=tQ>*_Zi z^$RblCWnz~uHudaU2D(YeEM-VLw_cUne6q1AFDs+$vqw!V;g6?^lHl_tk_8n<;F8LKFPJ(6wr6ZdPxm z!!=&D;=p^Scul(4xu46|Efa_7IO>et6{U3fE=oo9C=WyasP?dL7ojxhps$+I=7 zru6u`7r@UVWq;Grv^_qpq-U@9|>nMz{uBZ0l$G1JtMs zp!Wa+1%ZC6BIGch$}WR()Hd!@sf%lo1iT;smeJ;>rc@TRFgHN|^uxL8IZZ`H2cSNz zD$JjNfZ$fW!IlKvWwa?fdu=_8F@w$g4Lo&JxzA-TQK@EqU6f=^LA=hNkv7$H(NAl-Fdb`pD|G6?7tm-4DQ46ktwXw&wt6NT^lH^ zR}r-$59Hye1Dy;;G920^^z~=0AHIp@%mCF1f@0|uF`M5=?E&^vzi2P%;80y$lz|C} zOq}n1)4W2FjW9sofvA2nK$WT0YB9 zpqCBX#fR%RvYWpu8a9=YSxcOO5nN>Akw$2(b+xrWM;5=7OV|S$hQy$%0i7RHK)M1F zk-*}`Qr)M8F`s7#=j_|_LhWfF1A{NN0vM7-hl_`|6mMC%TZM>avU{{S5Q{S*nS|}k zjEfyHP480OE103)q677-BYaGC2vMjD5EzRDG7LQ{$xS%RY9#-QfWF58kNEFU5`@wU zem8T75zKC+ir9?1Px_YUEkbi}%T9&|HeDF3few&_%HqcwO&8v5{Kzn0YSSb)h>MKO zcM4|cqixg*7(tPI=O3&-JYP=9vXZe`tCndEaLrHTivPkf;|si%TbOw!B;AXYnc)=n zFzgvEiFepRQk+lsGi}RZV#D`W2LrfxH98<6KCk=BhOpLVhW<(PUXoRZseI|p0GvV~ zne$DI#~YRuR?|`DKK$xp*q(p##lO}~uxeYhMvd+cEm&oE8tXk|Z1m1M^XLMJPRgDvSMsw>u?)va34!J-gnRjKzw!4!&v7XTfxEQu5?N?L z{yWx<%fdAj45Sz@H92{fVrow*%U&vhQV&>mR6*w$SeDFWFaxpjbx^aLXf0*1)gT~0 zG{i`l85+d{_BC!d)R5<49bXK2czzF9&rHxhkF?c?PO))dHe5F(nIknk_&T%g>m+Qe z*~<@|n|Re^8S;%`z`VFh1Ly!McoP9*wgerQ^DOD#x!xaQZMXcFD4QWGcK??s!@q_?Va&}46&U<{ibDp^6ocWE(3le9a)&k$ zo19SCXqowICVF^Yetw|r0?^^WU@@z(5dJQKDeOX{LeCGOG8i%$)%*P*Ow)gR10 zaED!S(2y~RVZc~=_iFh=0PROtV8)yv(Ahg~CD!%$_!S25dc#Hb0KAvJM0y zy!t_*Tr+un?Hs3st_!ngAcWZbfknU5zaeB%lDrwaNb(!w#P_h=vh1eAkBIw>*VdvM zy|1f762h|%Sm-&Rg8NcU_YZcnsfh5-!(2V5q&XWo-&Cyzxz_8)o}+zvNwSy$n`^|m z*3rn%HqkmsVYdwK0M*uB_2Sw1Ly~zVQ_qvRT;*c_zQsR*-~;7+BewKnQH=X9CcBZ zn1O_bGf7Dm)>HgxgWZN&Gpy{EcZIleL}NvmtFHJM6jQ_9b}B!&#CSH)uc%*XIDF(C zq)7{7p^m~qk8C53H{Z&yQ5*mX{yS-kpd0B-O+!d((Z)&Eutlkdq(i5nt zDC^&peu)Rc5JoyqYTIPC&s;J@p-`h)6dotJna1s4a3vwKnJ=CCRU4;Yl8cfYzBj|@ z`SI>J0UH_RMAUrl60BM6ExdqN2~(<(e@7?=ZRe)=c&d^2T8G z1$*@EP3r_DZcR;%77({b#Led;U#$D2S(5)dXZaJ;$4}saOTGzG!vO5r-!9n(>Z_o- z9H+X+l7a6CHl=U>;Agndiu;9?=Y+jTc?nc4X0lUYLQ6lp#xxx&8d1kPot_t9{?`25 zTnG`CI=o(;>)HCs)3FPov!Kz$(o)0%P!i#asE>H7XKR~gZE1PoI^F#%$==g7iS({} zLWj^fnw;`RS~U)yjJ$bW(iPwZ$o!XI@g8uIgF&v5wOvyG%j*R2PDCh(lwmhweB+<; zze~R?T~whT-vq)VjU#$-4g#+S7*@g{KKds8(h`92o3BmYR_VUn&zJ(0`QDGWK+mj4 z0Ko4=ugG0(Js`d{J)RHwoPHsyKzMgO1SlHA$*jPiHjtYC?%>@N0Z;{Sx06Y9@yW_c ztf2?e!{@p>GXkI69^le8ZbO>AkpvH$v9bp5r#TmGyK2m*P-2LD9_<9U!<`SF)jejz zjx5X3*BZi@5{x>^;%QY$wE-3GsgJ+~AG+%vOZ|Fn( z;ygi!IP37a7Z;nd;D zP64K`#w+Xtev;p*1^9>!24%%lB2Y$IZ56%oO(OQ5OV`+C9DaA4=u$Kb;-9JuAnf_3 z8c#z$?)%vj#M!cC9Rs+oSz{&-&@(ABnWdL%BRGduRU|ww56|{k<`|rwzy7v_38|TL zPir()3Nr#L4QuF4L2hIUlRg9oDDiWtZkzX7XTWfiC!nXSZ$e-6U`wj{@S}{h@Rbv$6EeZlQag9%ieBk%GAGu!0%ZKu(t|A5)jgu5B;=fq9yMbT$BmF zs9(y?y131spO!J(*&udrZZT)r-W=#$GuG{$wO$`DS6~0jqWlFYp*E5%o15!Y(ZbMw z!tMW9Bhvlye@~yOeEQ!O#eduk|6JU^{^hILmMbFtCsq6(Ysj!)3W-yTvgYpx|6h3c z7iRi-iwfD>v~kB31^-(L{r7XUx04b#y66$u{8ag!lm0gj^y?^DD}bwR!3HUh=I=%; z|Ht-!eZ^D(Sd0W}ea%spnsk^hnH!S0hXRfN3jpmpsHdi1mDD~Ih{v?%P#OMlUcWCQ zzPhyD4M-oS2bG|Z6d@ShqHbbfg3NoAtpH!r7a3w5o%`E*P5TpJ!ON#QumF*4)g`m7 z%IrSv=9C`oziagV<2`5xDxfdmq83Zs5cNY+AD9vT@K^$u3G`GXPL~0?XlQyjs`0Hd zKkzehqx*Ra>CFK8G4EFG!p{=V@;}s-3S_8;d9irTXXleInO+}A8(oO9s0Sp`PC1T7 zc%E(TZ3fpv_}@Tcw#j;#f3NyC!|+QKZqF3xRQzDB{LJ>uf=^rc7nm+l!M)r2L!1tc z4syfeBjA7*e?J(#lJMVRC_>${`knU;3rw}q+_TF6JX%FP)s#K|PW`Y%<>x7I!t5=% zPcNWB*wRU3`a6>NfeRB0nD>GGyoQYhm-kUt_yfYLr{wJgA@+7NoL3J{u7pzY_r7Ss zts`k<`-uXp4e@>eA>jDkP7MbHGkL5B6b$bRL9;jA2t~*1!r9*JwUsXqvg(JtPWO#E z!=U~Bd4Puq5`ynboM7NBBm%S7*V@02@8?A+6ou<$30m#OpnLK@VwI)9hXc;X|EIwn zL<#O)PK|wVu3CGaRRH_Rv0SwQ*`k0B=@od}M)--W8xG(D#0BH1Y%J!-kL&s-^&7S6 z38AjybZ0hgK!yn>?YbEWu|q2Vfy+p|5RQe+ri6kzbpAgxoo5MzF&Y@*ZBxFc-{(EV zG@b+cr7eW_6ZLoDa6$V7al))q;aCrs(rfUd>_tK?h)QY))%~_AC|9azg&uAHpM*%q z`U=OtAv=>0-0el3Du+M}@LXSfFXcb@Dv~Zle0=>D+0!5NY`+<1NdO1bF8he9RT zbl*%|fa-Q(MW9jqz(4pFervd`0K9PLT}ypXzW^an_Py%B1L7G(;*v=x(2JFwXfQtQ z^~{EBIDKQR82KAA0+|tC!H~&*q|^!qc8`DuizxqTrf`MBjpuEX`9CHONglU;3X-Q+ zQbOHjh1gF#X8kev0pRFqVCAU|_<>OGeWhR4D7CDeeLHcNJbc@c9P5`;%m<EP+{=q zGH8u&)jnh-rCIhwL4O1Sjxtlf+`Mg%t^T{%n^SE_q*xkY&cffGgI|mBdZ%YDRVxPBL>r{vbvl02DHT=d@&EW5ao+cp=eT$n&ZQ)Q)buC_r5kE`a z%t>mb4Bw$xym~5myXd7rrjuYQopeo*sMCC}-lI(L`$7FglZDdWOl7JdZRHeM{q-m? z@TMPH$Hqs7RF_RLxPfqofkNe@JG0c%Su0Vmh|lD&U0j zSe->O8#^0;#;pm`Wdh4Jn`3lSMz3Cs#fl9Ti5X3P)c!!#sKhLdc%W32Dkosfs{l_F z`xd256+d(@QmK-Z`bU1~?E&eh0nx!0bRbVF^ML_VBTVns^mClOl%=J~--YX_CKf(HUC51) z#!e~){Bn#)euc|Z7YSA=ut6kv2=Cf6-6Ig=*D?aHF)ePSw-ApyK}4>_jt%l~>o}E< z`Dnwrg=f_^(T{la^!8(zG5#~}8gy3~NE`IELAD2a>KkbbaXJr!N9ch<$fV6})f9_6 z&})1nK%=tbb}L;jM@%kT1Xy+>@u4-r=A#XyNo!boNk*~*;&FRk3+NpEo?NSu5KW0{ z<28ZFm#f-=_Ql|;ME^CJ1b==>GwJ0tk?D8G!e`4HMSW5)|3!)ZQ`KtMmu?SOuA=cL ztQdEbv`M-t4G^e_f#-KXY|?}GvcrTUv_cGN-rwV^DXW`QL%Kg}p{o4BZ`wS$+rhyH z3}063E9;rn_`~u)!~<03hhFJIfHlC~s%gRmo~Mo==)$uee+x}Eh@qa3av${`LxZtQ z`HJcNsQt7=Hq-G|xuB+opA)D1%sHtQ$Hap^gEwT-3*t=v4#E6pX3$35{n+q2apubg zRe0Fe2#Dz`2YSarkKa1e9e>)=8HSq!<6`Aq#Qf0Pk|_t+4Ar>@@J+R>T3eJ)pUN>) zgu<-wAkcjUEx8f9=x0;EBspn@tP9t3=#_SeUpBprFH0S4J#~D3&BjSF+j|9L=vg%s z7!E6F<*xZU8FV4M_?c$`XT{ z*wysB-+i->#2lQBbP`!~5)0k7LAcC9`(wiTOv-1esWiga{C*TrsI(DU&SZ? z{WUHKm-K!}0WOwxs>_7HEIbbG$xOgx<~k6GMbxd;A$c-O^);N1Ro`ZEBH2iqJMMmD**R@)UnMZUdhZ4!rT` zvoCu7sMk=9h!bPla3w{;mxxpVT|yF&I{bM__B>P{rt366yA~X?d4g>uiI}-Vxq&vq zB?=R6l4Y5RE`2ScJe%>O){~ulN0Gw&mZR^Ii*|BZ2 zW81dVv2EM#*jC3jI<{>a=j(spefK%1?yXgoN>!>-seJD{*IIKvV+_@@-{f%jzt~#i zTc{pRBw>x6q{nNf*05~)*E}~PPqQEA&`(1e|MEXnYvyH%u4ew=>P&fi5Vp^CY1W*n zbGyyp;C`T;kJpuu7AD}b0iXTP{mx5T(D8Z;C8-PQf(>|C)61l@4xRij+us`V+wD)v zo=1`Eu^uSj^vAX}F3U#A30hy=zE6>xMPm&1rZd@mkum+Y^PzEs?Cr{g$aOY>sxH2Z z?Mo$(kCj#3{AN^T6KO44w|J4Y&4Nvmk|;Z*aY02d1n%`M38+Cc1Gma<%c&yg4o#dR z?KR|&mt?loxx1?&QVom$f7&LD4){8;czM#Rdz|}|mUwU-3-&yuQT{YUvFBk;v*8$i z4!)%&&5q-biS(BDiaa9T`7n!sVd>-as|Ead15JsmZ$b5l&VLG9Z!SvwMn`E8U>kTB z%x_5F=kN70pGz*OIP+{Va>i{zSSmSo0~nX1^MQqGa; zQ)Y@d*1PJ>Z*RC@$N9^Z)^v;y!++0Ve(clK0l|1l z9%~y6nv>$@Cg?^MfN&BOQp-)h{F+GvN1}WE90Ad*qT*j$Wg8XHFnFF!*#{yKvlO`O5GVaQqCg)~$QQTQ!Bq~Z2)^z3O1=%*tqyM}=`n+6k( z(=6wpn;eozTbZU;vl1gPkSwPA`o744>{J_SZZ&Uh&DJ}?NA`*#!x+>9S=iSx75wDmfp3xTI-<6ZJM*Z|xy z$D|$rvYA{v%*pt!Vw2kwMD&8Sb-j0kbuM@7>M0wD-G~PlP2J~&)`*G(N|LNwfeN3r z%B5Yf)C#0&l$s>Yh86?;x4wfRzRo5d<&(b`q6Xbb4hO>a0~dkEGeso7;pM@mQ-Xsr zTOs|Zy0xNTw=H(I3xMH!@Ae5SAUk=B#zDCoo8ZX-^78dWx0^g~RQ#g*agEiLab_e3 zXSS~6@iRKhH~B0~iXfD=x+O?~L7ea<+x;Avfq1Ma#UfkAfU7fnk8V?RTd$PrHJe4# z^@6Ovt0#Lg-#i_P4d!WQv5eE@3qP7l)#|^VJ_G|rWYOSEGp1PusLWLhEuG1S^dX94 z^}>!Jzyo&bTsNdo-Q$v2z;#@#TG}TK(2A{uT*YaniTI(T@iS%@j`NmS8TE%JVEgJE zPpVvZOK-2Q+i#{P|MgG!@2`27@Qh-KW90B=7Rk}E4<^dPj$I^WJ-NL#t86caclIz} zF<=QKyoqLYVqN8BJ+<`JZ`X&gqF77WRHbtg;eu90znc<-<&DQu%!=US#Y#j;iLwX5 z$~j~uim4JNf~6}(*M^IcB0^$BGD8z9jAZ4oDCyDWh*8Z@h=Vl_r0H)3;FL(Gq@tp< zM7oib<%mk|Lgcbxh^7{xp%sGVkg3Rcl|&pxyQp%4e@O`$NkXw`o)#Ci>nO+Ql+ejr zlO~)o9?DzGtQMWIG0YDu@Lnb~iGUHqf3b$kpyI+~!B=^~-vE%)9ADrznJgiC=k^0# zav^%6PtW*JibD?Q!T@*$&^0cW2?NgubPZ3FLEK&H30S(WH;gkf-H8@&?UY9xttwu0 zt~XnwDvUpC7wT%q0JH4~dIiOkv*tg7Lj|~Aa?$7fd=>>DzfFV`U`1$UKF9)a9?$fR zus_m}cOcvuWYS_Fq9MM};j7%1x|qTK>h~_P$n*Pwabtt6ET8tC3_7qP2*vA!J3{Ad zIO{#vmJKJf;k+!f_1;sJq`qN8uB--*%m8vqo09-_g9gq|)P3+_u`8NS5It^DerUMT`?G=yHZfi8^x! zN<1H6a|=VoXp)T*z0g5u0RgU&g5fHp#YF~d;JXP`P5BE$WW~YCp0Zl4HZl3Q$z{Ts zQZnRdIQNJ_b&~6I>5)aMXa#K+O4P`^3p8+vWS7C3q~Qqd$=wR^U}?!ALZlK&A|(&P zMwF}(>(ub55D9{`#YGsqVRW{774(LMV!~R5PyzZdHB?fh(ioz1wBy9t(oHeH4D&g8 z9Z5(NT>Yjz$C&#pdMyk#Ed=$PjP%5;k~y@t^!2nQk&=e4V>=ga;=u*%ISBB^bUCWo zngLIJ7|zb~L_OAbGX6-RN5H--C(dTGs$T1oX|>GZxTgnQr0}{^&*Q z{=1qcLoD&MrPUG{BLq&OhGRPc=7Gm4>UtS`UQgl#K22lfSvFmdOUA>|tANmhdHR7C zGpaeuR7ouO4_r1RZna?5>ccuATCkL%JNEi=lfCWpD-t0yR1>C*@7lwpDEBVp$;JUe zuZzPm#J~nq&_%?hRCAQ+ulW$hn^}xQhnl+y~=BkN;A;! zwfFjAamAdLUWhj^_?&G;&z>+o54H^SoSf8g2RM3KYwQPK!IJ_~$T|TNJD>NbO}mauS2+-ys@q}>1{|a2E)Cl8 z-`-wc?-L9?yr$#HA+VTRF3Y;aTq*K_4T0pu#|2?OggFaJOU)an*b5^;5jcS?`^oQ( zR-jIOkcgRjZ?65my~JKXArW}rP6~I0Xl(!-U6Tg@+m-1(UBgVQ19&p3s;dvk2)4nYF5?FQj6zVTRtUgkFU?Q1K{DmOupLH0TL5K|$fpXqU*OUrrrY@`oQ zDqjc|=)F9HaKl+-ahU6uP2hj3cJG>XKBlVfd*HV>G&v)h8T*}IxWOcmGB`@>mgjj4 z$JS`wPcvF?lMFT{w?7Ya($DMm0^y^QkYqP34FSXt--R$q%~x!7?zcwNpFQ@Uf9w~{ zwTwHm`aT^ruR)bx8|KzKjCl88l1Gilf9ZVRI?RDLaq>t~?g{oS~#nDPkubkLSJ$2M&P+$p<@0Wf| zL4}!6WPMc&-I8GQaE>e^;34^+^%6JROOc($IG-)x_hs*UpOwL4&TJ!1k^39eEay7~ z?5H&QJUtQy(Z*olA7ug#N|GADS`0LtKM;>HW~2di#^rLh01zgpx_FEkt-)?cphKCn z$|0|`B^Q5y!(tFZnw=zqBZA+*@=6?^{4Fe(CLcn9#kCA9BJjTH`-hemX?}hKuqaBA zn>A%QuHx!ySu4roWq+`q_swf0IR_lLMmW;|TtMA2LXx7AOSSOd;CouxPn4z~J9IQr zlzJ)7hbj$05>g6)E0_-rLg1D-2qO58AC`)i6e*s^K?0_pdjT40s?;u#(4r+BE#Zdi zy6x;Pujt03Ps1RB<$n*bSTeynm+wm0ZT3a;ikk z5m^)iI=tiR&pEof4v}L~TT^gNu}JWS^Sms84DMkRl&8>FSULSl%KHGDB1u&O8iP9i zQGe+#9X=C0tZ+Id{gU(BfC`J4pC)BUlA7wYY}bC63h)CF zM1w{v{ZmszPq=lQ=a)Nzd+!V;d@SdnPbwRYgc=Pv|MA1Ze|*lQc5W_^G32X^R{~g| zF}dXTs}4l&ZDNmFk2J%4DE%QonSF0=ZdM3}(&a}c?i&PLR?2Z+-qTJP_~1893gj39 zKnB=N8vvNLzIAsVYKl%5GbU?lCPB}`0mOxK(qVt;p9NEOSsiS3%nNTYiAfw?v|}m! zpH7=ST{msWEfwL9+?SW~U7qg~*RAcUq8;Q2pfgWyKQw#>PK|LUREl!U4G2l46HYVf zppo=JpT7>5(v$!89xH>6!Pdu`Ub1+~`hwugiBQlCRiL}w;J1L5iD zbdn!?(O>fcz^zYveYISWqE)seB!WB~+ou&_+gIH0vFgL$?nX|M*St=9{sKdFPK@f3eorQZ^3f0Q9zidW4-ZMraG zncu?BFr2-OIek5>ePEQ9q?g-5t-XU+7hc2;-;HM%b7y~u+YNxX^0v(>fiuyyW)muj zPFO0P{mvra{#^QPs)rAOlOiJENS4$k1Btz)Xk0S&m~>jscViSm{E1>(rlw>u zMQA&~J9~g=Ntk|d3=~2o45#{9pG#|?#W|h}A%{LuWGn+v*5XpuAVM@tph@A~hK>&0 zeHO~(H9$s|Eo%;+KOqop9^5$mMD-dTDAvYw6Xe3INsp5?&6e z17Ql_1pfYrRCj752^?;#1G|3E5G*!500lK$EOyX4?9}fBj*Nx%ZLT38O?PvT1}I3z zbMBG7KQWjzNhx6gcDeHh_5d|cZO@lygsYQS1&$41CN$Us|5RQr9Hrtf>*_3TJRK)_G+`SzH^0C!C*2V2wq zX0@f0ZYZ&}->ko>(TdO7j1BL7gfsh#P9XE`I4doL;Cgl}xxNl?8&%ENUHxSO-bVkE zZ?Sse;&sYg=ly}$9LSZAjrv00u9B{5aQHWkKAZSrp6B9Ef;6<)e1~n`-`!T>$8nP{ zY}j!CX|&C=h8AZ}epRDVj$DItR-5ycF2Qi8dv%(KeesV-(f$=V)3ImIm@@Z`=+c z@)wJuPBJyE!m7yzz0+GWe*gUw6KMYGpMo)vfYc-DUSbQ8_+V*F1B zD#)=va953*w}u5^OX&$@10M^Qh;SWiaE$`6EyFhYOCsdJ>PE&xW;MO1bNW%)~>5@@O}Ng zhy5d+qMD*sCn@_?Rrs{(TUQe#bnJCMe!ra!UKuuERrA;nUnd}{-rE0R5BSvoXVno| z3N8&gI8N|7170KZH_7V&t)Azu>wvI&^6*#kd8K-9GT5GRtWM^+LP9x*f86B4@^X+h z$XdOA#@-7(|5+Kpr{a;(tNAaf;|@hlMMZyTVm`$N%@X|C1 zkOeUVg;U_U`+g`DFjS|$6dA>L+X%pCWIjbZ0m)58hJy0|h-vzu@hQ_HK1LHrimi-5 zcqIshM0;E7kQqJA8suFp5LxsciHPG_*M1VMEblSkyzR#c#L`jOlFL?}5IRbrN}$1j zS1lc&Kr0m0s37_!`HMfu1XhftNF&qq(rFTB)0Ri+6ohYuI~az**Gv5TRxrpfb??O> zW~81YOq8ab@|U;|1|G#I!WJJ#M@L5!sb5J3-RPH0bBLYFP_uM7vAS5PH_vt$-suZU z5Ce)UbqCOZmRCaokUu0u=M2heE-|6UK+w#!f%txN|B%CraU!@y{s$%JkUJ1q{v)Zi zPnpqj;e(@C>5H?P2tt!7q5rEpBn^MVU$Qs`J5ebW$iQ`l^lNicJ)||On)gF*2P~KN zwPNR7uNzKb$vaKilH8EBZr6e^{HK8$tv9sc3C+k5d6;PDo45O6>+@BmO0kmWl7aD} zaU~5o|3>O+%Q*FudbI$zasLu1npeJEcEdvfcCDM|qO{Lr*jTOib9hdraq~7N$EIPc ztY)ECca`tk{g@g#NeRE9!cp}?PsLhPt{$a~O0_tfCw}X)o%w)b&uKZ}%-vgB60qg) zT1tMck1?puhHvZtNE|ou`MjM+x9ANycDdd2-Mp+fu@DFhuPwEDp6MO_Xd@NMX}PCw z7lmxVdQ)qakK0QdhiTq{2SE=C8Ewa9j@h9t{Y?guL%-hEL=unnE(v_U^jnEGFn~1R zAl<}TT4&iV<5v|{TT%hqkfb6PRY+j5j1-b$ei;?)W)HWLw> z-&Mi185h&|eRGKJ0=kzI)6HJD;H+UG&wSc)`sq~LZo7crYrXwl2CE7J%<5NBC?ghr zF4ukh=9z_1uYXcX1v3Wzj}9o|dU5>@dR)8z?d}a+7%&!l8MC->s6_K?(_sXkf{?hROE9z!_zU=M*9nfRh->LN{wG4ho?y?Mfe~eX`=|)vaGN`? z7-3=n;|$j^9zsO09Q|-%|4*49Xr&ya!(T?!{rwuGPQG++MYJf42jF0$Kx9M0>)H6{C2Roa*`|0xxWun_EK5cxd^mI>ughl4=_KHza>X^7Y>Z(2 zIV8ye2362XoBAP)oh!VhsCGIIQF>9t2Q9e5VkVtPnLp%R{13A~9gaEyJo!yFVUiA5 zpf-tu)d3)wQerg%m12()HyWE5ZP2l&=Fb_mS)c;a#FzUTRr72=E8T~RRA89w+Q1cX zw%q-JLcA@!1i27$HZN$?0IhBbF&r51(W=Z1Kx|Mi6@;YNVexZC#^&EX91GZo0lRPj zrkb9|$K-wNe!c%S#Wj^ZO&(_U@x3%KXm;n$VCQmE>c)d~TP&`91dT5e{&#uUwAHsxsQ8vS=I^gDt=5HXulIqVZ| z({XQwlmOiCj|Vr5R%ZTa!SKRRjmJZGb_9i}C|bB6GM0*+Cf2G5&0Uf@2ug(Ck|$(F zpUXTe8uH1;nTCKr`G#xG|4UcD79E&j4`_g{Ei>VEM#57_|6>(g0Q;W(}i`LdSOJT{~JEo`Tht$7nSR! z%8+4YG$_>CR0I7?V1nag1*)%qLiD@7;{zqp+^rTOrh5V2vSxz`G&xkKMYO=dqfUpK zZ*CjgMdn7-me@opcT5j?B`G!q6TZPpSYb-@xA$IsT*BsLMq^3OO`smfuEoJ(QoUap z=~d>>E0ErG^4nc7e$DG&Np1GBZo>^E2sWW|qF8F_3U)Qf>w062 zL3tThQ@4PwVD?h(dtT17zLD)SHM=-v#cQoJU)#80jYW+9x0TvE=|@8a%-58JYcR6) zxv+rK1bUiiV6%Qtwt2As^F4)^0)-3c+hrV1`I|3jWZ`e91Q3+>4x6w?*3Y}xV6_y9!r6bLSRl@luo2@$nF#V&a!sXn3j&*pR8@VE zh-AtJdIYrDi6Kf1gQO6X*8Lz!a|R4hu$Y7^g2Qmi12x=>NQ$_~(liB%0angD+_898 zXh2v$HbmxmGlg_dV9;Ixg26ECdcl4)3L++vm)!!}-j!KMIILfjOe66$LRQI7AF?vV zDnfAh(XM9;<>|wY=?-T?nL5ExxQ-%dXeh+lfzI&kAeUinX6Q;7ktTmtM2f+Yg|Ge9 zxk5Ac9TN!Kp^Bh1H(*haJbGuV(MZK~f6p{wAH?Euq0IlyRbxjv28=?B-sSiXYXuo22;*Od4IeJ&!c|sx7_+DcyMPFly2>c5D@EbL10QX9lg^vT!K6&8* zj9{j-UvF*=i*q=!5)vR$McE65T96> zxW~NqV%JPp`}lc}qVb|+hgu89;y^8i{>QJ=K-Eih5Bqsvwr`i6I^9YYAIWyLh17aK zrs~sAx#J*uKS?`Ar?y5)MIV5&PNwpDJ)P9W#A2+*Ux^SmBS%;>P zRGKV%U!InX!-0BR-&dUU*ZaJ^gFgk=JMM{TjoA?Gi_YysmwX`S{%qhM{>_;=U>u>x z50bM4kz>O{vtvA^ltlN~!6V@A<1mlZ>UbQ@QJ0)eUXQ~y+%Ea|9)pJM74-3AgpPpL zR%;7eb>%Nc&|5V2Ursw`Gw0&*`cFE`KI-Mb``p!H4$Rgm?X9J?t(@8_oWCbFJnp?? z9cl>mDyC(Qoqy_o=rwblmAh$|datO(?tQl6G!2|=XkVVSt810g?QFFyE~s8JHhpQo zKjNptwNv+p55xxuOklSzDiYAROf1Y*c@VUs!SK~{V9HCqk!89(LaN}ln&Vt0~a ze7OoPd%N+*RI=!>I_}(7A?ilwX?QixHIZfM)U)=%4r(R0-ElpfYZbm1nV+nfv2rX2 zdCB#hTH+hm^`*F}-8jv87xj+Nz%>>lRG@q$qKlvf5wob(o1l-Sy{5tQnvY&(6Ic4f zqxB=J++IcffFZYTbDB4U_d5Zn4+2Ec@6t6xp29xnvXZ%0p4<^~u}$+Zc@!E;qQa=? zBYH~rWxtZZjkCGnP_N0y*SKqS_SkSPwP)O^o;(&{@d|OMX)uNI&Gu4PFJ-6T>e&E% z_kXo6SL#@MS(A3Leayzk1T%Zdm;^CfN~4Pl8pX_fT|fWZ3xKjc{nbd98qVD@R8EKr z1lg}$3CZT}vUPjQ_xt{hCWjJgz;M~ zP+A4RW4RB8BU@4((CT+*W5Jls4x%MXqs-x>>`g_19V4LzDNW`H5Re$}_UUNA%-M`9 z6)F%KeMLP1MMz3theNk5rxFX||6{AsM2aGe#@@n+Oyj>`8jC{ovEs=!9V1Kvcz20l ziG?bXXf4^>lM{h(83ll-PdLpA+VmSsQK>;R`G7M#LY+Pz1FSq8H_$6#_ zg1T9cBiOKNbTwF9`#gVJaZp;j=oz-nurV`kP^?v-*=MkB-zP0Uk+0jTT{$LzYnM#~_SGUgR zLu^Uy{rpt)4bkgP5OrAyuNp>PKNUCJLu-+Tl4`fiBy+>+v-H^v|a`_V=Hy*Cb@BK z`x&CHoM~$g?4^!)U+!k-1!SZCpOew95&QL4W^6#*%hLt1z`V|1`EMzn9|AY_ogc!Y zDwAb|9h821srqxOFUt#Xc}D?@16#`o(Ac@SY|pj4-M%7jxE-u#de?b4+7oGa=!cxRy5wjL}T#% z!C21&u8QrOv7ZiUZ>Ll8-gOnTHFzmwySMFVFV|H%Z5NBvsfMbUdHlfNJ)LDaEuO}f zFZ1iy`ez+Ol!ZE+ss)XQL8%6RdF*FAe>?X@h+lQrZ=Iw9XD^Zbc>IV~*V9^2ZfJBZ zH&$E^e<#RLf>M!hA#cb5WOFq}9n*Q2Ut*g3d7QtQVCrCn5x6>v?hB z4MauUb}EEhsl&9~K|R;&N={ypOp15F#Aqd(pli1>Y<;_ZMD$z0c3nkJI0rbqEEJT? z(0ShoKd*}aVD!y`ScXuHaZG;_^TlgZ6*8Z`KCwA{oq|W?U!-`?Ng9r+P;Fk#-lr&> z;GFfh4$m|H^f@Xl4XQ?jdL7VpYc_dnZfe?Kcer}m=4P&Sqp~1O!>#C&Tjw?H6mK#1 zaL=RSKI(yXYeAc@Q=*(veQgaV6`3Gs7GQ34}P!I6}xl#qxaj!BJ|?pkq+)Hq?r zkH-)m3``|<2E!aD4lWEi#Sw9L-Ea`+<1wZ%BG#Pad@pWP;KbFiqJYnh`8Jo@{lJcr zhx?TvCd4S5E;%q@*mT-rqeW(#FhLzOuU@#@wXlee`B7{~&gmC#siJ}g{PS?R<(paOvF`j@8T4h199Gj?h z$)jTKTDN^W7NX3+{4rHYq-K38e#W8&jFyk>^y%Cs!|1LBA))k}8T~sFKk9?tdF)Q5 zeB!aD)3J5#lfEz!r`9HU>m4L&fQWFuDmYbjWK+iB@$-8Jb2a)0;?=ZfE(02jHw5{> zp=x+d%N++J(uwXWvDsTG3by5F)RkRjH_fGKq5}qL3=jKgd`o65da2t|cs>S!OpYrz zWUy=A#hL4CijjJPAFN?S-@u;iy8p(8FE(uPJ34u`=-<)K-H9^aSbRvF1(3+m)Djb< z!Mot7ShmR-GK9baOI4t98JAU(tf!A6pb^$OJoUHWOWN=Rng%Akk!vGpf|yJBy0MuJ zBKS^+U5)FPH8zD?r06csu-Z6*su^XlDzF`MKYW997gH2bw@)}f-yn~~*yu9&n|-y7 zSJ)X(F819_f7hL;`jp`?t3(c#4urKyUjq)&|0Tn*4AxyD80n=y0{p(`8-7A ztzS_R9R?As6*8{|Z-Z?qHxe+C){s@Hh=Wlru8Gv(xeZUrdO82a)NmmGvnO-ZME8;M z@oi-4(1s4DoXQ0*b}h6b_RNvO9H`agoP5UG<6<-ShXU{V6p;eBUVGMb15r|OpiaAj z=2cY;>}y+C3VbTT&32`1F&*R$Dyjl5Vca1Q?&Wzqcjj_vJyAzH3%I~e-R@BA+h!Ea z@aM&@w!jH}AQwcRM8|#=sN2|IUYT+d+C(Ph7_VeLmvZjiamm`I$|V_eY0Z?w+&-as z!9`7ECn;K~>EwD02rk!!JiTkqcel7n*mGDp7d26~;wN9l_ujG)wMWj^SJ(ZJR(|L2 z$svP}Hnv?_BilGD5AfFCi{&}>#D8|{(_mf6_cep|RN0DE{S$K3ras(zXYBkcw0S0u zT2@!21X%QSnJnL!Td5~K&<&4!+k8wG=1y7%6KZ)Uqs!qt=Z|YP*g8HhaDV6_sbJ>V zY@LCw1nhg^0NI~JCRZO$)S+9`m zFq*^VOW|eI(<#y)$PsQvWWS^vd+0gHHP6jbA2i{ehXODjIi%Kt^^SqZ#lFs)Z!Spgk!~;p)-z2fQK}db_q!(;E5*C?&$E4Bfhl3 zRz24Z^M16*i``SGy>n;ZAD3j=qOT7ly5qoI;+PQX@kE#CYAtI&P`Ua2Xgvyg#t^e; z%c4E=$LC11wfilqG)>M{ZGul#N{)a2hG$=+OVF$lktt}m{{Nw#{u`gfO^3iTV8SDg zN19c^7|+OVut4Y?UPz&p+d@}?V@;M$CVjyJf6QjD&q-v3e2aw1xcNf;Q-lGe3o{p8 zQbJBjlB(F%l%!!cf)s!fQQS=Xpe&-r>Gfno1ywZ8Bso_BkfuvAP7H@I8&Of}qK?hk zoXpt}?V%-CP=)U@bf=Z$z@eP0R@qR&;^V4aC>O*3k^z%5Ui;r6yIh5CsE81eK#v z6C5#)Vm-&P-%Oz91k*2=S<${Z2m2;!P9)_3m~l z`tKrko}<75g8sa4#`i{srruvUNrFT=2fGP;YTRn>D|q1bBuzXAAkq>*dJ8LvbSs{F zY|GvQChF}FOFHFdA!;y2UM|2n2>q zLooooy*vfRRGy^1B?cFTz*Dm7I!m^k#KDQrlaV7`>~X*Ig7hEWNXvqkfk{?XqaV)i z>Aa69Uj1r8)f~EV(BC*41QlNjm}o$+FS=^GtnW^d5eYC@Y!K9aK51&v0S!&^Q1`iy zulKrM$s75TzrbQ2*31@Qg75>!2CwfsaK&`llS|3NtS8#R zV{*PfrHwcX-) z{XT|+|7lu&*ft|-L3LZ8y1eX4yX=RziI&0fef@XTJz@<_WppJDZ!x0P&a)nCzRTbD zi9Ftc-a2Y~(Uikj;}O5p9bMejbolp=z_%qfWpxNO>FkcfzvbdKOu0i@uyQNLqG|mZ zH{j|r^lPixFuv}?ZM!L+27!O@x~733E{xt`8f%cFbtph$3!!e-Cs*s!6W8?m>L(Bc zUF^=XpSEQ^T@Cvndc{P479K%+U4Nz2gWkv|rYMuKl3oM(g2H`&JdLPxFJi@*Mds+Q zIZW)2#90jEosqF?B+iJ!T}$=N$tF(FmQ!kFW`&VH?%azbECzi-2R zbDrLswGOwQ*=K(A>+#@c$$uah#;)A5uAG=;a!0|}xXs39`8@XSE=tkCFT5_x=WQdq ztfLdOuuj>+;Jyc1Ky38z{P6QlQP*zT09Ji6rNHkp<>Oaj8!k?E1p-$m|_z> z3l-0^cQ~U_QNex8_8shgx7GgXRrZkE`oTlsX;i#9n`zr-c^75u#t$Pf-IlKR33Ps} z7V2Y>D3E&MdmA>AI+Q}4WDl3w<1t6n5!2xQk&3<$SL1Wk^&9u#{o^TjY+c70%fdRw zJdFft<5!Z{vHG!GdbftI*NieO&u2r9&u)kDCdYp6wX~wdoHV5A|7aUt6~w zhz81lAWY;nz0?e%fSHD^c!U2Ncqj@mx!JJ(9<&Jo9gpMcwy(EboX^l3(2fmSF>YadNT) zr56I@*6-Qv2N|pQDi6#YhT-ZlUz{MCZ+K4QHOuw4(3COhF1q9lG z@ni$6)cPO^WNJmxc>j5{gnjjkEr$1lQ)1Lh2wFB>0)9psvlR=lP%2V{&Hjx#*LJ;! zLZqa#=8u;N86qu{Jw_J5nhNgwGC+_xacG*R3t==Rn;>oUQY?PsM-7}YDntwX8860v z$pwf+%%6pz+#u;y7gCo7H}HfnEklNgB#i7P1h!Nt0~g8Wk@#)D z-Q5O^8Lb&92&R?GVgJLY^bCMhU|uiHQIqBYJYXb(Py-|im14&O&9{A?hh8gGh^(oa zfL$cxDCs!@*Jx;H#scLg6ibRC$P(_92mjNpeN+SOthj3#U-oA;Z!#{OFb}AOxw<_} zNz*anCQfb==X5DWBY^L=RM&9;7x#(k1wffVxTQ9L8R$)3KCms?y}GsD+9=37wjQt^ z{3Tu7J@BRIR3$B>Hhd*_VHZUMNZ7u)?+4RhFG>FSS~lINM;F-y6_{z4*~m_C!M;B9 zm;7w%Y0$sUglA>IBf^y_mXM(SN;C=J-z3AGXXLmNJe{_-j^2b3>0Zmqi}|^p9YKr9CaygU;5Q9J-z-b~ zo^u=F-i1Vhdv2bUg01eo1N=A2kYhub&yP0S*gtotAwnC-rrY{UatYpO2(1^p%}WRt7Qs37WDit2yO2fpcn)R?5@XqE%-3QFfyR zm9#vJf)Od~n2Sr32C+9gADYjVJ6d!to-QGO&T7n$n^SyBw;IQTRF+q&Io_^>yg+^% zkHM&AnsODQdC_8XX0S_deynuRXl~S1ukBS3WP6f6mt}%lD0UY@sFvxjmfId)H60CZtnvH3ImQrm=KI;plZIDcY_Q8q z=(%JF&gSs^x5LI^+BeAhk!Y}DPPOOlExlIvZAYq>e%f=Ui_QEVw57sP2k2{v&bE8# zPP1tycHOT)?@_dGty?pv`{&fME9V@|?jvo`)cK;DoOM)#H|3dUW=+pY`rkzuN;LPp zEQhzHE&87n*)E&P)pCc4ZBJ)mTW_z1+Umh?JeCxtR-_#9xu?BOw>RQeu1;JvWn-yY zHE(Z3>kmVy4ZaPKM+^4TTgT}%1lN%?sgD{AGf{{*eekBYAM8!T z&}bAG=I$@<600Z)o$|c2yk!@uH@emtwe&rg2}UL4xUQG1Ro56ks-_vw!Nxv|4NFTv z%T7{sIxTlKDN#1H^~^UkdqDE1q8#&<&m$bqfix7Gzjh0P9A@wzl0S&EKv9x@>7jnc zrvDGN^6xtU%pho%7U=lbx}BN%_ z`o6EGIH03pG-fM6_u;EHbkAtnanhU-mPJSjKwpta+~Hs-P!zE2NcuRSbCPcPr54YA z-_L#khH6V;r=X2ed@kJQs{QyEK@8GO=-HXoSzwYN%TB@=|6PH^J{fRU{`3tHW0XeG z=gjvz<**9d#9#c@N<5+aNYM#gEdZF=Q4vtw^Zj{!|1nB7L?x`;UX&n=X4YxanCGukl;gJL0YyZo3d`coHOje-5Z_OOxGKf?y4PScRMC>1Cdj zI)0kIrYVaZRBJ-byd~)%KXmu6SE|Ef)`MU@`DzQ>c`-u8?7mEKhTF4wx8@#mzO`M* zP1~w5k2inzFkQQYJvQdom_>n+|32b^!k^gBd!oNYGyCUDPU=dsz!*05kqtU0iUmT$ z@s|`G7D@KX(B7j)kS!0MS3~?_nne8FZTn$0+*^~ORF3C`ZJYkSmL0fRYq4q!p&mf; z6rtLpVYH7HCltMIsylk+%B-=}efRY-dwDJT1*{#n)*`%k5+-Eas^N81J$VGb<%$cR zUZ3B!qteN4Z^^Y0NRc}dm8aSDES1HxH;lD2?`k7oy8+Z0IDAV>HiKk=Ge$Wp;*(0< zMj@!H!1)~<^d@eQDkL`(wXej^VsAewg93ksy;R2BW}#MJ8FdG2mt22c z_4H`p6&lz4ZHUi0UOD44D};!IQmit#!)Jg*WWRqBm%%Q1e!Y{0P>xu+~r30z$^<56y` zPc))uesnF;a`9s&>XXOFTemB9UH$l(Pc&kf>+?*oQv1(Aa0rGth& zO6@P54k25M>TM zY_0DhlsXG*F}UV7hNp@@Wgi>lp1|7}raU*dedMEigz|5BbC%_0PSz0EYkRwDeVP_# z5Qb}B{TZUSnzV?xFjI*2YjaJwhT5h#tt1Abhjew-jut&8K%3d*OI>GTh7;dSCIUAk!lIGW|CShp2?vrX|axr}NZ+wPr)Yz)>F*W_$xm=eU zgX(m~NrGyP?#gb!fC!8~q>qI{Ru#hu?ixx{sa&=e5nWeQ@pgH8=|6TVJ5(Gg8E`}! zYc{tQQ8lre)+^WkK!Oczwyf^0_=qFcDBIJjaBT3H+PNJZpYc<>epn7i)?Tg=?~M|@jtwk zucJi?to|*Jr}N@CpcKvXzFhGs6NPd7I1nf|N%^aZW~s8~1-)^2z^^+6JmJJqX0Q|( ze~=vRpw}{mmv1CMec$Hu=rtR_f(1#Tbj4nZ>Y>TS5a4c-2nG8tWFia66fs5fC4}6S zxSOTp>a2#o;2?COjm+VYeT}Fjf*$d%mEoIEcf*T_74Avmcu|TA{3uU$0rhvFvFQOs zIMD>2wAqv*vk?(sFN25 z`vRz41C@V;+mXtsnAj3A5xHL%v@e9%5=&x~thfl6RVlq()UiDnkO0cYpq@1K{jZsF zo(3`ZoD8O);xx?__H_FdOCXSY$zOChK}dk=?!HPi&JjsF$LWQs4)d`mlg=pAjp*M; zI?C2(u_GV{+69`(_FjzmGS2`DYp+3M*+dL;`vE1iUU9QNiaS`}g}wY2;h9s6ZsHDC z9O?)iwq>nE!fRI=F;rW*LGqtW6bs%P14rX}=?dEr2}^HQyE}c58J>JFg`0n3CK zzOijAFraBLFQg=TQeG4kEA5Wr_X|Vc%!^O7PTc|+FgzZ(K`RUzd+iiXkITx}VmHWr zqbFf$4e87#1uLyiqaJkuw5*P=WXR;`$R zGsFr1NL}FDS!qs%1^oQ?it;61;d-H-cj^A8sVuA-d-Vks!#Px>oH5t5(bMW&LFkea zDNxDaD@$#tFDS~vN?aiAO8&4|IegAeN+v|G|0L(fA`c}eAqJ0;j-L6#CQF+B^Kp6Q z#TlVCL26Wo4eQPC0k`!gsrrmS=4{uAK+vF+Sq9_fN=$OvurvRIEZ72l%j5o=bMXsR z%(Cd=B1L2tuld-%xiEZ2F#I|ia6LxIdnb@{%z5PGup7-2!zxJo@wVtl0YS-SsYQrHcuXicuw=4y0 zF#dT71Wa7s(g;LVMqXw#9sEU&CE(D!qX~uZeyAV*{6@gTGBEF&8A6eA?omM^hq55b z7ZW}DwgmKjB%HEs+IX$GOHhi%32)`{gfZ0jnV3-#1Al5;Wz%#1yc9;}*LQ}(9IzZ< zY&>&+%XWK6g5oP5ACvR=Y)o|dP-H{kvvvkE8baeQ;s3AL%KudWIguD6jQ?cH3`I5f z^n=}jEr_Fo0Kk9$6Hr`$4P@>>AE$ixh)RLvd=dDG8W=lPiAAUCxI`lgYd9_NuTM%@$sU42pvZz!@rt}S%Th|Q zSIt5LHY+|UBO(E39Att1-m&328v2Xn}P$8;WMSlN?!ofN0`#R}s5 zhR9-mn-L?9htB)x!|o!P%*(zOXW1^&mFy+|NT_4GF6JE1cfBb3}^ZN*xVhBEW(UYp2=ai>Dv9K^UrqnKkedhH2xA$PavWP z8X~J9E{HW#>NS`ZN8Em8aS`N-D$N(3#kXld6?J3m_nX>@r#apLp<094Sn$9Gdud)l z-wVnj1i3)KYLPDIn`NmexZzI`EO5AWy!)shv&Qj$iE-?HviW8wB+f(>8Ste)RPo}( zndAR!Qm*>8spoF+;+G2&THW-v0~sCn#eIbCd2%}*ZguC7y1bg(Yo8%(^V z1$H62IEOMA68>?!O1%1>(QRQ1*Aq_BhOBaeO*A$tg56}fyNv%HBx)E~h0o)Z_1pFauldTRdsX%q~ETTbUJ z_+u?7Xj3r)>G~-NG8sXlG>EdPUKhe~j%E=d=>L=VVfW8r1;eg*5*S+M7!`$Yav+cUXb8mh_&1%ZC4LK|^^0QaPc0@89arxQ&;}p(XQ3Ws zx|GI(agGt6%kl6GB$Dbmq^+1!n(kU~Tr-*mm;-`syTtpooBy8wn^K5dJmJ#FBrD7Ic0`ZBy=hG`ajCSG?Nxw%@obkjY5**|6M5Y306NeRL!)VdWdi?nR zs5Ato3xt618eqi_oVv6^T8M8B+r#Oxs*Zh3Atm}D8~6(qc)!hS(nC(X555%=v7T<* zXin-BNN)G*?+z+l0|~;||;bOn6Zu-$Io-mR;#QEl2%C@opK$ZsK}~ zBYfTdBt+CJL%}L@j7qM^y|`KUE0kre4PFoZy%>SOuJTX5 zRvtuJ)gm)|?RMTsbazD0O!je-1VQ3KB&-FpD5V_PsB~#Z!C|Q5jJh`B!oi+J!8!r=zoIjZW z`kYJs0{n0;j!H|%>m!MaUg+4=>LZ3Y+vPY$m{B-(9YBi2sm&&J7{ z&6v?|+Wo5l}T%>TBW{Le~Uzke32kYuQT z0P*1)CsVs|TLk>$iWzYp|FOqJw!SGND?7WSW=U<-VM|cDI_Dq)6Gllbcz#tCJ1Ug6 z>s~a6;eyAKj_bFFQDrFe@0s*oFPk8y_Xvhy5*;{F;Up*x49-ZD%DDq+UBVae82=rR zZS2qXtEgqa1P)%(>gEgqRh9wJc1%_yj7D%A77e(jw}G%53S{yLNtgalEb%}0Ch}VD zv(xGvaVX%VJ%46S-NZ|<6#_9-G?10IN%AXmHiCI`Ldq~;-AakG}h!;^{`Hx>)Wtir&0ns z={njd5JHNWn0NSg(%ObHB(cpR;`QzuOGQK6Dpz_HZpU#s#!t1a%v__sMo9ivj#(s9 z=w$ST%?V@tpSK;q71H~G)}zMKJ=_6Hkv;>+ZL|^@#h!K|M8U`^0Yj|K3?W2?kzPhc z=gWrq9>j*%^ry^!KWPSXat~WB^JOfzD>gj1I48!pqo^uN9n2``)Bw3+hI3n&k7vQ8 zX5JVL-qf@rrJ5nv&@Ziqxpp|8$uxEqHmSdYSwHGM<+02FpaN{kA(EtOAr1Ygv&;)- zdRtI8#z?s*FABDj5{up)I(J6?DWM|auHPE>2$l%$JNPY)T-O#oP%mb0ut!Pm@Q=*s zDJ*VVF>7!Rle3G>#XUq2gGJY-dUoE1ekgZY=|cD~K?;z3_$DPI(Bitva}c3|tU90| zqfB7^WUYXC$y`}YV_v^|sjd}$nh#zDQ7_7tAN)a6GVS)d;8wKh1+5rCS$m1!Xho}& z6!2O0v#u6E-!7YlGS3BL=eyXoh61N$Rd`7f^~LKnWgzfz7fYP}7OyzoNNkY;v+O3z zw%XP}cbMIj(f*fhUbUR&%;fl6ToI>_92`VSCbf0a>t;w!jd1Yg56Wlh|bm z>Y{NlCBsomr8N}EdmU{x_JG~{S5-_l{cyU=wC?HOOg%e+*N9QI%@1QKS0>_nMEEu2 zwT7FJde@^UljxW=(rNa7$qlAH|7K_IU+I-GG2TjBT zOr~E3@OLJ*getSx9~3`S)Z(Y@x5WvNE|*}Uv^{h4@ipN{tIdXNOZ53wK&PQR02E|Q71zYySa6A7Zj!~Nz{mh2H9(xfm3Fmr>a zVaUKiX!Mb&W`6fyPuuAuADn~KP?Z1Sn2vSCOb<2m@=q9r4-^#i>v~2S5@wb^n=O_y z*`@pUGg&bRtcjW-X*3vVXO^f;>Tq5j3|jGpnF$-(^dJE&kiuUuZyj_)JRVH41B7RQ zI-^AJ`wp{|HzjT)>I=FE)I%;q)-3-+KU5OyqL=LlVo^SNKeZ*2vERUCT#WWR#yRnRisuS(fOgVtH=KHw|Chv$afEOwt{|J@ca_KXc|9zs?P-EhBUy;eo__!FYnCn%Wa;Su zwQ;9=2Q#rAceGSMPxKF7>Wd%>3SMuAf%4|P!zsN)G9I&sBZh&ByMd=tOg5YE3F zOSF`<1&D~}F6dssYTBW*{`ZriPiU^+snsPngzw?iBORN@Gz_Z$fPSdQREM8ynrfBm zq;Nw7(*8k0(3Arw^M-`x|CDQ0L9FRnUqP3&4Dd3v=bw}|gR5HfV0x8!c&EG*bWHY$ zhFhcCbMWR7TWYoy>MN&f7pKkwHQIu#!wkvyK%iZqQF22#a1su<>X8;$jKyxGTqa}w z6S)%BXE2tHWnn;`^)kGGc^Kf=t?&oZ^wgZIML#8^Dg?_PgKZN|HKXD8(J4tF*lLM_ z7Si|A@aiFhGHwu+!`fJ5wC3C~UD*1Mz0ULzYVisr(#UOY2%AF#BIlW>Y$VXu-O)p; zX$F2=7_UN($Nc;G}wV_hgEk zAYN4=9u2{w6&R0J4Vy254x*~)pLH%MR}8#ZeYreFPh15MEgkV}moiK?{S753B4qVn ze;aj1!i|#m>MX#|7t*c))Wp==OhJJq2>x>g4&M92Gbe0@KNV70a8)!uUS}V|Q~Zw3 zzNL0sG*wg^Ip2q8tSM`Ltzux;F891NLKIc4p||2FYQ+;PtC&|nE(orn0y&heDD_3Nz`kBjoa&V|qmr+V1B0%7KlcRq?i8Jp z5C~#i5pKtb+b2y+2|Kus(9g(2h1yzEUuS0V;SvYY{2a1Hm-yu>_n1xT)NLabQ$MCV z?OYM3R&8yTU!wb;_}wzwEHAMp?+8B3jj*TqVpY1^Bt6{|hx2DgdPBG^=pTuWAq9!3 zH&?rKcrsO=Awo;K|LWh??wv(*cCIer{lC=@;vbk1X~hFsN76gyVTW<(Id+TFn*MQqv; zV*d6QGiRqiG_eF0(Jy1ve0{~^tg02`dwdjv;dgV#Z|44S>Gk8F__?yp85GyFBE6Z zM<%o;(a7_l;HU#I1XU-TW)Ui6iC?ys2E2EjGSP2wTUBL@*eVvo# zK^jJMv)b-Qpl-LRxiRUE_j>;+-udrO=w{Q*=-W4!(vWXQretc-&3`4IicRI<#^%hvUi#<0Jn8NPAv zA)@6jdEOFl*ltq^Dj|xpIhRaQ3BlqTt{(h7KQ_9kl~u_^>}Rs`qyJHo0-V-r91ROu ztt=cXPGkArfJ+r-vP-)>2*+4@SJz53!E!f5rot7(A_>8}ZYUi71BuYAvl-jf+0#Qz zg_4c8#UCXu!If2X zFWItSyma1)xb&?_BaE;;@A17hvk)z8AxOp{TPK{mS=!c=CKNJ+=P$D;)E<6;MxKq( z_m50}ReKeH9rYc7{k9i>?UY|f%igY(3(Fgdt1X-7B_%EHk#{Fu;72}o-0mHmd)lzU82?0_I~D|W;suPOjD%Fnf^Dz88l5s0Xz>fLA* z+_-SFhOuWe<>`Ea0PI}3P-*5?K`)j5T2y-^SkN;Owke4IwW*20f!JG_tU|LY++6wZ z%H=lsiox=%miC>qyt;aq;j9tiBlHFEaPlUC0PAIJd52ki$-oZ#TPK_6sjPYj(dZIt zoH#tv&d@fI_{1rAgIT_7uyy=k?IJSf+cwI%1zWD%dU8x6duk)%~{bK&4%l$QK6^4xmeJlIhDF-m$z7Mdm2$kQQDD$gaUf37~WALkrGW@ z3OoWM2$y{kcl7k+q}K(DRNlTx0_&R(^bx@cJcPzDffFL!F-9md^+|a-4TT^7Yyt?q z;SMst94Hp3whSHESSia)jge zUvaBT#(tp9^e7&?HO`a{?vc^Qf5J<`q><_#OeSL(kbTorU_XOp^-FY@VQWZ3=|+=N z1hID#Xpw|VC=wj0nE&JazGox30s!!?$bmGNanPxQ>$yO9AU%(GkH-tnuOmgn<$a@s zj$v-JHm{WM11VTSb{8LedjR-1Z;*nhiRuX2#pk#mrWfs?Mu>0U&RrG7o3mcyTKSnU zeVkRRxjxhpp)gq`yw}2y*j4GNst~MWi7sHeESl1Dt6Qk>u@_q0FZ=X#;;!{KI_aDF zc?#&(j@_PC1WYk$*+gWDEgJK!^VRKI{2!7S_l#c-QKg!{dq5N#GWRhFo@j;yv+OTW z#SJj%`4*o)1d@#zte^7(WbwG+L@@_~9?F$Un0#lz{#pPBsRaJ43MSI|{Jv2_P%R~u zW)vtc?=!?2?sxEdfRJnUk2dH&Hbsc1X>)0|dNS@PNbHSf`Kund*Z!@&CHw5kvwT#M zPgew0mRhN1782cv}oG(>XejI=C%QZ6ZT^TCT7|+ z|EpRm-n2Z{cCCrT1KrB3$c1$q)Bco7S)_iCfPVu<)t%!l4{@L?H4if{)8 zb_c<+25O{)%@i$~$5yW~n7^xjqyFga@~9t1zyea$%wme$I#h)5zw71sgJPW0L3JUP zv7QEOxYsaU3Ims@o)H(d6wJ#!gl%WHOj%woCuzDsbZ|GuD;NNRoqRyZSus3HpzOOy zsE^Ti6qVB=rhcl=74XU(j>Y`LG(TAolz7L-u6MyO zyItD6bE~i7)~OhG7*El01nQx|dh_j}xj<>1)&3BfxEv2Uur;Z9GTons?!VBGP@Q-g|_rC2ZL9zYN#({e_ zU2|&+_HPxRe}IUo3`TQ#VTjTX#kal^RrrEw51x6mLyz>cOV22?lF7YViB%)5jTgyP zmXnp#Wy`ooNpgH=&z1v;KXuTdm-oV%^5>-P)G)u!Ih8cN6eDclxXwGqg(zJ{55U5R zwkMH8T=37^k>gsunp?IqO4!dY}TANYDLb)98|KgZKEqjdOerlGR(p;>kSpss?WP8sB zeKZZT5>RT{=)%?hW6(SUokQ3*#bwnXtmTjJ6amEVP;2Pb-TuQCXD_} zC%IG7W%fHZMs{*N1sH8gvT6yen@$ZybNt`)x<14lm@r9=E*EriFG~*&%pCaQFSD9) z^%yX1u?czH-7pY-EQa8xg~MN|p8*rbxn8#tb@HV3I>^$Ly@HY2Vd9C3%*P20KeS@Y zaS5ocE4vf(Fuuvo4$LzQtl4$z&~RV(K=dMv8WPE9g#G6N9^8+fo}WY{50Y;vJm-uW zWub|9KYvv#2SIc0Bx?$ifx<7I`-0dkdJ+0?G}el!Oi=!Ab3fnzck2Kc;(NsbHKg4GX_HP%YS&X*poNe-;d(0py)hX!~bZ>AHNWGH< zTKhh&JM~hKz^}@zfVx05_FI3?GT8KaSP;ZTLXgD5Fzj!D61_w}m2+B|CXTpZXtZF4 zx|?{SV3KnpWZq|`Mm>q5*P2apF@^@3aD2o5tWa%_R+BX@Vjv7k4yOZ|i`zPQ8G;Yz zL5w(N(Lst3^*=;TxPPT`xeV`ru_G7-Q*9l85|R#K)Beh<>C6dY4X-O|U1HVUCvX@* zo%N<55erK~+es>L32S>T;x>qrs)L<1wFv(&4HLw68=($cibf%~BLL(*+aG$4ON}+- z$$g&Z>{tn3++L8;1KF8G+??PT$D5%*eofqh8X?kXFZPx5hB?{qt;T9dxlptFX#H|O zCA_dUSec`t`JIp|b$yb8^**G}h+T7Y4WJD>Xx#>l!mfj7glRvTel;DgtXw%v7&i}> zYnBX8HMKn|T{YDAVTJkx5}tRaryAMFf4$z(oOt8>uLbD-m&{pSiJUqVIYbCy6zQ_w zL_QDmHs{y-9oL>%2)6vPP>2(?C`A?*sTsIX5s8rH&MmsC+45p7bj{%Yv_VrQMFO8< zUNMRJ@&ohMkcj9A`=PMQ3QeF0xz@PnZ7N?|4V=JG)2T~i@EN>VXHC)jdX~t$?r!GG za;NKbFn_h{)_DXVb9QxKeVl&5ph8uQUiI|$?T8hMt4WUyLu-oP`G`*4x~M!EZ(aV( zB2)A1C3wTm)uLhrS!{>hsv2nPb>51_qt?XI88RYm(#Xg3Vp2f+l-xTycXa73`T&qEA zmQ#XY3Ia-NkRZu-@K8=JlqS+3!l-_OVRoFg@Lk5fX|&kIf6c$6>%Hx zHN3W?C)MiCcQpRZc3|@f_M6t`(}9c@O)_558w-ibC@jrXma z_35F>m90`>`Tfi1!hj#Hqm%2~(H02U^Jdp&=b53mT3=(fboI3~o}MuYZ3h+*j2U^8 zPWAq?q;_#mn$qoDocQ&Cv(qSb2y?{wZ`o}x;QjgWaZJ_p*PX1OQWc?{>y3L2#rj%( zURf|I+f;h0o)a!$_u|5~$m7FlTIXM5tgAWknfZGrpwctwu0Fr6yEVkf-PP%0NAjRw?_)*4>*fu{ zG8*RX*^A}O?Wrc@PLADXOChuWs6fs6%4-2?yzBN7`)g%8E8FC5 zv5q^v-w}fKmN6AUIl_AEYU^KZq-N zHEHL=68o3ehyQJ5`p~m8$~Mq%3N@a$0HhH3Z}+jBfcU@I2K-v%=5DYouu~o*UVAA~ zuq@4NIl6Ge&u-?Cs4mbjqaup1!C|jT~Kk8@>68RSBjy+nBBh|T~FixOmUr~VBhV$qDumHkvAPZo@F);Kc;P3d~ zcTnzDIhsUt7h^t4eP5{~$T2;P>CC7nLMXJ3e>ZLPJiIXQG~81;}0I7^J6 zW?e0g(f-i@r_JWDIp7=a*kg6K0z1%2)MhqiYmPIX)bP3G9Ks|qhPdvoJ~A!SUpzxC$>}OcXslyq&Hnr zk~VBc>J|th4SVA9VH$$ys94;%+Gg%yB6Lp{Vc}E54dwAeNpKjKx|dl8Ef_Y*>#3*G1<*FUiUXLC^~;$Z%cLD zXvX=v-^=Ih-e=k#U{+BUGCuc8U2WRE*C?_1EgmA0W34wH3;$#iSzVo0H8Gs?*k343 zb?GxjK9)=;?~Zt{)B|511NFG)=h`~kt}lq+$L;`pNB7B6?EGy9o~|3s6|Ts(|7w8t z110(ffX~g_6uZwdBfeRe#{fwyIWfJb{A$ayHG%75+9`pmS%>R})`jMdYYQ(&Y%wBS z2i(#eNGDSbuQtW@=d8IS-?JRqMeJ(U&ZmAo??sLAT(7BgSKDQ;v*{&Qc6thQsmfxw zc4KKN1L{%$i@e%|&5FxkCttmXIlb3u&!z0Pjw;QKrW@a`Pfb(ky{Rg;n_nDK$}ATo zRSJ_pXv+7o=g#Z0rG{NtT0V$eQqsf0U7ZVC!{8$|fxXIcpNC4om;KpM_ebGk7lT)+ z^X*HTRn;sD5a{uN!0`IS!+w9i@c6XQbz2W}`FWrt$76Q^MGaW1^S;tSu~~V&4xydW z_83i+Gio}sdp}*#Yq{8|{?NJLi`IL;;XAv0IhdQ)@jNXziM83hzmU0*JGZ>DvCf(6 zd|EBd5wU|!NAEzssFc2r$5I5NY?vHl$Gdva-MT6Jnm^;9?H=J+&ya@L~n zu;=OX#BoT~*?EVLulEEhg`cr{`ZTv~FB2dkxy&n zw>K9eL>NVgFk;^5=lk>*FLJnnAwkBd56CjH{f&|e-o&3t8vcVW{-f)uGZt{Ti#w_NMsyL|l* z&H6)K*1;8aVCm9^+f^ivi9iICPLqo0 z4^b>AG@_+hgAbZ-p~h_DIM9XspxroW76AfNDIoxws;&CASEeD?mh>DxUsHcj_+Iaw zrro^WsH_mU7N=p3WC#L*|e7m};r%Z#Y*TEA&8(j3DR=+oi4t={=eo zT|T)Z$W?rOK9GnR&uiv$ZbVET9xcpG#;bUnCqg2i_I*0V#Cp#{JkRTmVxrqV^8o{x z!vd8Iq`1*<&nw;W?)JWTvNhM#Y%sNvEp_#>l9H1n!%5jrR2xw^aIE!Ev>E2q?- zzu*m{p=@mX+FA7A_M$Uy$?1#BV0nDJo7mpjSYl~+T2pj|Rf=1L=yd(IgWkx-{c4fJ zB;zD$_aKuB^!hjexLe;<2P@Z?#+p>8XHEW|4emtpeF#bII3gDK65M(ZOkRx6cH-E4 zQQymsy}lc~^Jr4$o6V& zuC>s$djsZl?mqc^>{2pbod2CyzY2OtbvR}ZBxH#YhP}Jk-*=-s6e8JGCWTe5^k-%%DXN>6Y+AFPq^D;%rzbbP(}k=UME^N==QQKAG?3o@tSK2S~EugH1rqPL7)Yb1aQ;%CkL3|jUhj+#7M3(VF&!Wa=fh8UM$o$szz(8>MGxO|0m027uf=je>raXs zNE!Mc+zkp0@u2?>$lhn?i63KZ*rjN_AMwh}73JVtj-?T%-C&M7C!H=F!Wi$xl148% zfS7dmJGMEKvRz&l(blK|_3Nzr^UTJ^CO=)cX#_KaDbOPx^@tWsXc*c)iNsIr1^D#} z{27mf3|AxaNHFz{QPK8Upbqh%4wZNXj(nQ*)q;-S7$3e6u7FFNj3}KEV)_zPefLK& z`O&&uqAHRjg4kK>Nuv@m14nh|>m2$+kXrnOsfkqI7f1FWkONWI8Z6NHP2nSO6h}u$ z@sSU`#KAsVrtMmGp!;CShjJK`2N%Y8uheO>#3^d&-yAtZfi1i6*22_!4nuPj=X}Q3 z8@o!>33TeeXs_L6p^bmpWI=dA-Y;s|MdLhLCrQm3#qtfUNX;lnaz6J{u!a_S!inqO zf@1ea9OHo0T*hwO&AxoPzIQ|og8=XM2Dt0Qs4c2)?P?lh(4==R@e2iihk#^)GSURm zZ6{=8WITsa4Xx=%(iZ=coMia zJp+*4XB(y3cSpd>?x<@j=CD{`eb#$;@xcN&W_3NK`f;I?=kpK+$Ws<9@O(uBeQtPkT^D#u)Le-2?6mea*Ow+ERjz#QoPa)K?mAuv&9iktlbOp!6Hlwi z!z#{)4Lfg9e14fqi6mlPPBWooqIo! zUMDQF^+wR<@^ZEBv57KN6Wf_uz*W5Dr9hd_yR0R@xvV*p?rZ5v_hsK{+PHDsSZl3L zXL9^e89U3NchL8}4PlKYn{R{A`M_bO#J%+SU2svweRRQ0=e@}3B%9#eibr0nqoOUZ z@Q(s>>9aiKhJz2`jZ`lu)~snEAi$OTL*1}4B6f-_gt*0xYS>%6+t|Wxsi1mFgbxP5 z?XFAj-H$nJuj$eTY%i?>I`)>2ZFL@D%C$cSY-&1r{y_$FWe2P%=(GSWe9mSQ0iEX& z{9Un8=2 z6~VUhGqE(s7~<&94YhJTwdsMoq(x!$nkACN<#n%>^~h|~R(4u#g>B|)YjxT5<0z_H zX(88Hw1BV3X5I0il` z`seI3z0YA_C-chO+JbayOv%p(2CvREKVN}lInT+nOI{<1@>L~HC7FQiEuw;<2il?<$A{(l{L zoer{|XS;k0!K}mlpKO3_wy?lxCwrWFco)sF_U87+=G*sD2H&Tvt~Pd812Lnut`y4l zjFj`TsWWR!E$^|h?vJal2erk9$+HugOT(OJPNvyW2z?l8!ZrJLq!6CHVpCLHP|3q_9;J77E83o4t{)t6%E>T&o%VNaEWjy( z%i{S?(c-=6;E=r3K;)?~#d&TT$bwGT4(7}YJX~qfYewc*xasF?_!|!6?8rQC9 z#J2{}p|zkBv%j@QLJf)v{D*yScdaPM@tMAd5GfhmJp2M6My^7=S0cUONPuBWi0^8~ zkcsccMGHR~$KpJ?_k$&H{w96>dQ4PZLIISe_=VAL=a4r%JaersFmjKSGg z201d!|{~eY|g2is*big2D~<;`Geltyjn=NN)yFSU-NX=6vrr z7>UC$1bECuii_}98~xai!-U$riE+K&R8-Wl=4$VDeQudl2R}*e!}8r-9Q6fQTVSKtn@4IOP8P0>nrz%JlsLNOK%l(9H5rZ4FH+UJ6zSk@uXQtwV z&lLo(&6!xv=z{4sHQ8&rc(Qfd?&%el!Uk~d?W>elsrTM-eC~9GT(VL&*$)m9zQlLx zpj3DJKIITvspWbsrdu60UfXD{TdbEP3P-iGEC7J{we3LnkJq!7*wbds+Om zHD_1$yHl*(Uc!!tt!Ef9;t!lR`y~~8B6qLx&L;su>=)23iQD8QA|%I_3W&Gg62yS3Cn_^(juueHDmKpj6o^d`tgE?0JP6= znot}+k+V0O`+GQe@#-6!fM*azBpiMh%*ZdF#Dhh{O0B6x|ZYpwE4At z`2nMv<$SARmX$=zC~Br)*}J@}R5y6!nf_-2J70_2(-Y6FjM19pK%>!2?LzPUr|w$(3E&* zCM5f;Im6?7G$TqDCwvF%!-x(CnWfU&#C#wfpJAgRY(-R&Oz>#gxBDRJ9t(qP2#%po zh@T9W2~-O6EescdJ|G+Ka}MR<%y(sh^CH-RzEyKu^O|$^T)v@xZdHf^?S^mZ9r`<3 z9S(RbPZ?!vyj1e=_}~iW4B04*^%yL+o`^(E+R6?EyH`gt;8DZw{X|xwnCyz+Xq=>w zA)8IRb&T#q1C$4!Z&zk~a10KJLv=-Ww+G{`3HIK3_oK9$lW1v(;KIJYTa!LlIxXB9 zW78QkGqtLpiS<_1bnJEiER}xV7e{*1M!vRXc(@f70Gm}$sz6PpwxRG`{^r?v>3 zt7>#3YnE12H`8-UNL+f}+}glE9+7IZb0TPg!2InpfBnZ^UB9bWNhJ+AzqU*c)SlB5 zIBa48ivqG-PPS!^J-AqFRg5wgN2_Gx4vDMerF&xo9t(@c4LSS$?(_ao+GTJSt_AM8 zA&xRAKIx;KW(V z^>}ts@AuEc`U)AY&)|4^B z$B{9K>>TR?=V+)ApggXtG;`g-+bvAvgu7MQA!u*pHwKkO_9DH+`yM^JwUN!XA?ArCwjNALdU^88pjg$= zL!5kG7#5i0E!9#RZj5<#9Tta9eZwLvx= zoP0&>+O(A(Hw{ada$(eShZgJ|Kc*5smISWeV?#qy6UrEziHQXNORCk`7q*({g-?I4 z&t#A=|KAhZTucvVj*O@hOq-(heR-csNFrMa-%dFNLA>+ncbw>^d}b~qj3BjGoYzn; zA9<|9G-iWTM-AB5Dm!mleqK7#E0S8TupIGd!M}VFURXpF5`_J@f?(ud*C=0c)9qvv z9y4@T($aCHA@tZ8lO*5ag;6jmsI=2e(#uecROZ9@4A`v;OJ*Pm3>D{pz`&QWrt_Se z#WKDz;l?I_t5~-D>nA-3!To&*W)KvbA=Zx0rI>#}r|`Y__rADBRH`3jNm|dpHPx$q`$bVS?fP%Jmz>ouBCIz_)t^W)<&MP^0Rx{4dxzVB|p7e!#$Db+8uaB)KR(j2~ zmakmUs?)2|7WXWe;MD4XvU*52M0d{I43FQ|@b8_x=1u%UW_o1eq33n|$xgqMh(La6 zZELo{PRQ62H_eszSE@VUJR$MO`w}S?z-1qYy{8V9ZHH+ZOVJ#OyU~!(a^& zB&o5)fj!AzW;(2-OS)cN16^c={ccA54PI8DjfHIRG%F;~Mg7{XZ@bwWD1as&j#X4w z%61d+bZ;IAv1;Uc5i(P9+7&0kKM1_D$lr4}B^kU-2k+>s$NX=i8#0(&5BDtwsWZlU zY|;y6*?0Txe`JXP0c?U}~-5fEnNTW&_A;Y%I+F<82dW<5>coqzL zHus;R1IqNqBFCAzxGR*|s{nmuJ;uiuM&7lo9RUdFs14W^w?(OAF@=f-zaQyJVtklV z6x`rns;%0B8h0)z5>$Xc@yDMj>~B(aWjQWJW-U*~yUJb^uYv({F}$-F3j}ol%j_}n zpJg>_2fG?jDpE%D(Pr_#9GsiWozvCBDIeO}Uep<8MyocLesAnb z;}r5`@fZ&jO3#q=%uE^BVapyV+hBP^^V%e3P|-(bMG?=1q-HpkEGio%eOK+|+n^sT zcV5mW=oGVU7~VlY7%4)HojnPv+h;EUw|Iw%SoXC2 z`OFRl)59o&E7hhdx<%Y(C8rsV{dtcgRn zNfdK1&WaSR2GR7hP5w|eXyqpbH$mZQ$166-qR87|Z*btqE0t-RqMYeT+cj&*=_gt` zsFB{f<%H+j--1hpXTcl1R79SH)gxX#sfewxmmq`Gl&{y!|QWbypU79Pc~RLI~tkTG`6PIg6O`44|D=tg_mOYoY>_B zDX)jEEM;Cql#_}|YpS$f&r)77dSkR5jS8ia3;?N)xIYE++x-FRMFn#ldSK}dsz#{i zi0wGxf}9U75nM2kH`{mhgsH$(QTmd4DrycZy;4K-2W6dV2;(e|X@?;!S zGjh1!0<3Dr@!g`2+iX@WOHol#)k`Zl(zXNq`*F=}P$_mAQK{02AA;op*Ra3-OBw)-odmOn;HW6DYpd@dSH*uBxR z&B}>*e{qz)vAD3a(E-_Y;Oj1sm_wy4g=qb=30rqWI^lk%=YA3yTH7-XOwow7mE+-N zR*)5)3a^4NJ6k5IfvH8c;(dQg&+BaX>}e6Jfn(BUKNDkv&&d;i_J|>(R~njr?$sl( zi4}B!$^bdyQ6982=rM<#z;U(HwqJ3YEY9s6Ol8d6h@4@)n@A4VqC)z8(r7&?%O%qZpq$(vBjZ$aZ2!0l&bRL~X+1w+o4( zvTypzXlsUc6W!qWB)5BN`l|-*=7FToaGz8tF~{ZZD6#f^c^S>6S-+y*&##;uVArmyRmz>JzNiIvykT@yhl0jp)I;&K#;sJ?~Xve7W7sYuaa9hwb z+QGkaa1JTy=$%>MSi}qWj%miUznJO5O%&HgrSg+**loTj8isq|nMxPUnnGVppUyOu zYMS&hv34*Zd0-!KA$Q9UM-wt1^u;qGj$#J18Bzs{c0H?z2-8+2!x)nzG z1N9MKTKmi1`LVvdBDR*cl9ptE@|Ye`t$-Ef4sP-dCcvRP{zx(7`R}ao6ZQla>?y18 z@a?C)*LUNhF$*VLqNd%-xL_&Q=PJF}&#~eOCUpoTSC1Zl9!Xj7TsA5u(l`X5c%>7T zoe`Gh@yg;URth8Ag^=U;AlD?G9JiZ~;tMUwp}()52E7RvxD&l~LDZS(lkA&_0objB ztK~<@wv3-`w5EsH8PS9r5r>N#%NQ9}vqJW$tyX4>z{L+5o-32q<&-cqOHgK`4vC zOf$X3_`$+T4LcHUu|w$vv(=utWm=Zgr>f9Jv$`(W+r~$>#b7L)=&R!h(=a6u1G!IS zAO+SnUB8qqgvC@1{zZHg=I=jj%Q!jQuCj$K$bN9J-}m7f)P3z)N`dfR+A_IkI#7~l zvFOBn1e5}MTBkC}JjW-wbM{?ranc<0vo#LRS2b~2BCpId`sX%P5Id-^w1w7(iMZ^n z_G0oMGuo1z#Hb?m_LI|UwfPzRUbz{H;Wn&x-b<(AbTg&RI>v>%WIV`V*F&6a-0BJp z;;bwQ=|7l*P?nNiVWx0w#aL1)Hwd&53q8GAa=y@Im*wxt)w*Jon&NZFTnN~zUDHNe zQ>#rAZmX;D5%%6sb|t^M_OKaQE{rgCeRq98IO`;U6^w0G)Gh}RB}fqd+Ls~&_kB<( zAq^N{E!`wf0YR^X$<@lCpcHUWyLFUW-R8bT@BWsqjwK}boumRz8I>t&m5tA6@O=;z z6gzI;%wl_v@h0u_2x$^tMds0}bvkNe7iOwl=T}2i3i!q1`tXclt+YUg25Ww3>p_aw zd}vZCJCKI2?qfyt4wZS3?J2N&KcGGnrE#E<)X6TI&-F(FP&NPYjW@`Z4;3VKnD|~b z;o$f&^+*%h!1E%=PN2%>7w|}HNUK_2DifM!7}LfLr1Z|RmYJwbXa>>Xvx)WnH+gv`1hG*)T+g7 z)uHWvBK$6IV@}##$;@wGe~-Ej^R^Ew+DHOYLSSWe#h_(9X}F@fu;5w02*H{6&N0%r zat_T5O-s}JSjKQ45fFA-D!b_yi$5|y8>FlzOKp1c8%DvzlF**_$D`9u95;x==oF*^ z)z~aDSXa~Shh48T{o0x&uR}68hld7x3dus&(Qq8>wSayp?_FUP$nWF16V#%!Je+5- zK70V7aeG(k*g+={VzOs;dlp)pGE0~!oq}j~E;{v={jqvECe~pLt|bZ5BeO!HW1lfg zAdZNM0{4sMGI5h>-v@*Q6V$|I4eRZu#($Vq$_ZoF=D{O}9$S7CV#Me{P7_RkEZ^;$ zD50BC^<@j@tr<*8x0A;un4a0>m=&3=O>nNFgty5@P2R@de1&8wbZd~wgMcZgTf&x< zh%`txv_6$S`DkRZt_mA{sSxlPbyx}~QHHS&*2V&-fQ&VSg9lS?!+Qy*^AWUM#lwdsBjFN)AgXBQRB{SHEN?I(EF6)$b$PRIF>yEn zrqr6yq(Bh?h<53hO(8P9l$Q`M2H_Go_EVo%gCY{2Q*{q1xl^W^z-vm}p+|5bwGviu zeO1|lgUa?a9e6&1N}OC4x?Y%}w@)E5nU_3(3OA}9@yyKV8ORom$8{GV0Zb*BZWr_7 zK*gWn{lj$yGRX<$JWyBFjhEZ$I=!EVX8FM!j5pZO-%P{lMKTa@3p%g7;m!2aulNt9ERnD<}DQaGA4TFT_BrHFn}7@zZ?(^AT_P6GNV-G-CgTDlp*(y?7fWpAexymI_xB&6H*0kSWB8t@oMOG)F7pA-O3iX zvZapd9E@z?@?$9|^tRmV?)f&)TwN$h`-(FYaqws28{_QI?wLmhMncKK0hRRFQClnZ z2&@{|u;d6l-sySO>i4ccaXQ|Tn}^i6*fyu6{zoU)#62V8`YxZFkfR|`F4>>sSXPme zqe<@eF;KOlnl?yqxjugo@9u|jie5a%}BoqTP3a3Tp%tqAkDMd z(92&RH#spwY!{vlO-2WLJ|c$o5_QZ>BG7NsSlE?8R=+mvZ=Ki9v zK&?m}PwE+;l+ZTsN(m42` z4D6I{CkRRWq}cF4z0y|Je?ME)k zxORt4UE~vTT>ss4k|p8J#O=Fnq`RqpJ3I&( zCEz_Pee>$300a+nh_H)1wVN$f*uuUC94%_E>7^}YQ4-n{c|wuk zv{kv)&bm6j87ax1-cyNzK-wj|&Pq>AcuORz5sBMsQ zOoo!q6m8!v1z8G3$&h8_?&q(g@oFZsl{%BYK~&AU2se^yV9f$a%K$D66fmkSHS6vz zDs8Lm$hKO3+z(=k^!@3p4ej3e#BL6kqP&)XXUyZgYg_#GM~tk4JR}`q9P3(4Fypv1 zX6-VM@*_7u8M|_`}ls@9TzP8 z*!@c8MKZEfEH0m|Q5b9f{tR^|+fSmRc~h4T?Mo;UiTj8`NYZHPodJs0Z8U<&Yu6fG zZn>Rj__s=do5c!~ywt-nLae1au-OYT5HdQCyIAx@HC!~t2;&CKf&u}w%(t))ci@413uSdn%RMvx@Uui}M5v-S37c^Pone!W!-(E|;)rhDh_tneL=|^O^&Q zbC)SVwH7(q#1+OETL#i~k1g4X9MN)2X(|5z(Y5s#yJPdY&&IKp0b?r-`;lk$ge8!LvczM)Cb z@U88}HUFHcw0#*JN3T>mgqAYdkp~~Aqjq1tQS<)ms?G~2NddH^@ak1T-k35eZijTl z{-~iBE*_Pu8(BU+l$b7=@kN8K|1b+_!|!l~YxBnY%wGD@XssM_4d9~_w#)SGRKy${ zFe?FuFnL0Dy9}dmgXw1JfTph@OHmjCbuX5SR+1r_xkt}=!pb_kyBGCfU`zlN5cI`F z1s<-b7^&`xV?->jDKkX*7&<{QmGt`dG)`Wwx+8E~H*UpU_gSMEv|FTIhKvDU-ThmdZOxC$IGvf>8DN;mBg0{xL8!BzEQ1 zBv|g~91E<9m#k)7_AK}>5za3kJGJ39>F~q*4}ol6u+Kw5bfaPPiM+W7>H^X9Yw7)5 zsl+e}f^?T4V)H#rS?v^Fzpbl?&x0@5Iz58$@MAN2%HEysdT^ODaw=;+p)$$8?dy*R zTPYYCUap}vv~?-r{Oqbe*(1(yXprbq4$!?OH6igr8Ag2LrfqQ}SA<4P@1pJ9sN9GN zd$VfJ|v|g)|AeZo_hTwGH1QAIV_s6lB+1?hjuDath`P;A#<+V!_Id&q0>_@ z?O8+Z|9auSNCbPDh<+&xO#hVZ2-31TO>A#bUI*^Dd_i;!4I5Ttz*g@$8g~-U$KIVC zX+~y^{Vndfo#mop2X464uykHF(e78WTbq7_*_Nr_cU z@l^{`hB0u#UT$YSt>S-eBHv2V7ytUqjqq1>l?#^>BW(hbio0Dmx=Ar=4RnEme~_ z?B+I{9KplEc@9}TIlcIpjXbKKTj)V|24p5{99q#3)xL>G&Gv>W@uyDf!x=MfAEghL zAgJ*btrHDkNepeFvzPg~^ks8VA$vclcBlGY6XOj^-L8arTqBLR5PfvqR889KeC_L4 zUO*5ErIh&uKpPuga_f0LJRIftOfbJ`N;k8fZFpm+gBLgjM%&(}{Z<^Bemxa2J zeOHpVD#6AYCnZ%IIy4piDV4Ccr-yNcBAg~UIeoGzr`z?VE`A2%I)Tso#8kd{6>$SG z50K*g>J~E$Za=%TXQRB>)AOc&(EQI{urk1Z)BW_t+EZx(Do~{E6;XH#H(}MjiI|e7 zT|Y$1R7*?Cq}a+wN|NF<@iS}|;?sX5%<>B(; z%uKD<*)~r5aVTyuF{7=6gVe#{a*G)I3G{r}8;6kpdWioX!*CYO5wxQ-x2gHtNfklu z?2vobL6_`(l825Qf24hrz`%808LK<~IT&_uabY2{1?R&=WEwZueLG=TVxI5|{10VA ze@e}NC-*%ju*T=dN6}Alzm5%_M2C}NnOf6jf|dM`p8(8|D4O+Xsp)Kpp~AP1JqK)) zdURARb&!7kQNQhWEu4gU17LrV)Q7zGcKWd`;^`~KF0p$;grOh$`%1oJhTlE>G;Cg{ z`paGhLdPit4py%zjr!|AUMruiRcTdjvK`c->8W51MK1YI&eK@3w>7BjEHL*5qazX9IA1<0G~oAKDi*Xmih-vkH*_YKqw zj&k9?uZHeY@QJY%OR5>?;i$AA_)vlKX%pbq5|v*;TAYygjr$Bcf-M;*_K4aY`&SwK z*G7$aLN@zY_+q^JRchYp!<)uaL+ItHvl|QpXSzcjX-37p>;dv5mK&XRE$a6fX zCQg@WJgda0)dE0n_U2{ZK@v7mwh;u z3BhOSZ-hMuaNL?5j2UQu*`^V7NFJUOv+Dk|NU7v!J_2>(=otE{_I}zUT5_@oHh*-+gE7EtwKJu_gyl zS}6!L78rOc5PUy8f=&$s9*9s>M~e*g4B{0>DPKnkxbEJJ6{AetL4Dxy$bHzi0Ek1j zjGlG1cpOOzz34&@sR>AX21(mrBKIvXoS59lR6Ixee1q42zpCGTe?G4TXW)PJtsGoq z`Wd5@<%eIxo)H1aHS*q*$SjOin()!lQBUp@4|JJ8zeXzx7zp}7q3nW0qADf7?`qcu ziDW4#Q{h*Ux+T<#6Y$FwxQ-Q8u`#!V0k%G_wPK-Y{sDQ%0gY4Xx8i+Ue>p zE;YHRNb4s}1TVU+w0ePtxz0)ZByp3EyHwB1N^hK6X}q$4ipdz5w`UR|GeHPTL%d*PtBGLs7ulh}c*ds7;>DBGsM!si+^g3vdR7NVmOCEr zI^v~6BAa&eqjLaNBd&i*nFl^-8uzm|?dQb@A>c4?F1E>Uj&@z^@bmEv1kq>LHHO0{ z$oe+Ep8Kd1EC9)PLD*{q|H6ew9Sy+7GD0M!bH~8Ekhlp$B5*tj8^*>fex4liBW-3t zP86D4-W_=|7_=`2Ljdud7W4cSAeaj&1c{qeIRg)OS5i)d=00#C+RPESBR`+Y{-{275=!*sP}YVD z7`aONQ6zcRPXwp&AtFD(9g`XP_Q4$=ti=OLoafV@ZVg0zb=b>T{@)y$C)5S{#IkOK zf=%x)#sbVoikhQxpXRbX^0^G}Q2x6FB8I6+*N-GBxpe!ihw73XZ+ib}$2ayXtr?7z zy;1#ZFJjY`(VOPO1!Oz}RL01ROy|Rqpywg!ZZkCG%lAW(1mP0W1jEVh0Wy#!&y841 zQj<7JpX(SGQ%Z?TTJlRv3e=N@m&WBCKfSjv)=#L=mAKSmZ%SDso_aW&-==cn{%+Bq zZs8%8+b)*dXBB@rL+PKP9bMLodkY5pq+9}yuKZaDbJyYkBtOStl-Z@}BqpK1`|?4HLIx2Zl>!D2Str};%XsEc>2~0#-`Q&J_Lw?K-V)I#_^UZLa;Y zZUonSg|%J3n62Q&;}wnYjf_CM&0M))6mjw|YyO8px)29kOu@Qhx5PkO20WiXdjb5_ zT7QpvcBsnrJ|vi3pkU2CR`A4A^<)@bgmKt(hau5ZyL%Deep%X>B++!K&Bwzz7yCAn zGLS#@I-UDIbFMY)S@>F9tYA?AcmS?aDy(neSfh+7H#Hrh2MdVga27pI6aUF`sH8) zic2{SdaAqfzo^)Mdrb0TkZaqB!n7FL8EVl|$=wl=g6130R57b>)VrM0L-6qx`42c+ z-wz~Q<@g+aHW><|WD~EpXSbglh+sul=UQs(sYIhjSde*1yd@rn>jl5Fp0J5J2(-G{ zo{C&(c4Ih&zP4p0I9*erPNHi{$IM)iVwy*3kzlT_3H668H8a7o5fZ+Cavo8x$oo<| zOa%a#=P$?ik8N-Ra#ap*IZMAJ{-T1vM~PgUEo>CqSy_<*ipzo~g_02Ln28=-!)}~F zmV`v(l=Jfr@MJJN7(qQ7nY}9BKv&-o(>LfjhN?HXti*cI?WZXf-%#SJ^LZ{AyL^GT z}tfSwnLa?DMeZ`2gm3OYQN-Ef4oLb}Zekmn|R z59;QgSa|w;WK%otS~`5Dl5HYNSQ^$2Li*%7Pd3U@VE)`F3{1gWW z3s(agFhW7y*FLLv#9=DXO_fndf-}Z9r(05!CP{xiAAcOGcVlosh*1Mm$M=g|iaemg zN!jk7#H^F_G7S+|&@5sBM2~%V7RRDXz)x6C2+0wjQ%3rJSIhO@dF!!m6x%uy9g9~5 znoEsrWo(LJO@yNe4&$_$)pMPc09w+q7y*r)A9jQvy>=1nFh85>aiGdvzyDC)3`~j^ znd)ja;1SU2lfjffmjS}{6uMsNYVWg#j)#K;nxn2zgp5Lh)NsWia@0$V0o|UBa!DRr z^LYRJc8?{eex-n0HE3fBgiM3`b)22nzjNl5o}Bsi$WQNnnLcYU{5LVAT-v+il=ZKL zLWFD^m#4!l{VdT(7$}xK{lwtoG7a9{9cc*G-3ZES2>4x4|ZrxW<=89UDA zhodiSaQN-X8Q`xN)F$Ex+7_1Uw`FFuG=eE8$^h5e2iSGu1zt*=B%=cx{j{uUh4 z|Mg#e;La-$xl*uJDvh9eK_aXvp|56LNh`h1%(MLYI6}Do5|WR+g!I%n`DFjKHc}$~ zWULy|*>~E564O)rq-0%xtER=&=#%;OM)|9{D5)k#+;Ams3n|`cftqyo8c!X=46A<1 zwIB&ZK_!6`Xf=Nw{#zhCVJq6vD?l`qpO1eDd7a|K#m?&qR9*t^I5pM>MZt?1UYG56 z31U5Q$YTWU`7K7LvII*v2CZLhSDGskU(F#yYB+PmRFq9BNt_2+ z3`&2#$z_LJhO!Ip+t#S@NTb>gN-lu9Bvvh{7CaNhgF-s$3{li~{QM zpkE~aNsobVDV~Dhx#C&rU)6Od8nkf~lm3)r^tvMzp20ra%c5XZlX!XC5T}ALV(6M@ zMzvLmLw#gy``r`-GCkt-OFN$m+W!HYU7N0a5vdl|dSnGir8fxFRO{NxyM46vorotRO_2EM( z`QUqm`Vn;xRmi)yC*;$gxrB=8;zq)YZ%@VN4^jT6x}jo+*oVV1~9b+2%g<+V6GtEhWn_5iy@+NV`PUC)4% z_s-Jg$4#jZ6?rd&8DutJ+Fk4MQliQx#IHLW7EXotg>U7tchMR}v!1HZ8Rs|V;m7=H z2cM=p!vKNfQa+bjr~lhjZ{fco#mcmqjWZ-tK#)f^b*zc?f`=s2{XDf7l0hw=@CAgU zd92h(7A3Z{qSVj{h#C}Ff~vloU17>tca;{tFQn)PJpX4mJ4eV_*Eb$Qv?z#&Rffg< z{df3;KVW=A^Xo#;$oqX&jyrpUaS3Rhd5HTZg8;3kLk*diH}MABB~F^dbhJm@x|E8L ztqSGHc8pVXQfy<8jRObNw3MgwH`A2I1AAI-(Q&RwQ=?YMD#b#mtr-!WYmXUuO|o{i zK8yqhJQ=EkK>sV9Ywr?gs`q7%S1<+lU)@|>*g+?+xWUI4 zia^23%dhfOzpdz>()DZ7$?-5OIt{)YRL9LC!};8?xNDgJU;)O!0!W*PvCNqrEG#T$ zc#9v=<2o_b7s>zr8NYo?C=8*OeWF5nc`cHanHguWv&$(T+TxMG$;8Cu>3(&%Jj`K0 zPnaS_J$^+*dGtTa{~--y!o&Iqv$jFv?;i5&`4xf0kS5J;8F5{%u|OeK63C++kDG*NwA9v6TRV&ms^Ly6 zYpZ+psjkkgKaMC7)5RC6r&1ID*L27~jADq79??_*JQ>L4=1e?Cz((!n;w)Bt+rm8u zJI?i`;Qhz%_dOU&s7mtkc4G+q7Dm9G`Y#^hZwvbGVTzd`%$rZYd}Y$DH(a~FIpd{N z)~fL-VB4YxzCnCE1YEvSYabsUDJj^M)viH73H?JWAFi2PBajvUS8i*ZG5^agwfvv- z_OZV_A8aK}nxovsnk!pu1ZDLvCkRn4@D7J!VC$jM_z6arA##38m^LPY2)qe~TwuQp zrgK?jEUO&8Fu)nRRH3utxi?jm$X69gETJog>xvmEBb%>Re7XLkXd6_O|}7*(a6{fpo`Lc$;8qMK|jz~qQ~l3sXGLV^C3 zXr2ZNMk>n7nP;UonZvw5@3%*7@ax}DE5JgYKso7o^VNoONgJl3Mhq+NXY4gM&U=m^ z9?@L=kj}Klrx{r$z}l!D#-Ft%t&C?|h!yO;-LNfIR@~Z+3A5nNZugR=idLxd!hX7) z`R^6h2p6P)+jfclcjEZf{gUkypZdxA_CI!uRRwxosP{GF>c9lyS|t5TL3BWEdvz{2 z?Ri^k{vZeMi3RF9`iD@)80oo3WB?Rs`4Z8_NIMCUGmHXd8a$s5^}lmP49}?hsAjwu zfZMF$ZP; zx?utg%9Hwz2^P`QT`zx)9542TrRi^nDT%6Q5 z9|{1}@L$9oXo8SH!fUe6lGEq18S^0y=8t?U04q-DpI)!mXipp^hMQG~+@MGZ5aMMO z65M{eH*o|21-fVOUVQ>ahbw_ySgEl4o_#_usFTeJ!MJDo8$#z=ql z7&;ND5=cax4Z}3WYyPlxhS=zE0VxfYI`ik6UT>gJE4rQbRM6Gr0ko6|q+ZZODv^FL zGEL>Xy!VQgE!+maE{3I%eL$at0^4bos8MCOOr~E1=0CCn-XKFQu!zeeLJ>O~qlYZG zuZmH%)y%7@1{O;j-jRw-?(%4R|HNBUC_AVV-u+5Z6p88OuMX8;7U?e^MP8#@ejI+K z^Ls${%cB10_uZ6$7<8Gt3hUQ2;BP|q@3;P*4CJ+MZMAHu{(ersd^;luBu?B`do-Ua z2mUkOKYT1z0`4__)V%(b^2SFmy$K5z7Cb%Kt1f@Ivo|p00t$ z-Rm@ef6U)J94HS05>;0Uw$Fd_bN>3oKgR*zQod_x5L*253aC~4|9trWE%aZ7;Qto- z7dihwiT?kG>hOWgY^ik^Z-RXN;rxW@&z1V5a z%NstzaI-Bjlc6$pKE3j`iOW{Ldxf5hRl`0OF?1OSU5CbZ}STYbai4`1br0Nt!EF{Q-IJ{6mM-r#_;rDy{KW$Pe z1EklRigIht0|tuANb7ugHrNS0WS8`1EM8WXq7N45&@h}syX^FIU^?oA@h^?Wj<_%( z{E-94d~5g=tiAOLwoVH_;lvdvrRo54=!O)edixnqr;IM0TSZ#-b zMRrtFdYsOjcG9nM3aZ}9W%yDkxz(LWZ{PaM;r*{2F((J(-WiW$*LcFay15AgmwN{X z2jR#>#<*$exK~+`SlK#Uu-5qbMMa?y!x|qnIXOAEM71Dox%zh(G4;p@Atsz~OSrq| zh=_>bEx{b6VV%1d47@^+KnJbt6`EXMwqxGKg?Od-CQ8-|Xr9;IB?J;Kcu)E0hT+qT zkiuJ@4;x9iZe%fchz?(YPVt%_wM8ij!qrm^ob+CnyyHbyTFnmXGIv3%rCrPs^l?90 zAtF4!2y+N{W%OBEu}72|t;TYa+@9SvkKBshKnB+|Cj6fy7ZMT#nHoL}CjdD3KpS`? z`S?V^{{G^3@L||!^LW}8s-rSAZ!z^jJ^2~7Z}NI1F)+@F{mxbN%_2S<83)Rw$Fj=8 z#Ym>G^(#GlRL)-cpj9EMTRV30K_eKNIz{9r>hsn*a2Nfx$^_QJL7pOJhV>n?;$9%co@; z3>lvz-WD0L{Z8+xdOHf`7K$81$6lZT9etXwK% z2EPY0ByOf(t`iTFJV^!G+R`Z2PSBtOHqE!OxBN23SW&>0zem&^e6vsgVtT=_lu3Ud zr}y3E0dV2o>e`nx;L8=GIE^V2s(~36y3b45nb8d7HT{dCqhqD(6Cyh?@htu~Yxghx zo9+x56A!9Li;REc3*aI)^h{@web$HBQ;V0%7c@v~ov>K(2NU=pE(y5pp&>r-?eMnO zL=T*N!OrorOsiT2lJBAyK+pZ?DjU~u6KW-8v7gs3IIc?JI(hFU^&GV!)2xCQt5Ni+i^k#t!E@t)JA_6yuADgh2`WjKV&+5R2-MN(oQfxm6pYr1G3ffypl36Mj1@E za@u%DWlc>@a*I$RKshLTZprJ!>(@3dPw5T7to}|1%0od0!yI1AA1!;V#=f@5c7mZW zL5u*gVVn2Su5z$8{ZybW=(Fe?pd?;JBH&U!Lttr5C{xG#{BC>r1ZD_22pty;u+Vxq zYgW8&Nky}Td>1EJ7P5fKyu&Hr1^sR`63NlGt#Q;6PSIf-4qezQ^t}!wpmWR7<-eO* zvuLIG>cq(*-BNUB$kWqYo}_sib8db|jgT{I9I2!yRh~W@<9A5t_EEJ{Tz%o5r46|? zoyKZMcqCZ`C+W&>xy{I4C7~2iFX|D;v#B`#70p*8%Z)PKkEElI?^;0-4wUkDZY+>w zLjvPAGN7#5EjOo-v0)dPzC-a~;iN(hHV>AZ`VP}|{!tpSEZmL*!mdYTtNe9~C24Vy zIuCc_E-ZU$4N;}-d6WbE%Wb{ zdTD^ms%iyu_eU)*3)A@EpMNV~=?D;Z!0J8Xl1-oF%!HPH|M#Iw`T{gS>JaY@!t9eL z3UkUp_#6}Ac-}W}pPI3NmBTv=>;8%o*o&2$2}ZH+9H@y~VKr8gtM7e7QN3dxAdw7M z65uQIz=ASpxNR?~q-01*DF_qbp&egApVO~)qH(4lvLCroi#LjxfXAANM30pW3rn8stx)KKG&P8RW4^Y*r za+|+3_tjVW7UK#p8(PQzt!?Qv(37OHP(iERHn|N3Ye%oBg>>s$*p)d)hM=TF39s zE6##ssO^+vYYXlLF{>;b*C*BS9=$>GlnT}uKMvnAwr)|6>~MfBXHNaA8Pf^djJ(PR zHZk)x|4D5kK*ogiin!+(;%t~0Wfz#L%}Rqa_h=_k_pNYb$B{V9QfCxI!I(+P&rS6E zn(XO$To{5@1)2G45k9fIFywF5gmg~aQS7j~v8*y%z(m7!p;K1>^A} ziT()b=f*d)>{f_+?pDZQ*eA66MPUoiLkzLuw;8}-z&@;4?+2Q_^9tkZ{-uXz7UEAUYbPHW~QpO`dW?2NNr}Uz^WiH z#|CHlX-t^~Mf~7^*DG?O9NG)?vPcv@t{&j+UcgxXvwi z0bdz!H?*?o=zi3bYkNbfIJW_T!*YOcq)NwjiD7WLX|p*~NNyyYl6eowB>Hzc7|>E` zvW63-IN=LN1l`6~GZna3(GPF%xpff`W`eJwf{XFK>epaRrT;)0Pxr+(h$3?FXP1Wr z^9-uU2gb7JK2W|XkcF4W1vd)0(8c|Lw_pUf)}Mu7u^(Wi1iW7YlFpfiY-Yl17~6+T zS}lZem-w#X#PqChvr_s@5D=;ax_Hee)=tz5W2m~SNW_mkqi{3R$mdHX_QPM`{_kxJ z5Y`4gD}aSQ6@oP5;iZNhDhuxN>e<-sO3XBq%&<|`VgFI7x0QE6RmaF$^PcNtUGV$l z4+0;LugSZcdKrX0s; zG3foBQQ01$OTCGZy|mhl6m1N&)lJAtdnYeY$lSz-HW1EU#dl%EuF$D7l)6a1Lb~So zl8e@fXdn&JHH;-AeS$>B3=B`_K`5IqI*7~VA{=#$$#D+vay50VUZFP**Wd0dR~-7% z^M&I`1i1+2n<3#G5zISzmfR)weEq*mhKsyK8`!`CwRRhMRg#B6QiK)a9^ytmE6Zb2 zHv=I9ruN>fN>6k1r4X$1N;AB;SwRqp`3>E3)Fg7$kLKRJ1;te*%qm+L+I5D}c%L)6 zof%BK$rDg4ycuSS9-3mwMS}aV%#}Y#+?hl2Y6;P}s0u!Z5q(!}Yo@HuWo(?g#T&z* z3hgWa!Id;1{p@{Je0l^W0+{VMf^3Qj?isZGmSYbp-p8T4+dyoM=vQ+F2j2@~savtn zPps?k(}H&S@`cNBN66yX!D7rrRq@DcA&acCp+@7hp){KN;{Gi-yP&)*znK@PwkH^| z>6Uq-i5Ss_eO3%-k*vP?*o3K(PR`?vU#%+tM#H;FlEN^QVp8CO1aP46pSB4tJ`dWN z{&abHIp#JmRT^2$C58npxnGhzGChIheNB!rBn9USWkR;oJ|QRwgb1FG@lN%=sw!K~ z*CDl5q_KXeeKEfE{a)!=+acPPc?Dz%0r(u`1Uk-H+m7w6AtI#@r)eF&04^QvY)Gs9 z6hKS-v>eKeWNRA2L#`WGYlQA~qqfG833l!shp4o50>9)b-6nA!3qo9*r;vR2k+w)VB@zHuW$lRaK`-(T_WM_bqtkBku45$FunmDiysPvZzOwKHY!I}z9qE!1#p->#Fe0Mi6ARS&@r<{ohSvsZZ4$-+ zE!nn{L7_QDX^Ac(CkY0`V8QUJ$QoaG{`O*+TYL%58Uc_;_lcBMK&cskcprT zoIr4qjJ;IW{7e1)pAI*8PlQzRu_^>D2f&3M0Kix5g4!1M1M;vngnKGz+*Q&MclqGN zg?c-J!<7N_jh~5$xRS}Vab(}p+V;)zgz1PJ*Je(5WXkbVVxs~8kO&Oh3BUtzmddEZ zCjBivyQD*;eA1PbGuH%~2gRf??pZzBSU6fbGK@{PH1s-7&w|^xPA~vf4F-VbF4tJ( zIa;(&$W#4t6O!AB@bcEvAs&ZZuw~aFqS?IZOajP#F8-^24HpL*Vz6z zUuDvdp(OCx2baBB5%S zuHyI)zj1aqwmZfads(6vzfyBS&G!CY7I6C5;IaBS$4=trc-VlBREIJWdnAICR|<`i zP6sss(RolOiBv^An(z&yEUGeVg1EDeGz4T&P~c`7LOv;6?{jg?Pp1zf0u#0!G)wI4 zJJ}gPB1d?3Z~23kM}@_wwyP`Gqq<8m67M30*O7rB_5^5mq&Ft36C+7j+Q($rh?8C~ zi0_um!X@6pVS>q(p`%35Xd(aepUxG@ZvaZCezZ&|>TF~P5sa`#nitcrPnxn;|WssBZi;wv0*k4@Nn`4!YT!QdKxKjaE>w5YgUBe0zoC{+Bi7q<}ZlGk$=O zIK9R{%0z2seMOlxUAK3K%H9A=sNuqE7>1H6UCvY>D?^Qhh8KhtH$539^f`Vy=dw06 z%O`&5?0lQMgO?+SEJBfY3o}dLiKwXc9N;!O@>%7FBRO&`!r^XRLoxu~$9wAafwk6| z?DWY>nc&{zNVnbI>spm9Ed}PsH`ird!lPD-CbE>{yR%hH?kF1Y7ChK#$aM#a!z6XC zEORI^-Ji4C7rEg!y`e&;5a_Pj0R7~g*0OY9??J(PbwzP>d{Pc!9ISeqB(?E`{9X+6 z?A?dNX$))=SJhaGyIL!M{Lh;IbpzsTpwas$LZ5JxkdCW2V;<-qm8lJBJ|>F5W+<%V z7FbM7DC-^Zj(Z`;T@V6I*dK4J;{EnHE4&oe<=j(}58Jwl4&S+dMFz=UB0`9HDmtnu zF*(Ak?M+O-iKI9H$CdE&ozC|Q&9bq)=0b|o?9fyml^`e3HF0u1(a0)9@VwV%TV z9K-r}5)(7sH%yj$nFaa-K-nYq+#C7rbn#5wi=Pj7%Z^L^t@_{Wl7t`6ge_dl>9lgy zkcbv{N2-eH+vFQe@=MqzUB~JhD>)aC+O5k^;hZ&_UJP;g-M0os!>-JzMi|^VKrp|mh z{$V~oL)hn?ffU>@=*yBL&-xUq)NYW~imum$Mko0EVp+lJj@4c=Qdn9#kHVIPdMh-Typ#Y+r0Z8zOpC zzN@F%g344qqVy2{c@m#Bx_Ud?Fs0+mA6w|In(L`G0w0 zG`n1Q)iv!N;if=1Hv*PmK}W$_ATcerDSI_c0^SHsXC_78)uP^VS8q<3Fl&7=AdA3W z_}SKxHVB7?9qQ^>K!E<+QOKn5!?f^)ca&%=K&5gE)Z+MVFJ_Fn79+ggeTluOO}n>( z-v0S*RuZ+3r>~oxYFN;e@%2*&Ohbm(V$Y&ljnOlKDs+gT9YD1c^BYblygIVD_T{H~ zG6DeybN+U8xzW2_;mzB5k2i@hN*gJsm z%nv#TC}(Qlm0%f4moQ|CO@3bow+&y&g#s_0Ic^m(xp&@4c+)|F_RzgldO#b`&ac&{ zd&lbO0!~>f+i04;<`w+P%l*FT@3iXEKI@8q3&LrAr~RnvX^ey6tTSsbj-em9@=Y9T zoza3fT0*h9l?ReqGwDVjR|9@q+*YQ-#*7dMYN&*_HNVDbU%T-23`A>2dQY-v570WL z-lOqo7%$L^);nr`3~#dkGSzckqu!!v$Ro!fMgccbxyh)2lB*3v^`bj>+r*3_sE}EV za|i=bFN5-HTobBIiRd*&ePWZswNoJboBN@P`zTv%aucGu;X{Y~5)COd2#PKKcTLhw ziA=lfj*9GYhfG3zIS5ShMR-MoGOlqPH!BNQ8F`wUbBe=e-81XrCFly>sAxH_GzwUg zrn!>BRj5(zHlSCdzxawod~&tW>BCE!Y99C9X5Vp6D1~AZ0(U$tcK88>WGppVapIr8 zD4-vla;JNf=?W|--cm7c+$9U&Bv%zhQq-%4xdgo#j)O9de1^Y`I^6j@0ZruFM?1Ds zW~#K2+Nk=BRNmKgCGP3~Tp|RW{Z{L}X6>j|kETHGs~!P3Hi`a;s1&ngsJW4c5R4Wpf_KlFwF6(A*&Ir$debV+fd3B7yeq5NFaodp}-UJUVC+5T;aM3 z9hM7i@$x+&Y)FU4_#A8LhdPk~xqz@AF)^;#= z+p2m-WQES(9+vK{wPO6M+HF(S(1`6)*@Fpw@gA8X3u^(Rfieq9>5a`p&uTHEk6-`C zdjDIW*aiiDi}@M}j-iaBw>g5A`DlvXR`pZ98Alj`WMOr#S6x^O*rdwpH%g%b>p}t5xk*`Tyv8%b+;BE@~4?aCdi?;O;Js zySq#9U?Dg(PUG(G?h@Rs3GPmC0tqC@G~d)rP1U^55B_#hclWvX*=w(B?HT#CR*f~m ze(A`4?Hu2g?rm%3@*y4XwpNel{ZFs^gtM-Y#iJ>}oA*zKd~IU4lKu5z22fuiXkSjl zmf{@b8n4_RA7-X*N*DO>4thzJs#xy`)c2<(1A~k?E;W#ug+fdq8<%3*v-%%uKfmnb z_sbvC@pVq<&-v+WwY#1F+j$F|aUN!>_x;_RF9J2VoHnw&ET6F^w3P1o)r7+jiosm! zVu<{;q3y!t^)7v7=N&#+J{|uPq*cG?3wcF;*{Q7ECio9;R8 z4a$w#8O-;BE1hV$Ei$0l_MRhFI*?;xTZM+;c!&IDmcpV;7GtgscSi@nUvq(8#n>Y1 zmc*?>?zq`V>0XNZDK+k18YOmlgG2ch0+qeJZ(unOM~Ccpv`)8i-eNT_yb3&}aN-MW zC+!7?L!Q^GOOIA(*w?HVv+}n;kmtAzh5(AIs0f9Y9QTMCKkwexR93nbTe@UGML{Zl zY1d~5>v0OmB&469EyLyd1oEGzHBU7!{?ps#vD^xDZu6^kGfO4if~XSHPn6LBre*M%vnea}Dd%dpyU52F$qh7NG88D%WIfhl^S-?x(@7HaYaa)X>wsd4sxjB10No zzAw%%!gRNhPl@jZ5+JqxnfR_VfBJ^?tJzcn(%lt(ogcy9t^L81sP7+fVq7W*!bBVm5w#>&yVDh<+ zIu`2r@)g6cicH*Q{rR8;)6NkxhjYehw|_XMN;TrPo<6+Kd4H3X+(dDIYdBJ5_q1Tdj=5E z9T(#mv=T|Yx*vWR{xBbouYkl%YsV0+_{M(#&O@1~x5K4lhS3+bXXK(3Ao|QxIIKGz zmO706L_MHLiCvVu-(0o-jG2alXeDq8e1*WhozoK!uNQN#PQoV|aAE6rtX2yi8!qL@ ziYn3L*lCf0koBrAp%&{c8YAYEjD@b|jbeX`#TMnNuqb~VzVCf4s5!;(h4+F=YHGq~ zCft!4QFlj?=(noXYU~r+$4tPeA6l#Uclf@VV#JU$c7J>Z*`qu)C$e<=R+GXoTAnT& z{i4it{~Ez?s)_bHiBt*&ZzFMIt92HftRhJl3luBXi-*`BkGlW}R=P!9tmRKYw@k!p zyMZ}AkI+t3K%(lmm`pPPz%3^O)KCK& z-e_W)K}VF$-Ov7)3IO>>4N0DKZ(X{aGbz|=cgnaX2tYqWT9Jiv2wR;yiEq&FXz29E z7UUkdI1m=vAWTJgOmo>|i>*6ya=D65sO_?#0o)XKlnDZ1Ok!X9K#1$pG-Nm-OZqLx zO=t^QfdX?Ag~V=XBB5XNfz`ffpznX|kYi~?5NI?@PU2?Yf$is@`pNP!{X#X_m@a9V5aP5Jgh=&k$naTd-3Xjc-U)A352dwvxeyHJOS#ShXS2d}BZ z%G-8`5hD3u0p$TnSwtl5q6&f}`VJL-ovdgTMKcTo!x)EJl{%^ik}PGR88-Ye?jcq! zv^~M)=gpS7z7?n|x?iF;NmW%{_Er7R&d0?`QZf=BxDZGNw7XCc3VIdH$DCJ$%V3ue zw?15a*!xoMI0tJ3q&DbV=8`gO2_?H$&e-(!SM#GG5YXIoDqwfP@StTW=Z~MTiuSJ_ zuNO}U$O&b89;ko)S?@<-KdzAQc2jH{ImzJ`O1x=i4}&9+8_(jXg#Z5+4gSxy!z_W4 z;v=UU)&flnMxuC}T|x!stjs4rb&@b^aeZ|rlyK@*1wOqA)~{HJ5K3MTi`$>Pr!k4yrV(y&vBcq~a)S=_Xrw8M7BRz?fbu~4&l*?ZL-j@%#IspFqR*@mnzP9h3`ETX(uw;WVk#;a`e?ZVrBrlVBAl#M=0Qi);M{Wr-H`m1}zFrM6{q!^mkL zUB2MC9bMw}%6kWu2vcEy-BU|gr~{;mXWWTH=vYc=5x2Nzw@w+lwy~I zqI`mp|45?UsZceEGMwJrf`e5EBxFWq*&=TFO7`PCX_mueA}`ztu6A`6CSHb%VasawQd>9f)H;02F3=!IIM=#3{4$&izsDg5rRj_#8&r}FlqUEVm2}H`_>AMmWY4s7y^=I+ zE6C~Gu5voQ98O)SSu;kJ^%;$tfjWN9w!b*hg|vlDst^pRnP(dV0_)x8+UdT)@Y_{PTZkhU7AKuKp#d zX9d(X#-<%_hu!)6A;zsJ^a#o&NVpSZ_gqV*PDmJLd=#3si#cFBc=^7muXpR14O=zt zGvH5W&6S4GeY6M8dBc#MTa3FbwG6RbI%Yz>^ki*Bes%hMT~+&E72Oe2e%_K07qJ#+ zTK!xcc|lLBq?5nkF=4zF6a)$yamM?(CivJ||FyOPa~%sXXd$Olh81Hx_};#;`VnB; zom(E_UU)HbvJpvX@5jPX6(bH}X)GAFg3|XJ@b2}17H1&iHFm~14d^gUAh|L(P6UJWI)<(80#QH<8CDN9T$R?QG+w1Z){_Q{iJv5*7;SWVtIoov zlngBt8&vcK`W}GtAtuan066VXgky-_Or9P(ri*5^MMF_o@2o=8YC23GakvrJ&``KE z5;y>%O4sXL(G1`4wklSWD9pwIh(uQAM~y}-SFqd09Y1Pa9)jS7#iK_sUlf7M-@!8v3hSDu`3UQ=3MTAgKY*%J?v- zXpUE0$(2V?wyT5fb;g9g_!KsFfJRR`BekZI%;+j^EuOEy8=c7(#a{~zKr+rH1EJH= zjJz%?<`hz!3A#T~nFm__qZl^=IM&7J8d--RiL@ zj_X#A@UxfL84X-~Y2C7)dyq=*9Uf-4;E^lPxhOx#fR;8(j{y>MB&z-qRsIR7d`T0wYKYlUNi@f`A}fWOum|ZyU*d&XwhoMxephF5l!5iwa&N2G zJ@;^>>dZBb&XC;99gp@>R?-ppAF!gUe&8&wfg^8 zJ=_7&9sfUB01!i>WW@Xey8fz5AtW z{O6?#&ArrcTVn-}@@DA4iXqAk#LBWpCTP?y1p2prZ6GbPg~_U0XXBXi$)%k2RRhdX z^6Bp#8XI6sc<9iKBu;wgWe2}(frz=5r1+-kC8VMn<|B_Vt}8hCtd_?+csg&JtWVzE z&$iAvR;xyicz>o$)Lf3ICan5nihgf6`iZ7uoK+GR!I9wsvG{=<}JbW@** z5F+KuaW5a$cqmyHAAtJh?rdeI>?p@@Bvn0-&}QiMt13JrBI(EKhsM?Swl|61GS!t^ zen&4q3z2%Oy70+Kv7-q9dp9i1Q9Jpts+{blgnElEc(>J#H&_^q-gSU;3EZVD>Sqz` z9p$N`>@CM7+#!x9KL0sZLZiJ{Z8F%g+*+5agq}QjksEJW{kx+#DbGxXI3(6ESgt4h zHDZXduRPBXmDcW2Txc8gXFoj>XHU3%zv0i&p|$1g3mhi)@3qR%ML^^K)Xwm|D$Ui7D0B$l+kG%2j6Ia4`me)BUjlxs<;AbD7b#N@ z8pdy=u{J+&F88`Gy!jzQj?H(MNzh;nIu_mJ$KQN)x1T2u5Tom1Uc5a8D%2m81Wm3D z!^A+Htop;?y2)qn!m`e`U=Z17*$#61Q~d0dOQ+NWd*>zt>v{omkX4G^I>1q^L;1sX z`Rd&^4*E{GZL^()RTJcAn{8rZbDt1~-ztlknJcsp`$8Eq&3+56#1J}><@wL6^9qI@ zu*O}P$>U1lz-X$C{Bydq^x&OUuvSOXfRY2--bXLdr&)eF05%gRPhI;`F3S!c#d zQa*+y{PIZ>JGn=GyQuJFjUU4b3lODEIShoY;h`uSK>@dZC#UEh-B81X>8!nm`$^jf ztEN~dU}v@TpiyOj*B_$)Jy;C(rSlV$-X@Z*&&M}HR=DDRFQ4IU+uthuA=^LhucMeL zkNrZ=HPfzQcT|ujc9zt3OaBrF6;HX{wXS%@2@_#&lVp@+6hml1M69DdI!ikGRn~j? z45g9IMW{1*yzc1NwgVTo5vR^_5vjg388&asY*-SLN2o?j{jl`vhm~47Y@X{RwbMXa zgZM<;?0BbZ(sZeZCzn$>I^p7ZpuO zRqY5Cc_!^lU6-$k>1z*RNI{r%pH*#WIp%8Y=*0ExaSmBTR;Wmr`-Fhk6ubHuF$4x>{^ z{Cz3zG4tAIPGGE4Y1MpZzUxFC@e_~}bz)oMw8e4m#U0TSwJmG^r_afYFZuu8A zO&5B69%AM7*e0g0A}E5!oLO*{r{#0Edk}F?e+~TBer3%0D8dDs=yl`*&Hhku7cfbz z0ln|t-km#gY+%LP4fNsss`~xQ8;E;^hYzL9CqgYZ^`R;!YoAEgeBI0scd9sjeb2IL zCRnk&w3KyHOZ@h6^hIfg>2|(B5f0o7f zQYZ)m4E8bIsp|=$d{(HCo|n~<AR3A+8H@ojcZ!3qWGUMshJj{EC>H6=O3{ zb&GerQ>Jc|irA?7-3RoI@xe==H^{XmnBsG__>|hdYF0&RwmS3Vv`dJ9PAx=P{H$sH zqq3*WYbXs1G)CEJ4DZZ=-^%zQg~Ss1yAcf{{p49ntR{iVrzR-sLHjHJA_NQf_GOjW z)YIX{Fl1+A_)G>G%Z!}p45bdL#kCe0!(#;TL7;WHw=<#dOyqpn>Z$q4qu>wfYF0~; zqo_?f=Ww(wYLHC1GFHl~?N(+S)^VQF|*r7|X!_GbCSgluKx(Pd2y=>Lp7 zDBI(P;T*dn&baHa;n7xk+HGJKzfx+eQ&v<688(iO9dQLMbI0b}FFl5jU2rrUJll}-&O8EA9hCpl0vg*vuEvM@D)HnHz|KA_Bm~ zxCd#-1d@HTY2_n8fd3WB`nnRziQ0>d8zSyvJ>2v=Jx}0p7i@$#gRELOR^<74`GKi; zCx<(pQ$qMQK7~7R3m+61v%y`iX=#B#w2$&)mh=AQr@Xk#|^5<*r$Y~Y&&kyWAuD8 zkL7}L=#ANvIA7Y+?O18Wc%1Bza}GEN+WA(M7P;QhXmtfwtBIDSJ3X?YRY__=A5by- zpYtV5Xv&x{MoF5GLROo{Q($%MwARRE4pkB59#JbW5n;$EQiPUxkaSL3`e25 z=FsVa)pkrsiFEYDC$FuL?^8M{@T{Wfds6&-P$u>IJf$p4PEx)B<-z>qxcn#uH=xL~ zbU8~#8;-w>32DLJVrx4Z$em6jYsUa3f}|T&!4nn1Nr+(GQW`BRMdf2}$MB;{k?@3? z5uI`(?9*rkeg<YNDM+f< za!s?^^gkq%^>J}aDu4hNE8QY*Yx6f64$cfZVvexbBripKkP<8x+PG}kCzIStm2ze4 zuRn=**p3wbC|(J+T3?)oGH{A1v=Cw4WY!1o98lwl*AS&=(;6FlUXA;X2}Ib?9Ih!N(i+Xae8MyC|C;AaHHpclwOZG z&_Ba*z7S5aH}S)jazRiRHP&N8n;9Dzow35(V3Cq^KqJ*K+^LKQ#8V=yz!6gag4+1IVBMs7j+dCXob zPV#wP)qfp)XpaRSOR28wC|!cO{d~-qa5ns{hDc$6-rTVz6va7;AFp*vlN-8~W&wgK zRt$enaOcEqZkwR<5G6bZyA6}TT*1Wa!2i`TZR?{UltS?P7zfL=Ou`YSj3N?4CEd<( zAVywfE2Et;$z#wS$&oeI)rb*k?4XK^uDu7JTsmNo6{{l|l7xPYIi%~D1k{(Xy#rY0 z(!i1MZzggKPAD=?w?<3~1Y%}LtbE|2^sp&b=S7V=vXHQ=riuMTYyWw*+T@{1l;!8r zp5WIZ;%#xL+7f}m$vDW26;%e(l=^OzXVo>CzS1tg@NU@c|V z#Z@TPvB!#kNQ#=L*^~OALD$L}s&H&e5i$W~eeOg}_12Zs<;sinXi!LM-^XgUDEq6T zsqX9B>kFRdE0QQ|Rv zk+!u$I1S+}cew15sMX|Vrd-_Ab2a0~q~MYsUSv|7-|%VxUBz1Zs(%)&(9sSs+sgLv zok0VC4LuRwd;au1_{rxU`?qX*Sffhe%j6_#WKdhC&s8F1V&VI%DqqC%SvUqgDV8{L zNl2MQSc^v9Blx(aPIlKUcYV)94Ra zc-0(h7DYT`e<%TnpoWDSO2MJInV4X&Jxmp(uswqoOqg3$Gz0Ox=oI~zZYX0$$gUs{ zPls3=)mV^9IUhz%J|Tb_TZxpb7TaO6SXAUc^++1NKmn2(Ws8DQL_B`HD#LKlS>4sM z`~wN8Om$5`ejTPoS}59^njy2)N*!@XidAvew~Z3WfrKDV!05oub5~=zNFsP(C#J=- zsu+~ty~q2$dlQ$#h_lpQU;p&)L{2#E|K#r7#nO(ah52qiY0c_EgFRgSeG;i0a~F^z zUlD5nXNwTUpD;*DrFSA_mh?SZ_pFR5%*0lK&4A8~ny*YQYM{E%gkL!s1ItDSY2Wag zfTS!<{NgVYC0*t;(t5@ExXaTRKFrj4$cZeTZ8BTN$4VYP3;|=NMsUad+`A*L)h1pS zuE~!j*){94RR|MjihnBHUdl&J4mwWCG3ewp1B<5YqMgU#jAp@^z9u&EFT8&kIJx!k z6b2j)7#a`!^qkK$rdb7qcS3dqu| zbEM*DE>whSya+5n+qqkVVoM^)HBs{jmaF}`{&jN&=@livDk1Q`xN32jR2-C%&%bDB zUu3012(Fw&&-MwoeDDa&&wyRyGtuq}CCG|knI+3b7%RZRcB9i~Tyalp&7KET|EPa9 z*aeA*6|w#+J}kp0B>j>IY!JO0vquA|)LMEg#*&H6&0h(yMMGgZ^QyehALVVsz(O$m zxA2V*)XHeY!L`H&%)U#Dy+LRW$Kaku0tC)XPW6vOU&v!1NoM7pUN6n4G5I6grO`bL zl0I-YLBH%1J+ql+5Glre+~$HXiKRxfOQ4TIf0%(s1K-IFlAo6oYE#ap3<9PY|F=ai zdH^v-9EJ{ff=~+wFU=pqM0pc#3U@#y)=2a$?Ox{0UBi$QQK`KsF3XS3noJZQmQRX2 zMwPqRdx7+TfndrgZdDswkz|%KPN7Y6U?Qm?3O>gXTPXK=_q+vwKxI^I_XX+J>pxYT zD!mFIQPOW6VEC)N5`&LfEv{%rzhJ)sGg0D`7QF48Yhm9*{ahxK?vEp- zmBP|3sQV>{xqX3nzi^{jxlW(G?NbP8x-Q@qQ3O7EW zg1)=x(`K>yjK-kfnGtZ0VS6a^lohuTw}FR;%JpAWq6;0%&KEJV_*|sM`%vWU%W>>6H`H z8~|!|PG$M-ZY{mF&x$jp__q#eMDhh58THoP%V~K#O{SmK?weU(Kv!|=DH+gviGRHa zl+hI4=mvMbc{)?IU~B(H`BHkWlL;H;#0cw>k;zczQ`PfwpD`v(m&{Kh_Px@8ceDL- zeqj;4rcq^Do1M=3{AT#%y*pl4J)v=#cILiK&(0q*u8$jqY4&h^kqqGi`%JkDZzA4r zc>gif-IbOTAAo}cX0n{Yg~MrM=~8kv<_D@eiEcZ9j~hv}xsnP;ejZaCH#+iCL$ zOLWr}x$)671X;SYM_o3t6_$o&YxY!8az7R_wT8WO*36b!`DbV<4m`2w{AiwVG}iey zVf1>VCnWc2NfP4b!qUcG{x?llH`Xn!VXY;@00ytKWy)_rm_AwJk0? zkOj;l{AJ{lEjIT!uN7g#V9!DW50pq|yiF<#(;uTkE!c1lPEluO9&7QAXt^nB`0>D~ z=bhVSIY}ns^oj1Sp+5(G3Iz=Vf$VOZ-%_~!HKCJ(LhY;Psm_oQ9F6StZgGwj zho1wggmY8#;{#_9BlEBtJ>-l)SE!;2=Mgq3Vv6NYdY?jQ;zdVGD<|z0YmX25U>*Ux ziCWcPRKR1_$=I2^@VjTOj-(_oS~e8-J$&rlG3C+1eQjVXLxdsee#;gP?v@pzo~OeU zd+Tk5taW3$NA_xA=UEV^Jx*Uq4$|<)3y>xgp@aO1{4V;Fua}^>qA*_*C($3p2NYa4|9OZa83yjXTZ)Rif z0j}>|br<%z)qbrIdRBE>6}(_+6@wMbo{h>VN50nppdW4G7lI=JIYxH?Nvr4QU9cf+ zvV9>6k`fEML_PON>Y?(j{A_DVeuL6(%@vQ&!hxJOjU4M>dH)zU%$PZo)fbq*wKTIS zq|C$n6XSB?QaNP*(^da}7zbtqlGqLs1Y{(*N<~5o7~doXDbIaYAPntK5J)^qOGdOY z>xMDE3PVvF?}$O>9z>)dbp+|JulV|IKQK1_O-Yhj zj-OMTceEri!f6F%8cFbVugwH#c$xsm!xkn}V^rzGO)wkKc<|G~ z)A^mtyc`nQ_P!7!el|PD4m=OK{kT1ew{I8O!u}8MFDq7XEVAyD)rhAq7y>kTanO9gvSr6qFOBU~eipu~D)_!pk#+OhfjRiG7GaP&b3l~d*^soIy3+G+ z!rNm)7&P05AU!00TCgR?xQ4xamP%%k9sE2Cb|lwuPy2m^oX8|I%p=i9Ll|Cz8|M&q zxxFL|_0&ba@}0Z1c4=Z2wD!~30%WVs%`5p>GFih}S87MAJ+s1|?y=LxSPT}Gy@@%a z2F-4oMaC+3O8>(%ii%%F83u!Ser-UygPW!9E3f*Pi3$|9cyiorb`M3p%AXLvBHsCM z3(q}z2(|TH@8*|ku3zngP#z-*fnlbiYt;e(RB&3*9F3*>1??wa;E!pRDhr(8>yt4} z$WDy+%?y{ASl`XjLtxnh@#2r7f`Ql!u1)-;LKYCN$^c zAr~G3Jncg9vYNxyQY{@okhH2On5iDW0FqG^t492CDkZZ2O9B}laB=6h#Ov{~Q4|(A z33}e6oY5`hVE4)x7B<2@WPQiRpcm2~xH(ebg=b|KoDy&DeysQVQ5~CjQktI8Lueib zQ+JKGiUC!dHdx`MIFWnhNpET@p227G)H7|(hP|^N!FZ*OP_O1&8Ql}XCcjMkCZc!{ z0BB+PkNc1~CI+j5nhb(W3b;~AT4vJ}O0H&& z^}f5YAJ;+ zpFT3Nu%X3ZoEnWLNvDk_rYaOkQ8n}TjlVngU&{D;D~Dt4K(XQ1&-&m4$1Fr{eNn1t z#c4V0uRu+Z6wHD0XTM~2=3;=+PmT2GyZ7Tv{`NTb^eua1^B5Z4XCT9n5@L6Ho$RV$ zHo1-7FZ}kh@@=MqZty5T979E+d|QF+yeXi0zC6x^maJCK*+KMJk@m-&E$%x%}Nrmjs9{RVAD`GpScS1wQ z@hN43F3xtMenu%X4LwHXsi7J|A{Cw;>g^ghHs&0l1p}}uS%?Ga?4Dc7MK0gDmoCzY zqsr)#kt@8cZrqMwy>8WnuuSHWBf?x0x^V!nA*VUOc$n^&+z`i}qGvsqZe}xt&7U7M zy7nCx^g)%04rfAw9OHkJtA$|pQp32lJhmITFBV$%T(+Ot$NyIRm>k@3!?LX@zr0Fe zKZFH?juvGm&hA*3jUT&$=nmMTDeOD9ZPBj!f6Dqn-fkkT;6vkUKm2%0Ygj0v@se2a zsQ^@P!0$RxCyf$}-si=jb<#!WWn{>lRNk5B;>jl%*=WFW{6H1y7?%7o=-4EDjg0^z z3d|C?4YlLu{pmUJh0ADRLsivdB!+o*JI~vE^VZ+#%JVr@2p*?Buz=4xX=35Y7qHI~ zIHhS$I<8z4s-Tl&&XoB9p{;~g!kvY!Y%g1K43nW1LD_*GLgdyB^9x4-x#vca0{P?* zhg5)6@F_h0I^T%MLeuCUh>a8-`#0ZlPwBvwB}sutkIVsoFT@&lsWcHtv`~cc*Rz8gC@M2|{XBbp$VSBXC{FBW%+S%u_6@tN?rG-uOEFoY{f&l_ z;C>#(juXr|OpHfj0_CjTpoca>sXnUI-BTcZ_nsV=Z|>x|OW3c{U$F?`4(M}l%6{#| z+H$O0$aFzJH!ujHaxz_qzTX?N)>Ha?+jIHSojq~V^yjxGR=ey}hJViGNmA>RPNk=c}G;tuA*Bxptafk-ha8d02*gQ2dS zU4ZeVoCv6vFUyIbOtFGXs_~vln=J~vV*rxcQ#DX~P9fRuI}okw>Ysu^YBQ`b-mx~f zv{T_9(p#coC-uIq4w2Etu|~7Dj3lyr%Dt5!d_gJ1ZUBW~MGR7^64NT)ACSNKWT(j1 z`r!CRMlI8wnNWV(K4u;v60CUO*d31NDK`T8Ld>6L&=u)vZy9Vo|w51YRXm zAh35^%4^cOh5A?=vMNRz$<|E--{vr?;y#xoBsf#J_c10EzD#@YZ@e$$V!tr^BP6UL zRjG|zQ3@tSF0_{%8aJGguW_U500-K%o12jTdNKKJm0>FzSuIn+qDT>aJlT5CEv_L|DP_FK1uO4HVfnGM@0J3)oL=)#oveS)#r%ATtX67y%fy-eFQUR+~>Qp`A z*89lV@d09tEEz})EfvWQ>=<;3Yygq*cPaKA9<<~os+(NBme3@s(AOQY0-_zHggEi~ z-l_jm=`<3W?lSJo^GG#B?IpLFNeMAqx3}j%-JxlpZ83a;h6YQaL0npDWLBVX*}bOp z-Wnsic!7&>*!salZgKWVtE;&vXZU#n^}3QAua=XhVM-}Ni7*LiTip}qFJ|XR0*HKT z%JmNRw0wtKp8(u{ZbU?S`x4`w>U5ZMX+s;R7Obxn;C{9&ng*O1e)d$4DZ}P;Hw?5% zbo1)S@sJUE7>P7}Ju>~>2G1e4#8msSEi@dJYp{_KVQ!6fC~PHv?x`vE=j_~{jhfb6~LfN!Ij9-duW)Lr`*NSE4w#>-S0*lq%e6LU+uui-IBTg$d3kE z9`TRkUIEfQ{T6~AgllnUzw}4+l_l~Ty08{s33V29x@KAF&OthC_RIzKXoqp-U663- z=pdA5g7;J}eVQ=8!72U^nD+nn#S&x6mxM?*;;ZAkrAlMDVU_V?v_6?IpxTB=9t=g< z6Z=(iWtB@;3BzDX5{o?9>y`=` z?n69FI1mVzPF37e-9R!8aWc@T$wQ4{DhLgU+ytYyc+&?shFcSA^3Y=M#~QUpTW2)! z*!k615gZ#(^@`fv1P$#k{jQzS6-ET-m=ws!%S=d%Axaqn6-;F-Ygh-ZUvLp#o<-? zzR7>wZlSUf{f8HHqCStDe|I|30!Ysrm_3=5_BKcvSm14C4>93O2ro9z?hY(bX6{921tatF~- z#5U>Zx#QxKdfpD}ZGLQOT;OGg{jj3dJe-jQ$<#9bqc_#{DFR?1^u6H0;P)0ftI$6|Qf32zo~_d1IW66yInOe#&JW?&sAA{ijOYyrjAGZT zGA8LSIwfFC-(RN#8Eb*$=gqbtT{=vz30_WT^KSC} z?SV-%s40|Rt~`vr(INJPbzgZuN(Hnlx_>8m3*^?h?BnhEgN#jB_>Gu!8ePhD9R7{fMHt zA}Vi*5{G-cs8P8FYvU}VaQT#*>Wm*zGQh>(?rD{4>iQO;88pcmLa&R^EsqJV4fk{0 zMoJh$5T8P=bP6134Qmr_H%ggr2pEX`^VfU-Q_lQAQPM|K>=&nZ?*p`#R);kyQ)?esFL>bBHaBu zl{WBIEjsV9bBSWeDH`jPR3@$+oyWM> z>@L!G22j}WMs!j=J=A~O!m@%kX-7UIX+2kIcJsMFqaQZl3mJ%Bojj{lxh@q3Ljbns z;f+vS2zJgtop_d;SJh;&@?{=8e`Nm)3Z!Bz;q7+Q7HL$^6GDX5mRlbivwpj5EQyB{ zDcNX8>`HZUkQH9__^w$SCq#(^h+7Ku@}1ec!eR=iTkxA3$ej~XDw}!Sl`N)%$l%ca z{qbA%6!SDX#29qYb) zVMX&nlD8^rRtmQ?Hkl}%Nlcn18mYA4Fdk4aEVr9{d7igU5I`5QaAB#V2;)<7O79M- zrE}ZvWm_ZvLK$gj{ixX$w#97Ja}{|Y8w=Wl%`$${3WLAZii@bl2IGAT?;QZANn)p^< zKzIbTzf^d+>p?IJuJ3GA!oMGE^nz^Z@-_QNBHK*;dC=wWN1k<0AO3TuIUa@4(hqd$ znf}S?nA|k^CTjd}bTgGaBhE4A{35WNBQNni$KTOcy0s1ycu4l_JF5li~TONYe!j#vmzdb)4ZF+06yZ!nXnrb#4?20j5;(tIa_z3s=_E%z5 znLj#W9nydJ=_K*Q<)i5AXk;5PIT76emj>n}t=VIIvJ-FsEZnt{y+t*Z><3X?v z{34wojT4hvH@ActfIgtbU6GP-Z1FAByYKV3{XLh6g?ZWKy1PRpMJSon9y94uye=j& zGjo<4%WzW(eQm^Q#pMlnQ?V(aVBsgp23I_+rQ;zVJ)IUo&9IeRpiHVJ=h@>p#r{*1 z(J~rou;xr;+ycK%E6`K4epIzVXqZ0r8M6<2qvgfMjO5#>3$T^~$~(~|QM@N(mgjlf#Ne>=$-mwe;bW!qQg{X9T&T1l{;+4 z59GwJVpPr7p&e-uT#6u8;}NC~=e9HCsBCbn*ICC`E9KH0C}gPkX17JLd6F~#A^Pwtm5a0UY+dhAn7Q*O3a`tU87U{9aAJbYZTe<4z+~HS?egh-Jc>imd z1uFxS)!i$$>r!`KJg?RNdDmkixn_o(x-N%cp$$`0k2vP&gyj;2;?i<;_PXqPP{2hI zs#FD_K|AyRVk?i}8ZD%T#K?-yW#3|OaicCs@tdYhOuKf0#|ZY{NHU8j_4pp7T>0zV zvv1hTsf8|`AO7QKNY4+0ts5NLAX`L8Y|cp;gV>XpuQP$lB1Ixq&k(y{rzxwDfo(t- zm=ar6Y7ekTLSco)lw8%SOW)!KWNO|;0j&$&az;J7uyczAW~ea)A%CcrgJ+W*+6c-J zxXE#Z9T5)62Z<^id!i82jm|ESTyHULz;$$gk))BIT!bUKjb8(X+W(Urg{)kj0r#=S zhJhdAwHa=sPgMMCPW&LdCJ5*iFEh~+T~(WYUsAqM>}l47W~CG}pE&@~BtGOAvpB6a ztU0Wt!jMOWo0#X~TXxP~N5Kfjl${P!7|M|ay;hPAT1AbdWD+lA>ey+==dgXy>O|6t zgXIuB()*4`Dx#P}E8w~lF6)t)wW&Urm^jhO3J4O>aT zdE-oJga3eOuyW|?51#(hss+w|w?mD{!*|OmJVX)&)885!)6i=0Lz!4-mMRMfM_7Ey z?4%jNUq_km1Z3p61~H%J86&yiL~}i|jvKLx8L6@ND!l~J(@#wq%P*tiSfyJS(?x}H zbxz$-dVfcOZY(YCE!j4TJpA>mtr!JnJT}P9XMB%5N3?T@hz%#QVH;v zOcv?Em8H|jmuY*lO|>9J2Exr^S>@c?0;HkP8e6ppXq9I1&RX;APOUN9@pn*ukFI(^ z^j!3SUD!64kM?H^hL?GKkRcUTmE(;|!QZQqyi#Z`A5K*gl1J2ed= zi4a+QEFHR%^z#dwX-8PmxPTZ|z&~P*WT}o8J84DkxQ0UbGcmXjISnGLsv?rxRC+n2 zx6OA`Y0g%p38~7Yn5nryQJLR4QaEe9TdWvfc$&YCQi%SdpZ2u!?eerKrJkx<{NFbk zam-b9xHQW2I22_7bufN84gF85x)V*?{&A|zF zsFG9@)h>oF^e+7wL=G>a?jrg&2G__yjtyqy^eZ7T1S5SPzVl=iK{`-a!XggjcB-Rq zbrY{-6|r<&+GHRo77ZA(23Yg z%ynQrcjbk~w9GY2y8hSz6FixO4h0Lx()N|klG^S(Xkz}lAin`@AC(EN?RxE%SV|nJ zZC_`bfJ9yV*f63G-93I$WzCpoFQpcbRI9{C9vbsYB9c&sxftv3pRmF{5!`H7e|Da-3BZSKPs}i9*=+4a!**zqRT#q-Y(a3` ztr%)5*Hn1u2cum_NL9h=)u##`rPMQW-}nzUPAPc@&kZ(n2t^heifmOP<d2m}E|P$`O_^dcz5U;!1W!cqhz z^dc%V6FVDPZX3orndFPqK%W|bJ`|C7pF`a2* zf^tYgz2ivM<%<%nlf75?RfO!z`KHB0cq9dDqe@T$1qFxNpGG~yi~9oY;%6NksKA2q5l7`|vk}?)BxkjEHAGF*R-T#YUMfjt_2Jd%_y`7D&EsS~f})@RxoKNW|0 zL!|t1Bf@`fATMCzi6&@TWo9;le@-=$JCuP@Al$rP0*RBF@Ev)`294!aV0$Hw3FcnU zZ9-X(`Dxjxu#RrF%hp|>tR>8(#`#9uD07&l9M(;-_ndlTsAqNi^;@#InSs)d+??|{ z{ak5m2XHZofG<~j)k*#%;~K12cs$Srx?z!^%Tp*l#!;_{4Zr!2gTrE2tcRYWI@9Y{ z?7Rc`%7D%F#ee8ppZGHEt2a+fD@`}owBt2^+Qo&9-ge~^wHrG>Cc>PC?GSq|GOm@Q z?okhdU3CFVM33AbIpqo$e>KqntUA&c@km#Lc*is`qX&d?r+5oKoK%}cMfH|}l?{?? zB3jGw6A9i@_;~6)Vo>%D<&v<7p#Cr(uLPGvcFkQUM^`X&qc=n@)$rjmF(jR}#wf1` zgqQP1#U}t>iic4(#QNMwOMvG@o~YB|udczFn;nGW+oyVxq%m5=hHTEQC+Wq3WeRLS z+#mAyI`6W>2svj+?RM7IMEEeLcXL>da%zm|k%bB!q62jc9C#_OErn{0WhU#Ca-tu< zu9ktOUOFC-F3BEjt6Gn8jJ|h5_+=ls0Xv?15PT(~F>0d77a7_5RA)9ah_(G$w@hY> zzW>St1Yt*zICBjv^{K=?4JjR=g9;C7jT+v-uKV(}Oz=&n@Pn_t^WBZ|tPe>*n4+9j z{Ol!T?XT&k5QUXYjh<##cLZjpA&QYX>}xbze5CsnoN%g6FYu^CIskuhASK zwklkW5qA?$@ac=ZczgaM1oLhtB45=p!`zZ9v zV4-N}?LM?o@>%v%fe^OXc_^cf$^tgUa+2c~pXI|vg@Cb*R?}RCNz@ZBud{w814g>b zl+!8j6qdA;5qj8sp%Qm))08S#BzKC3K{Y@VkUUEBrz$#1#3ijQTg+I!{gjy9sio;t zf5LvBInhqOTgWfLMUkB~I2=^)YA_NcTXkVHdo? zt$u=&vkK&6bImvb#F|nMH`7_eV&0e{S{w7+$f?xMrxTa-)6Omchy5<3)AnPERv4|)DMsho&8q?SY0u}dCTV|wX#B}G)S)u>EljG^7!;(*+j>J90M* zUNz{fsKsOu8dy9s!?~OCIp>4j*$Q5949Baw2dIj+CDq=I;S1sEoZLU8l`ohUSswrV zc30CAnHP>6ni>P0V0LA@Y)mn7J~6e>fGi>}m=#3a=PlK$S8{I%TPJ9$UMMQ*r^)Q| z;N^YX*d5a=0x9Cc2UDgR`}?QZQ=;B|kS(Yu{8ny)x65JKU2sc6mjU@yw~4ilRcN%c zt1!nnp@he+IHfhB3)2npLw+zmMCs<3PJ}BU)atK_)kz3K2cvwe z9cp?HiM#@!`f_8mn72^&E0KWs3q$vbfX}Ve*ECTS*4;|sshbWv$-XCQmLjOO1T2gW z?Jr%9R&7Ss7~-!EkMl31-(xzSWv?@tb2iS?&)1dS*f(?Ug^l4lumL^nz@7=8Z)L1} z6}F|K+jeAMK_J>KVzYRb0R8B5&J{Nihh@#t50eJV9M()zl;v*#yHF{pO;>Cld>YaY7?)U&&=!?|eogl5tt3zW(ljLi2 z$?6p!TZy#vx+K{_jdp-KZf*x`Z^fQYhFZYu6T|C%w%V<)DEs&F09ZTzJqg@pn?oKi?8UmLS9uni_97#dI?a-PP$Fd~NHd zJ8X{DsD{Ma9v+NWyzJ8?l{*+k{v-L93kJmaQ?Zx@u1S4;otX`RV;+o z8SHh~?$;kM8o0|ca%x)cl7U_=uPph?4t&kf8i$Hc73$gpkG^?EKAR7oB&`IdC1WS% z$UZch^3$MTLF2)wy`7-W-QR}7s}BBY$}(xBCK` zU&*gpGtkdhRtZ_>C@o7^s63;6_HzHtzVm}_1>3qSk8o?+@>E$rnpbWZdK9>2Pz`fL z&6-)$UT$$1L@Ky-hF7$y>P3hRGC+!3LBctA9LR^pLNRc%Yl}K|u%Sw2yH%b$Aigw^ z9%`zW^!UtyKb`4pBQM^TSzXGC=dz+SB-&LRr(U;#k@Ql~7uTOl1vU>YdUQ|%dqU9` zXBHHT)$$IgxEq9Z#;%HW6q8cBZ-v#;g5E0{ReY@r>K(FBXbm3HKKqf(SSh?%*tT7h zcyb+t`ciP+mOuBwzCp#IDFbTo^Mc4xe)4?f`2LN%18#7xOZxS%8Sc;iR&m))k_7oS zDzJ{L58hY~?nqpcG<h9BU^HSguOofKmlWcESo-KVL|aIo9AD~pir z$&QLMsSOVhDRCJCNclJtTZUWYozXG%Mv6M5^>A(QtBm7QZ>JWb_wkhQ{2p~z5z#JR zy{b=fT2Z6jLx$(x3YTQ-PN$Q!0qmh0ac4`JC^3??%F^5VnTRn`6&X)Ba2`>dw&=Ke zkdKpNAsu~JZMWu#KW)Z)G#Dn(J9%JYFZ}>@?%#IC%iK9kRVPQ6$6aK}aZ(!RIpVlJ zJG{=8q3ZRUYp%NQGPQ+OzA|B#^KyQV9T?5ZG^l;yk1fJ&)bx6CIu<3@j14gARQ#@?({&hDrM<=4-Id{2dlM(F&o^cQ2Xc2G>H-Kk z4{1*wx=_68g0`cw!`q!)C05`0M$wBmXpOPJ3|6E#V7n#tA!j?#1^ipp+p5B*TURPm<+)M+x48}uP-8IQ!;zq*NFNj2rm}`;Uea@H zOz$py8V3$p514o-wVZP9Ve?JTG4#8|mDxbg*zUu*?57{{3l186vpgOs#}>?`FhHWU zhu(_(s&;Hq%}Jpjz`H7! zH6*j?{d-5LuwE;M856HNZ=UBG^ZRtlVlF*(p)=e>6}smyYzU)tocM-*)l;3#bYNP2npil_UQE82LLlhd54 zGus;3aLLFX@Mkr@iFodICBxFbm^xFcWX8$qMaJ%#MNTe`&t6tJ#-8=&t=GAw)~FMxcTZZHcrC-2UydlG9gy(>R|YVNqx_$N9S zGw~~Rrbte4)m%SG^pkC3$uy~fWzE!v`I;f=|c%} z?6`78PT@02(X*85xw~@e=z+d=Kfq12g@@cIdGo(XYXz|d;Qyd@H3iGJ5c|VoA%6nMzCy5N$Y)mM-TFs^-C4@6thI} zQ$-f+aYq(1(QOo@(#Sd$ZN0{v*NrR?jAcQsopuyZ-tYBcZZzkFyw&48OJ`b);^q1M&QDQ>gQ|K7&HJt2?ghOEZ> z-u_hOzb<1TLM^0|L}Nz2`afnw30!{TUn2i6%N|g>1p}dm0&7`)6)m@>=>MCUHE&Pc znR;7!G-Fo(-XfSs61K@BPRy~0?Gm90D~cL^f(aEz()=Sha-S1k}d!O z{Kp4~!RdNO=|UjDS-xM7(*M8?h;QBaA;-hu2s+@;Ky+XrKR8{sC8q4x|ut8-6_d zmK@NBvjAW88}uBu%ya|-9TbrNwR|v+n;5{?UmgpF#G!ta(RW)|OvUBj`c)=Jn$qO#NWf+(?`Ky8M=w90L1BHStw*vh2*9T z)%L(S;T06r{aqYQyu92YaF`4X1_eNf4vq?^uo^!gfRr-SnM5Ke$jXw*WEnD2#*64A z3&&tEvM_`!0wE1(Nc)`gB;ov}J$-}$1Qy691{UvQPjn%WT)aFXbWEI`moG^f3I$r= zck4(lj(=i%`ur#YpfF_VcVyu*Fj)W%0#TwT6;8O2d<^kKW1I)U9Zwg3&&3$;y_xu) za(7WdE6LJx17e#xn~5JvN}Is{6X|;fn+2kW_jDpTt6-3bAId*Vw`|!Io_28nG!aLS zqJCEWF5S|!P_t7b8vEL9n(gB0^b5m*9Dg!=>v{ev{;u=a=lO>`zN`II13 val df = Seq((1, "andy"), (2, "bob"), (2, "andy")).toDF("count", "name") +df: org.apache.spark.sql.DataFrame = [count: int, name: string] + +scala> df.count +res0: Long = 3 + +scala> df.createGlobalTempView("df") + +scala> spark.sql("select name,sum(count) from global_temp.df group by name").show ++----+----------+ +|name|sum(count)| ++----+----------+ +|andy| 3| +| bob| 2| ++----+----------+ +{% endhighlight %} + +

+ SQL tab + +

+ +Now the above three dataframe/SQL operators are shown in the list. If we click the +'show at \: 24' link of the last query, we will see the DAG of the job. + +

+ SQL DAG + +

+ +We can see that details information of each stage. The first block 'WholeStageCodegen' +compile multiple operator ('LocalTableScan' and 'HashAggregate') together into a single Java +function to improve performance, and metrics like number of rows and spill size are listed in +the block. The second block 'Exchange' shows the metrics on the shuffle exchange, including +number of written shuffle records, total data size, etc. + + +

+ logical plans and the physical plan + +

+Clicking the 'Details' link on the bottom displays the logical plans and the physical plan, which +illustrate how Spark parses, analyzes, optimizes and performs the query. + + +## Streaming Tab +The web UI includes a Streaming tab if the application uses Spark streaming. This tab displays +scheduling delay and processing time for each micro-batch in the data stream, which can be useful +for troubleshooting the streaming application. From 6964128e259b3001489809a4d75d88f920b28e62 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 12 Aug 2019 11:19:19 -0700 Subject: [PATCH 043/149] [SPARK-28017][SPARK-28656][SQL][FOLLOW-UP] Restore comments in date.sql ## What changes were proposed in this pull request? Restored comments in `date.sql` removed by https://github.com/apache/spark/commit/924d794a6f5abb972fa07bf63adbb4ad544ef246 and https://github.com/apache/spark/commit/997d153e54e3034f469191f69118897cad3e9d46 . The comments was introduced by 51379b731db4523d5e10ad9f2c063e445ae10e72 . ## How was this patch tested? By re-running `date.sql` via: ```shell $ build/sbt "sql/test-only *SQLQueryTestSuite -- -z date.sql" ``` Closes #25422 from MaxGekk/sql-comments-followup. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../resources/sql-tests/inputs/pgSQL/date.sql | 86 +++++++++---------- 1 file changed, 43 insertions(+), 43 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index f2c9e0b406e69..8341dc07e7e88 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -234,61 +234,61 @@ SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; -- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00'); -- 0 -- -- century - -SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; - +-- +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); -- -2 +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); -- -1 +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1 +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true +-- -- millennium - -SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); +-- +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1 +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 -- next test to be fixed on the turn of the next millennium;-) -SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); - +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 +-- -- decade - -SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); -SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); - +-- +SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 +SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 +SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); -- 0 +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- -1 +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); -- -1 +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); -- -2 +-- -- some other types: -- -- on a timestamp. -SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; -SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; -- true +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20 -- on an interval -- SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); -- 1 -- SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); -- 0 -- SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); -- 0 -- SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- -1 - +-- -- test trunc function! -SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); -- 0100-01-01 BC +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC -- [SPARK-28141] Date type can not accept special values -- From 48d04f74ca895497b9d8bab18c7708f76f55c520 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 12 Aug 2019 11:47:29 -0700 Subject: [PATCH 044/149] [SPARK-28638][WEBUI] Task summary should only contain successful tasks' metrics ## What changes were proposed in this pull request? Currently, on requesting summary metrics, cached data are returned if the current number of "SUCCESS" tasks is the same as the value in cached data. However, the number of "SUCCESS" tasks is wrong when there are running tasks. In `AppStatusStore`, the KVStore is `ElementTrackingStore`, instead of `InMemoryStore`. The value count is always the number of "SUCCESS" tasks + "RUNNING" tasks. Thus, even when the running tasks are finished, the out-of-update cached data is returned. This PR is to fix the code in getting the number of "SUCCESS" tasks. ## How was this patch tested? Test manually, run ``` sc.parallelize(1 to 160, 40).map(i => Thread.sleep(i*100)).collect() ``` and keep refreshing the stage page , we can see the task summary metrics is wrong. ### Before fix: ![image](https://user-images.githubusercontent.com/1097932/62560343-6a141780-b8af-11e9-8942-d88540659a93.png) ### After fix: ![image](https://user-images.githubusercontent.com/1097932/62560355-7009f880-b8af-11e9-8ba8-10c083a48d7b.png) Closes #25369 from gengliangwang/fixStagePage. Authored-by: Gengliang Wang Signed-off-by: Marcelo Vanzin --- .../apache/spark/status/AppStatusStore.scala | 11 +++++-- .../spark/status/AppStatusStoreSuite.scala | 32 ++++++++++++------- 2 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 6a9677834de9b..964ab27a524c4 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -136,6 +136,12 @@ private[spark] class AppStatusStore( store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).locality } + // SPARK-26119: we only want to consider successful tasks when calculating the metrics summary, + // but currently this is very expensive when using a disk store. So we only trigger the slower + // code path when we know we have all data in memory. The following method checks whether all + // the data will be in memory. + private def isInMemoryStore: Boolean = store.isInstanceOf[InMemoryStore] || listener.isDefined + /** * Calculates a summary of the task metrics for the given stage attempt, returning the * requested quantiles for the recorded metrics. @@ -156,7 +162,8 @@ private[spark] class AppStatusStore( // cheaper for disk stores (avoids deserialization). val count = { Utils.tryWithResource( - if (store.isInstanceOf[InMemoryStore]) { + if (isInMemoryStore) { + // For Live UI, we should count the tasks with status "SUCCESS" only. store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(TaskIndexNames.STATUS) @@ -245,7 +252,7 @@ private[spark] class AppStatusStore( // and failed tasks differently (would be tricky). Also would require changing the disk store // version (to invalidate old stores). def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = { - if (store.isInstanceOf[InMemoryStore]) { + if (isInMemoryStore) { val quantileTasks = store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(index) diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 75a658161d3ff..165fdb71cc78b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.status -import org.apache.spark.SparkFunSuite -import org.apache.spark.status.api.v1.TaskMetricDistributions +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.util.Distribution import org.apache.spark.util.kvstore._ @@ -77,14 +76,23 @@ class AppStatusStoreSuite extends SparkFunSuite { assert(store.count(classOf[CachedQuantile]) === 2) } - test("only successfull task have taskSummary") { + private def createLiveStore(inMemoryStore: InMemoryStore): AppStatusStore = { + val conf = new SparkConf() + val store = new ElementTrackingStore(inMemoryStore, conf) + val listener = new AppStatusListener(store, conf, true, None) + new AppStatusStore(store, listener = Some(listener)) + } + + test("SPARK-28638: only successful tasks have taskSummary when with in memory kvstore") { val store = new InMemoryStore() (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) } - val appStore = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles) - assert(appStore.size === 0) + Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore => + val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles) + assert(summary.size === 0) + } } - test("summary should contain task metrics of only successfull tasks") { + test("SPARK-28638: summary should contain successful tasks only when with in memory kvstore") { val store = new InMemoryStore() for (i <- 0 to 5) { @@ -95,13 +103,15 @@ class AppStatusStoreSuite extends SparkFunSuite { } } - val summary = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles).get + Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore => + val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles).get - val values = Array(0.0, 2.0, 4.0) + val values = Array(0.0, 2.0, 4.0) - val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) - dist.zip(summary.executorRunTime).foreach { case (expected, actual) => - assert(expected === actual) + val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) + dist.zip(summary.executorRunTime).foreach { case (expected, actual) => + assert(expected === actual) + } } } From 163f4a45dfb7c7e14e8456377a73741e1ebae983 Mon Sep 17 00:00:00 2001 From: s71955 Date: Mon, 12 Aug 2019 15:47:59 -0700 Subject: [PATCH 045/149] [SPARK-26969][SQL] Using ODBC client not able to see the query data when column datatype is decimal ## What changes were proposed in this pull request? While processing the Rowdata in the server side ColumnValue BigDecimal type value processed by server has to converted to the HiveDecmal data type for successful processing of query using Hive ODBC client.As per current logic corresponding to the Decimal column datatype, the Spark server uses BigDecimal, and the ODBC client uses HiveDecimal. If the data type does not match, the client fail to parse Since this handing was missing the query executed in Hive ODBC client wont return or provides result to the user even though the decimal type column value data present. ## How was this patch tested? Manual test report and impact assessment is done using existing test-cases Before fix ![decimal_odbc](https://user-images.githubusercontent.com/12999161/53440179-e74a7f00-3a29-11e9-93db-83f2ae37ef16.PNG) After Fix ![hive_odbc](https://user-images.githubusercontent.com/12999161/53679519-70e0a200-3cf3-11e9-9437-9c27d2e5056d.PNG) Closes #23899 from sujith71955/master_decimalissue. Authored-by: s71955 Signed-off-by: Dongjoon Hyun --- .../SparkThriftServerProtocolVersionsSuite.scala | 3 +-- .../org/apache/hive/service/cli/ColumnValue.java | 11 +---------- .../org/apache/hive/service/cli/ColumnValue.java | 12 +----------- 3 files changed, 3 insertions(+), 23 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index 9682a2a80eca9..f198372a4c998 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -186,8 +186,7 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - // TODO: enable this test case after SPARK-28463 and SPARK-26969 - ignore(s"$version get decimal type") { + test(s"$version get decimal type") { testExecuteStatementWithProtocolVersion(version, "SELECT cast(1 as decimal(18, 2)) as c") { rs => assert(rs.next()) diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java index ce833695f5867..547c6056b4fe8 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -23,7 +23,6 @@ import java.sql.Timestamp; import org.apache.hadoop.hive.common.type.HiveChar; -import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; @@ -140,14 +139,6 @@ private static TColumnValue timestampValue(Timestamp value) { return TColumnValue.stringVal(tStringValue); } - private static TColumnValue stringValue(HiveDecimal value) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - tStrValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStrValue); - } - private static TColumnValue stringValue(HiveIntervalYearMonth value) { TStringValue tStrValue = new TStringValue(); if (value != null) { @@ -195,7 +186,7 @@ public static TColumnValue toTColumnValue(Type type, Object value) { case INTERVAL_DAY_TIME_TYPE: return stringValue((HiveIntervalDayTime) value); case DECIMAL_TYPE: - return stringValue(((HiveDecimal)value)); + return stringValue(((BigDecimal)value).toPlainString()); case BINARY_TYPE: String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); return stringValue(strVal); diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java index 12d49b25142a4..f881ab159ea67 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -23,7 +23,6 @@ import java.sql.Timestamp; import org.apache.hadoop.hive.common.type.HiveChar; -import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; import org.apache.hadoop.hive.common.type.HiveVarchar; @@ -141,15 +140,6 @@ private static TColumnValue timestampValue(Timestamp value) { return TColumnValue.stringVal(tStringValue); } - private static TColumnValue stringValue(HiveDecimal value, TypeDescriptor typeDescriptor) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - int scale = typeDescriptor.getDecimalDigits(); - tStrValue.setValue(value.toFormatString(scale)); - } - return TColumnValue.stringVal(tStrValue); - } - private static TColumnValue stringValue(HiveIntervalYearMonth value) { TStringValue tStrValue = new TStringValue(); if (value != null) { @@ -199,7 +189,7 @@ public static TColumnValue toTColumnValue(TypeDescriptor typeDescriptor, Object case INTERVAL_DAY_TIME_TYPE: return stringValue((HiveIntervalDayTime) value); case DECIMAL_TYPE: - return stringValue((HiveDecimal)value, typeDescriptor); + return stringValue(((BigDecimal)value).toPlainString()); case BINARY_TYPE: String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); return stringValue(strVal); From 25857c655900e0be46370fc34487f54a4e3178a4 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 12 Aug 2019 17:02:28 -0700 Subject: [PATCH 046/149] [SPARK-28647][WEBUI] Recover additional metric feature and remove additional-metrics.js ## What changes were proposed in this pull request? By SPARK-17019, `On Heap Memory` and `Off Heap Memory` are introduced as optional metrics. But they are not displayed because they are made `display: none` in css and there are no way to appear them. I know #22595 also try to resolve this issue but that will use `additional-metrics.js`. Initially, `additional-metrics.js` is created for `StagePage` but `StagePage` currently uses `stagepage.js` for its additional metrics to be toggle because `DataTable (one of jQuery plugins)` was introduced and we needed another mechanism to add/remove columns for additional metrics. Now that `ExecutorsPage` also uses `DataTable` so it might be better to introduce same mechanism as `StagePage` for additional metrics. ![Screenshot from 2019-08-10 05-37-25](https://user-images.githubusercontent.com/4736016/62807960-c4240f80-bb31-11e9-8e1a-1a44e2f91597.png) And then, we can remove `additional-metrics.js` which is no longer used from anywhere. ## How was this patch tested? After this change is applied, I confirmed `ExecutorsPage` and `StagePage` are properly rendered and all checkboxes for additional metrics work. Closes #25374 from sarutak/remove-additional-metrics.js. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../spark/ui/static/additional-metrics.js | 90 -------------- .../ui/static/executorspage-template.html | 9 +- .../apache/spark/ui/static/executorspage.js | 111 ++++++++++++++---- .../org/apache/spark/ui/static/webui.css | 18 +-- .../scala/org/apache/spark/ui/UIUtils.scala | 1 - 5 files changed, 98 insertions(+), 131 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js deleted file mode 100644 index 3798dc47529af..0000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/* Register functions to show/hide columns based on checkboxes. These need - * to be registered after the page loads. */ -$(function() { - $("span.expand-additional-metrics").click(function(){ - var status = window.localStorage.getItem("expand-additional-metrics") == "true"; - status = !status; - - // Expand the list of additional metrics. - var additionalMetricsDiv = $(this).parent().find('.additional-metrics'); - $(additionalMetricsDiv).toggleClass('collapsed'); - - // Switch the class of the arrow from open to closed. - $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open'); - $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); - - window.localStorage.setItem("expand-additional-metrics", "" + status); - }); - - if (window.localStorage.getItem("expand-additional-metrics") == "true") { - // Set it to false so that the click function can revert it - window.localStorage.setItem("expand-additional-metrics", "false"); - $("span.expand-additional-metrics").trigger("click"); - } - - stripeSummaryTable(); - - $('input[type="checkbox"]').click(function() { - var name = $(this).attr("name") - var column = "table ." + name; - var status = window.localStorage.getItem(name) == "true"; - status = !status; - $(column).toggle(); - stripeSummaryTable(); - window.localStorage.setItem(name, "" + status); - }); - - $("#select-all-metrics").click(function() { - var status = window.localStorage.getItem("select-all-metrics") == "true"; - status = !status; - if (this.checked) { - // Toggle all un-checked options. - $('input[type="checkbox"]:not(:checked)').trigger('click'); - } else { - // Toggle all checked options. - $('input[type="checkbox"]:checked').trigger('click'); - } - window.localStorage.setItem("select-all-metrics", "" + status); - }); - - if (window.localStorage.getItem("select-all-metrics") == "true") { - $("#select-all-metrics").attr('checked', status); - } - - $("span.additional-metric-title").parent().find('input[type="checkbox"]').each(function() { - var name = $(this).attr("name") - // If name is undefined, then skip it because it's the "select-all-metrics" checkbox - if (name && window.localStorage.getItem(name) == "true") { - // Set it to false so that the click function can revert it - window.localStorage.setItem(name, "false"); - $(this).trigger("click") - } - }); - - // Trigger a click on the checkbox if a user clicks the label next to it. - $("span.additional-metric-title").click(function() { - $(this).parent().find('input[type="checkbox"]').trigger('click'); - }); - - // Show/hide full job description on click event. - $(".description-input").click(function() { - $(this).toggleClass("description-input-full"); - }); -}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index f2c17aef097a4..b236857cb4d73 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -16,6 +16,7 @@ --> - From ec84415358da35b56c3e7de0a551bca3be38f534 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Tue, 13 Aug 2019 10:06:32 +0900 Subject: [PATCH 047/149] [SPARK-28280][PYTHON][SQL][TESTS][FOLLOW-UP] Add UDF cases into group by clause in 'udf-group-by.sql' ## What changes were proposed in this pull request? This PR is a followup of a fix as described in here: https://github.com/apache/spark/pull/25215#issuecomment-517659981
Diff comparing to 'group-by.sql'

```diff diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 3a5df254f2..febe47b5ba 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out -13,26 +13,26 struct<> -- !query 1 -SELECT a, COUNT(b) FROM testData +SELECT udf(a), udf(COUNT(b)) FROM testData -- !query 1 schema struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; +grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; -- !query 2 -SELECT COUNT(a), COUNT(b) FROM testData +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData -- !query 2 schema -struct +struct -- !query 2 output 7 7 -- !query 3 -SELECT a, COUNT(b) FROM testData GROUP BY a +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a -- !query 3 schema -struct +struct -- !query 3 output 1 2 2 2 -41,7 +41,7 NULL 1 -- !query 4 -SELECT a, COUNT(b) FROM testData GROUP BY b +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b -- !query 4 schema struct<> -- !query 4 output -50,9 +50,9 expression 'testdata.`a`' is neither present in the group by, nor is it an aggre -- !query 5 -SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) -- !query 5 schema -struct +struct -- !query 5 output 0 1 2 2 -61,15 +61,15 struct -- !query 6 -SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 -- !query 6 schema -struct +struct -- !query 6 output foo 7 -- !query 7 -SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) -- !query 7 schema struct -- !query 7 output -77,25 +77,25 struct -- !query 8 -SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) -- !query 8 schema -struct +struct -- !query 8 output -- !query 9 -SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) -- !query 9 schema -struct> +struct> -- !query 9 output -- !query 10 -SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b -- !query 10 schema -struct<(a + b):int,count(b):bigint> +struct -- !query 10 output 2 1 3 2 -105,7 +105,7 NULL 1 -- !query 11 -SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 -- !query 11 schema struct<> -- !query 11 output -114,9 +114,9 expression 'testdata.`a`' is neither present in the group by, nor is it an aggre -- !query 12 -SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) -- !query 12 schema -struct<((a + 1) + 1):int,count(b):bigint> +struct<(CAST(udf(cast((a + 1) as string)) AS INT) + 1):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint> -- !query 12 output 3 2 4 2 -125,26 +125,26 NULL 1 -- !query 13 -SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData -- !query 13 schema -struct +struct -- !query 13 output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 -- !query 14 -SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) -- !query 14 schema -struct +struct -- !query 14 output 1 1 -- !query 15 -SELECT a AS k, COUNT(b) FROM testData GROUP BY k +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k -- !query 15 schema -struct +struct -- !query 15 output 1 2 2 2 -153,21 +153,21 NULL 1 -- !query 16 -SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 -- !query 16 schema -struct +struct -- !query 16 output 2 2 3 2 -- !query 17 -SELECT COUNT(b) AS k FROM testData GROUP BY k +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k -- !query 17 schema struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`); +aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT); -- !query 18 -180,7 +180,7 struct<> -- !query 19 -SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) -- !query 19 schema struct<> -- !query 19 output -197,32 +197,32 spark.sql.groupByAliases false -- !query 21 -SELECT a AS k, COUNT(b) FROM testData GROUP BY k +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k -- !query 21 schema struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 +cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57 -- !query 22 -SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) -- !query 22 schema -struct +struct -- !query 22 output -- !query 23 -SELECT COUNT(1) FROM testData WHERE false +SELECT udf(COUNT(1)) FROM testData WHERE false -- !query 23 schema -struct +struct -- !query 23 output 0 -- !query 24 -SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t -- !query 24 schema struct<1:int> -- !query 24 output -232,7 +232,7 struct<1:int> -- !query 25 SELECT 1 from ( SELECT 1 AS z, - MIN(a.x) + udf(MIN(a.x)) FROM (select 1 as x) a WHERE false ) b -244,32 +244,32 struct<1:int> -- !query 26 -SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) -- !query 26 schema -struct +struct -- !query 26 output 1.0 1.0 3 -- !query 27 -SELECT 1 FROM range(10) HAVING true +SELECT udf(1) FROM range(10) HAVING true -- !query 27 schema -struct<1:int> +struct -- !query 27 output 1 -- !query 28 -SELECT 1 FROM range(10) HAVING MAX(id) > 0 +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 -- !query 28 schema -struct<1:int> +struct -- !query 28 output 1 -- !query 29 -SELECT id FROM range(10) HAVING id > 0 +SELECT udf(id) FROM range(10) HAVING id > 0 -- !query 29 schema struct<> -- !query 29 output -291,33 +291,33 struct<> -- !query 31 -SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0 +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL -- !query 32 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4 +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL -- !query 33 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5 +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true -- !query 34 -SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true -327,9 +327,9 struct -- !query 35 -SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false -- !query 35 schema -struct +struct -- !query 35 output 1 false 3 false -337,77 +337,77 struct -- !query 36 -SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL -- !query 37 -SELECT k, - Every(v) AS every +SELECT udf(k), + udf(Every(v)) AS every FROM test_agg WHERE k = 2 AND v IN (SELECT Any(v) FROM test_agg WHERE k = 1) -GROUP BY k +GROUP BY udf(k) -- !query 37 schema -struct +struct -- !query 37 output 2 true -- !query 38 -SELECT k, +SELECT udf(udf(k)), Every(v) AS every FROM test_agg WHERE k = 2 AND v IN (SELECT Every(v) FROM test_agg WHERE k = 1) -GROUP BY k +GROUP BY udf(udf(k)) -- !query 38 schema -struct +struct -- !query 38 output -- !query 39 -SELECT every(1) +SELECT every(udf(1)) -- !query 39 schema struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 +cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 -- !query 40 -SELECT some(1S) +SELECT some(udf(1S)) -- !query 40 schema struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 -SELECT any(1L) +SELECT any(udf(1L)) -- !query 41 schema struct<> -- !query 41 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 +cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 -- !query 42 -SELECT every("true") +SELECT udf(every("true")) -- !query 42 schema struct<> -- !query 42 output org.apache.spark.sql.AnalysisException -cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7 +cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 -- !query 43 -428,9 +428,9 struct +struct -- !query 44 output 1 false false 1 true true -445,9 +445,9 struct +struct -- !query 45 output 1 false false 1 true true -462,17 +462,17 struct 1L +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L -- !query 46 schema -struct +struct -- !query 46 output 10 -- !query 47 -SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true -- !query 47 schema -struct +struct -- !query 47 output 1 true 2 true -480,7 +480,7 struct -- !query 48 -SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L -- !query 48 schema struct -- !query 48 output -488,7 +488,7 struct -- !query 49 -SELECT count(*) FROM test_agg WHERE count(*) > 1L +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L -- !query 49 schema struct<> -- !query 49 output -500,7 +500,7 Invalid expressions: [count(1)]; -- !query 50 -SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L -- !query 50 schema struct<> -- !query 50 output -512,7 +512,7 Invalid expressions: [count(1)]; -- !query 51 -SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 -- !query 51 schema struct<> -- !query 51 output ```

## How was this patch tested? Tested as instructed in SPARK-27921. Closes #25360 from skonto/group-by-followup. Authored-by: Stavros Kontopoulos Signed-off-by: HyukjinKwon --- .../sql-tests/inputs/udf/udf-group-by.sql | 28 +- .../results/udf/udf-group-by.sql.out | 273 +++++++++--------- 2 files changed, 154 insertions(+), 147 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql index 393de498edfa6..0cc57c97b0202 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql @@ -20,29 +20,25 @@ SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1; SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1); -- Aggregate grouped by literals (hash aggregate). -SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1; +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); -- Aggregate grouped by literals (sort aggregate). -SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1; +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); -- Aggregate with complex GroupBy expressions. SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b; SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1; - --- [SPARK-28445] Inconsistency between Scala and Python/Panda udfs when groupby with udf() is used --- The following query will make Scala UDF work, but Python and Pandas udfs will fail with an AnalysisException. --- The query should be added after SPARK-28445. --- SELECT udf(a + 1), udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); -- Aggregate with nulls. SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData; -- Aggregate with foldable input and multiple distinct groups. -SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a); -- Aliases in SELECT could be used in GROUP BY -SELECT a AS k, COUNT(udf(b)) FROM testData GROUP BY k; +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k; SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1; -- Aggregate functions cannot be used in GROUP BY @@ -51,7 +47,7 @@ SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k; -- Test data. CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); -SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY a; +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a); -- turn off group by aliases set spark.sql.groupByAliases=false; @@ -60,7 +56,7 @@ set spark.sql.groupByAliases=false; SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k; -- Aggregate with empty input and non-empty GroupBy expressions. -SELECT a, COUNT(udf(1)) FROM testData WHERE false GROUP BY a; +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a); -- Aggregate with empty input and empty GroupBy expressions. SELECT udf(COUNT(1)) FROM testData WHERE false; @@ -104,21 +100,21 @@ SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4; SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5; -- group by -SELECT k, every(v), udf(some(v)), any(v) FROM test_agg GROUP BY k; +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k); -- having SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; -SELECT k, udf(every(v)) FROM test_agg GROUP BY k HAVING every(v) IS NULL; +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL; -- basic subquery path to make sure rewrite happens in both parent and child plans. -SELECT k, +SELECT udf(k), udf(Every(v)) AS every FROM test_agg WHERE k = 2 AND v IN (SELECT Any(v) FROM test_agg WHERE k = 1) -GROUP BY k; +GROUP BY udf(k); -- basic subquery path to make sure rewrite happens in both parent and child plans. SELECT udf(udf(k)), @@ -128,7 +124,7 @@ WHERE k = 2 AND v IN (SELECT Every(v) FROM test_agg WHERE k = 1) -GROUP BY k; +GROUP BY udf(udf(k)); -- input type checking Int SELECT every(udf(1)); diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 789c91fa3870b..febe47b5ba84e 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 51 +-- Number of queries: 52 -- !query 0 @@ -77,7 +77,7 @@ struct -- !query 8 -SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1 +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) -- !query 8 schema struct -- !query 8 output @@ -85,7 +85,7 @@ struct> -- !query 9 output @@ -114,111 +114,122 @@ expression 'testdata.`a`' is neither present in the group by, nor is it an aggre -- !query 12 -SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) -FROM testData +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) -- !query 12 schema -struct +struct<(CAST(udf(cast((a + 1) as string)) AS INT) + 1):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint> -- !query 12 output --0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 +3 2 +4 2 +5 2 +NULL 1 -- !query 13 -SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData -- !query 13 schema -struct +struct -- !query 13 output -1 1 +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 -- !query 14 -SELECT a AS k, COUNT(udf(b)) FROM testData GROUP BY k +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) -- !query 14 schema -struct +struct -- !query 14 output +1 1 + + +-- !query 15 +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query 15 schema +struct +-- !query 15 output 1 2 2 2 3 2 NULL 1 --- !query 15 +-- !query 16 SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 --- !query 15 schema +-- !query 16 schema struct --- !query 15 output +-- !query 16 output 2 2 3 2 --- !query 16 +-- !query 17 SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k --- !query 16 schema +-- !query 17 schema struct<> --- !query 16 output +-- !query 17 output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT); --- !query 17 +-- !query 18 CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) --- !query 17 schema +-- !query 18 schema struct<> --- !query 17 output +-- !query 18 output --- !query 18 -SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY a --- !query 18 schema +-- !query 19 +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query 19 schema struct<> --- !query 18 output +-- !query 19 output org.apache.spark.sql.AnalysisException expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 19 +-- !query 20 set spark.sql.groupByAliases=false --- !query 19 schema +-- !query 20 schema struct --- !query 19 output +-- !query 20 output spark.sql.groupByAliases false --- !query 20 +-- !query 21 SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k --- !query 20 schema +-- !query 21 schema struct<> --- !query 20 output +-- !query 21 output org.apache.spark.sql.AnalysisException cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57 --- !query 21 -SELECT a, COUNT(udf(1)) FROM testData WHERE false GROUP BY a --- !query 21 schema -struct --- !query 21 output +-- !query 22 +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query 22 schema +struct +-- !query 22 output --- !query 22 +-- !query 23 SELECT udf(COUNT(1)) FROM testData WHERE false --- !query 22 schema +-- !query 23 schema struct --- !query 22 output +-- !query 23 output 0 --- !query 23 +-- !query 24 SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t --- !query 23 schema +-- !query 24 schema struct<1:int> --- !query 23 output +-- !query 24 output 1 --- !query 24 +-- !query 25 SELECT 1 from ( SELECT 1 AS z, udf(MIN(a.x)) @@ -226,88 +237,88 @@ SELECT 1 from ( WHERE false ) b where b.z != b.z --- !query 24 schema +-- !query 25 schema struct<1:int> --- !query 24 output +-- !query 25 output --- !query 25 +-- !query 26 SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) --- !query 25 schema +-- !query 26 schema struct --- !query 25 output +-- !query 26 output 1.0 1.0 3 --- !query 26 +-- !query 27 SELECT udf(1) FROM range(10) HAVING true --- !query 26 schema +-- !query 27 schema struct --- !query 26 output +-- !query 27 output 1 --- !query 27 +-- !query 28 SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 --- !query 27 schema +-- !query 28 schema struct --- !query 27 output +-- !query 28 output 1 --- !query 28 +-- !query 29 SELECT udf(id) FROM range(10) HAVING id > 0 --- !query 28 schema +-- !query 29 schema struct<> --- !query 28 output +-- !query 29 output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; --- !query 29 +-- !query 30 CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), (2, true), (3, false), (3, null), (4, null), (4, null), (5, null), (5, true), (5, false) AS test_agg(k, v) --- !query 29 schema +-- !query 30 schema struct<> --- !query 29 output +-- !query 30 output --- !query 30 +-- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 --- !query 30 schema +-- !query 31 schema struct --- !query 30 output +-- !query 31 output NULL NULL NULL --- !query 31 +-- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 --- !query 31 schema +-- !query 32 schema struct --- !query 31 output +-- !query 32 output NULL NULL NULL --- !query 32 +-- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 --- !query 32 schema +-- !query 33 schema struct --- !query 32 output +-- !query 33 output false true true --- !query 33 -SELECT k, every(v), udf(some(v)), any(v) FROM test_agg GROUP BY k --- !query 33 schema -struct --- !query 33 output +-- !query 34 +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query 34 schema +struct +-- !query 34 output 1 false true true 2 true true true 3 false false false @@ -315,40 +326,40 @@ struct --- !query 34 output +-- !query 35 output 1 false 3 false 5 false --- !query 35 -SELECT k, udf(every(v)) FROM test_agg GROUP BY k HAVING every(v) IS NULL --- !query 35 schema -struct --- !query 35 output +-- !query 36 +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query 36 schema +struct +-- !query 36 output 4 NULL --- !query 36 -SELECT k, +-- !query 37 +SELECT udf(k), udf(Every(v)) AS every FROM test_agg WHERE k = 2 AND v IN (SELECT Any(v) FROM test_agg WHERE k = 1) -GROUP BY k --- !query 36 schema -struct --- !query 36 output +GROUP BY udf(k) +-- !query 37 schema +struct +-- !query 37 output 2 true --- !query 37 +-- !query 38 SELECT udf(udf(k)), Every(v) AS every FROM test_agg @@ -356,54 +367,54 @@ WHERE k = 2 AND v IN (SELECT Every(v) FROM test_agg WHERE k = 1) -GROUP BY k --- !query 37 schema +GROUP BY udf(udf(k)) +-- !query 38 schema struct --- !query 37 output +-- !query 38 output --- !query 38 +-- !query 39 SELECT every(udf(1)) --- !query 38 schema +-- !query 39 schema struct<> --- !query 38 output +-- !query 39 output org.apache.spark.sql.AnalysisException cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 --- !query 39 +-- !query 40 SELECT some(udf(1S)) --- !query 39 schema +-- !query 40 schema struct<> --- !query 39 output +-- !query 40 output org.apache.spark.sql.AnalysisException cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 --- !query 40 +-- !query 41 SELECT any(udf(1L)) --- !query 40 schema +-- !query 41 schema struct<> --- !query 40 output +-- !query 41 output org.apache.spark.sql.AnalysisException cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 --- !query 41 +-- !query 42 SELECT udf(every("true")) --- !query 41 schema +-- !query 42 schema struct<> --- !query 41 output +-- !query 42 output org.apache.spark.sql.AnalysisException cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 --- !query 42 +-- !query 43 SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg --- !query 42 schema +-- !query 43 schema struct --- !query 42 output +-- !query 43 output 1 false false 1 true false 2 true true @@ -416,11 +427,11 @@ struct --- !query 43 output +-- !query 44 output 1 false false 1 true true 2 true true @@ -433,11 +444,11 @@ struct --- !query 44 output +-- !query 45 output 1 false false 1 true true 2 true true @@ -450,37 +461,37 @@ struct 1L --- !query 45 schema +-- !query 46 schema struct --- !query 45 output +-- !query 46 output 10 --- !query 46 +-- !query 47 SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true --- !query 46 schema +-- !query 47 schema struct --- !query 46 output +-- !query 47 output 1 true 2 true 5 true --- !query 47 +-- !query 48 SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L --- !query 47 schema +-- !query 48 schema struct --- !query 47 output +-- !query 48 output 10 --- !query 48 +-- !query 49 SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L --- !query 48 schema +-- !query 49 schema struct<> --- !query 48 output +-- !query 49 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -488,11 +499,11 @@ Expression in where clause: [(count(1) > 1L)] Invalid expressions: [count(1)]; --- !query 49 +-- !query 50 SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L --- !query 49 schema +-- !query 50 schema struct<> --- !query 49 output +-- !query 50 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -500,11 +511,11 @@ Expression in where clause: [((count(1) + 1L) > 1L)] Invalid expressions: [count(1)]; --- !query 50 +-- !query 51 SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 --- !query 50 schema +-- !query 51 schema struct<> --- !query 50 output +-- !query 51 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. From 016e1b491c9099063df8e2e76bc58d64fc369490 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 12 Aug 2019 20:42:06 -0700 Subject: [PATCH 048/149] [SPARK-28703][SQL][TEST] Skip HiveExternalCatalogVersionsSuite and 3 tests in HiveSparkSubmitSuite at JDK9+ ## What changes were proposed in this pull request? This PR skip more test when testing with `JAVA_9` or later: 1. Skip `HiveExternalCatalogVersionsSuite` when testing with `JAVA_9` or later because our previous version does not support `JAVA_9` or later. 2. Skip 3 tests in `HiveSparkSubmitSuite` because the `spark.sql.hive.metastore.version` of these tests is lower than `2.0`, however Datanucleus 3.x seem does not support `JAVA_9` or later. Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)): ``` [info] Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available. [info] at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215) [info] at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378) [info] at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392) [info] at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087) [info] at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247) ``` Please note that this exclude only the tests related to the old metastore library, some other tests of `HiveSparkSubmitSuite` still fail on JDK9+. ## How was this patch tested? manual tests: Test with JDK 11: ``` [info] HiveExternalCatalogVersionsSuite: [info] - backward compatibility !!! CANCELED !!! (37 milliseconds) [info] HiveSparkSubmitSuite: ... [info] - SPARK-8020: set sql conf in spark conf !!! CANCELED !!! (30 milliseconds) [info] org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:130) ... [info] - SPARK-9757 Persist Parquet relation with decimal column !!! CANCELED !!! (1 millisecond) [info] org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:168) ... [info] - SPARK-16901: set javax.jdo.option.ConnectionURL !!! CANCELED !!! (1 millisecond) [info] org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:260) ... ``` Closes #25426 from wangyum/SPARK-28703. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../hive/HiveExternalCatalogVersionsSuite.scala | 15 ++++++++++++--- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 4 ++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 9bc0be87be5af..9f7e85a0fa093 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -24,6 +24,7 @@ import java.nio.file.{Files, Paths} import scala.sys.process._ import scala.util.control.NonFatal +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.conf.Configuration import org.apache.spark.{SecurityManager, SparkConf, TestUtils} @@ -45,6 +46,7 @@ import org.apache.spark.util.Utils * downloading for this spark version. */ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { + private val isTestAtLeastJava9 = SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9) private val wareHousePath = Utils.createTempDir(namePrefix = "warehouse") private val tmpDataDir = Utils.createTempDir(namePrefix = "test-data") // For local test, you can set `sparkTestingDir` to a static value like `/tmp/test-spark`, to @@ -137,9 +139,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { new String(Files.readAllBytes(contentPath), StandardCharsets.UTF_8) } - override def beforeAll(): Unit = { - super.beforeAll() - + private def prepare(): Unit = { val tempPyFile = File.createTempFile("test", ".py") // scalastyle:off line.size.limit Files.write(tempPyFile.toPath, @@ -201,7 +201,16 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { tempPyFile.delete() } + override def beforeAll(): Unit = { + super.beforeAll() + if (!isTestAtLeastJava9) { + prepare() + } + } + test("backward compatibility") { + // TODO SPARK-28704 Test backward compatibility on JDK9+ once we have a version supports JDK9+ + assume(!isTestAtLeastJava9) val args = Seq( "--class", PROCESS_TABLES.getClass.getName.stripSuffix("$"), "--name", "HiveExternalCatalog backward compatibility test", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index e2ddec3427665..d23e0f2e0d937 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -21,6 +21,7 @@ import java.io.{BufferedWriter, File, FileWriter} import scala.util.Properties +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfterEach, Matchers} @@ -126,6 +127,7 @@ class HiveSparkSubmitSuite } test("SPARK-8020: set sql conf in spark conf") { + assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val args = Seq( "--class", SparkSQLConfTest.getClass.getName.stripSuffix("$"), @@ -163,6 +165,7 @@ class HiveSparkSubmitSuite } test("SPARK-9757 Persist Parquet relation with decimal column") { + assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val args = Seq( "--class", SPARK_9757.getClass.getName.stripSuffix("$"), @@ -254,6 +257,7 @@ class HiveSparkSubmitSuite } test("SPARK-16901: set javax.jdo.option.ConnectionURL") { + assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) // In this test, we set javax.jdo.option.ConnectionURL and set metastore version to // 0.13. This test will make sure that javax.jdo.option.ConnectionURL will not be // overridden by hive's default settings when we create a HiveConf object inside From 9a7f29023e57d1773a92edd54fb29bae79e3399d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 12 Aug 2019 21:01:19 -0700 Subject: [PATCH 049/149] [SPARK-28383][SQL] SHOW CREATE TABLE is not supported on a temporary view ## What changes were proposed in this pull request? It throws `Table or view not found` when showing temporary views: ```sql spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a; spark-sql> show create table temp_view; Error in query: Table or view 'temp_view' not found in database 'default'; ``` It's not easy to support temporary views. This pr changed it to throws `SHOW CREATE TABLE is not supported on a temporary view`: ```sql spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a; spark-sql> show create table temp_view; Error in query: SHOW CREATE TABLE is not supported on a temporary view: temp_view; ``` ## How was this patch tested? unit tests Closes #25149 from wangyum/SPARK-28383. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/command/tables.scala | 22 ++++++++++++------- .../spark/sql/ShowCreateTableSuite.scala | 20 +++++++++++++++++ .../spark/sql/execution/SQLViewSuite.scala | 5 ++++- 3 files changed, 38 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index af1e1bc59cbbc..9377cb0174673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -942,16 +942,22 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val tableMetadata = catalog.getTableMetadata(table) - - // TODO: unify this after we unify the CREATE TABLE syntax for hive serde and data source table. - val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { - showCreateDataSourceTable(tableMetadata) + if (catalog.isTemporaryTable(table)) { + throw new AnalysisException( + s"SHOW CREATE TABLE is not supported on a temporary view: ${table.identifier}") } else { - showCreateHiveTable(tableMetadata) - } + val tableMetadata = catalog.getTableMetadata(table) - Seq(Row(stmt)) + // TODO: [SPARK-28692] unify this after we unify the + // CREATE TABLE syntax for hive serde and data source table. + val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { + showCreateDataSourceTable(tableMetadata) + } else { + showCreateHiveTable(tableMetadata) + } + + Seq(Row(stmt)) + } } private def showCreateHiveTable(metadata: CatalogTable): String = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 5c347d2677d5e..ada3baf3cd720 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -162,6 +162,26 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { } } + test("temp view") { + val viewName = "spark_28383" + withTempView(viewName) { + sql(s"CREATE TEMPORARY VIEW $viewName AS SELECT 1 AS a") + val ex = intercept[AnalysisException] { + sql(s"SHOW CREATE TABLE $viewName") + } + assert(ex.getMessage.contains("SHOW CREATE TABLE is not supported on a temporary view")) + } + + withGlobalTempView(viewName) { + sql(s"CREATE GLOBAL TEMPORARY VIEW $viewName AS SELECT 1 AS a") + val ex = intercept[AnalysisException] { + val globalTempViewDb = spark.sessionState.catalog.globalTempViewManager.database + sql(s"SHOW CREATE TABLE $globalTempViewDb.$viewName") + } + assert(ex.getMessage.contains("SHOW CREATE TABLE is not supported on a temporary view")) + } + } + test("SPARK-24911: keep quotes for nested fields") { withTable("t1") { val createTable = "CREATE TABLE `t1` (`a` STRUCT<`b`: STRING>)" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 64e305cd5c371..8d4a9ae6a5760 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -159,7 +159,10 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { Thread.currentThread().getContextClassLoader.getResource("data/files/employee.dat") assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""") assertNoSuchTable(s"TRUNCATE TABLE $viewName") - assertNoSuchTable(s"SHOW CREATE TABLE $viewName") + val e2 = intercept[AnalysisException] { + sql(s"SHOW CREATE TABLE $viewName") + }.getMessage + assert(e2.contains("SHOW CREATE TABLE is not supported on a temporary view")) assertNoSuchTable(s"SHOW PARTITIONS $viewName") assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") From 247bebcf94df77883ac245aea63e7e871fc7aa44 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 12 Aug 2019 22:38:10 -0700 Subject: [PATCH 050/149] [SPARK-28561][WEBUI] DAG viz for barrier-execution mode ## What changes were proposed in this pull request? In the current UI, we cannot identify which RDDs are barrier. Visualizing it will make easy to debug. Following images are shown after this change. ![Screenshot from 2019-07-30 16-30-35](https://user-images.githubusercontent.com/4736016/62110508-83cec100-b2e9-11e9-83b9-bc2e485a4cbe.png) ![Screenshot from 2019-07-30 16-31-09](https://user-images.githubusercontent.com/4736016/62110509-83cec100-b2e9-11e9-9e2e-47c4dae23a52.png) The boxes in pale green mean barrier (We might need to discuss which color is proper). ## How was this patch tested? Tested manually. The images above are shown by following operations. ``` val rdd1 = sc.parallelize(1 to 10) val rdd2 = sc.parallelize(1 to 10) val rdd3 = rdd1.zip(rdd2).barrier.mapPartitions(identity(_)) val rdd4 = rdd3.map(identity(_)) val rdd5 = rdd4.reduceByKey(_+_) rdd5.collect ``` Closes #25296 from sarutak/barrierexec-dagviz. Authored-by: Kousuke Saruta Signed-off-by: Xingbo Jiang --- .../apache/spark/ui/static/spark-dag-viz.css | 12 ++++++++ .../apache/spark/ui/static/spark-dag-viz.js | 6 ++++ .../org/apache/spark/status/storeTypes.scala | 4 ++- .../org/apache/spark/storage/RDDInfo.scala | 3 +- .../scala/org/apache/spark/ui/UIUtils.scala | 3 ++ .../spark/ui/scope/RDDOperationGraph.scala | 30 +++++++++++++++---- .../org/apache/spark/util/JsonProtocol.scala | 4 ++- .../spark/status/AppStatusListenerSuite.scala | 6 ++-- .../apache/spark/storage/StorageSuite.scala | 4 +-- .../ui/scope/RDDOperationGraphSuite.scala | 10 +++---- .../apache/spark/util/JsonProtocolSuite.scala | 7 +++-- 11 files changed, 67 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css index 9cc5c79f67346..1fbc90b832bc9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css @@ -89,6 +89,12 @@ stroke-width: 2px; } +#dag-viz-graph svg.job g.cluster.barrier rect { + fill: #B4E9E2; + stroke: #32DBC6; + stroke-width: 2px; +} + /* Stage page specific styles */ #dag-viz-graph svg.stage g.cluster rect { @@ -123,6 +129,12 @@ stroke-width: 2px; } +#dag-viz-graph svg.stage g.cluster.barrier rect { + fill: #84E9E2; + stroke: #32DBC6; + stroke-width: 2px; +} + .tooltip-inner { white-space: pre-wrap; } 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 cf508ac573f39..035d72f8956ff 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 @@ -172,6 +172,12 @@ function renderDagViz(forJob) { svg.selectAll("g." + nodeId).classed("cached", true); }); + metadataContainer().selectAll(".barrier-rdd").each(function() { + var rddId = d3.select(this).text().trim() + var clusterId = VizConstants.clusterPrefix + rddId + svg.selectAll("g." + clusterId).classed("barrier", true) + }); + resizeSvg(svg); interpretLineBreak(svg); } diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index eea47b3b17098..9da5bea8bf5c4 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -402,7 +402,9 @@ private[spark] class RDDOperationClusterWrapper( val childClusters: Seq[RDDOperationClusterWrapper]) { def toRDDOperationCluster(): RDDOperationCluster = { - val cluster = new RDDOperationCluster(id, name) + val isBarrier = childNodes.exists(_.barrier) + val name = if (isBarrier) this.name + "\n(barrier mode)" else this.name + val cluster = new RDDOperationCluster(id, isBarrier, name) childNodes.foreach(cluster.attachChildNode) childClusters.foreach { child => cluster.attachChildCluster(child.toRDDOperationCluster()) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 917cfab1c699a..27a4d4b64175e 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -29,6 +29,7 @@ class RDDInfo( var name: String, val numPartitions: Int, var storageLevel: StorageLevel, + val isBarrier: Boolean, val parentIds: Seq[Int], val callSite: String = "", val scope: Option[RDDOperationScope] = None) @@ -68,6 +69,6 @@ private[spark] object RDDInfo { rdd.creationSite.shortForm } new RDDInfo(rdd.id, rddName, rdd.partitions.length, - rdd.getStorageLevel, parentIds, callSite, rdd.scope) + rdd.getStorageLevel, rdd.isBarrier(), parentIds, callSite, rdd.scope) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 11d38318b48e2..70e24bd0e7ecd 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -425,6 +425,9 @@ private[spark] object UIUtils extends Logging { { g.rootCluster.getCachedNodes.map { n =>
{n.id}
+ } ++ + g.rootCluster.getBarrierClusters.map { c => +
{c.id}
} } 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 540c1c45d2a94..9ace324322947 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 @@ -42,7 +42,12 @@ private[spark] case class RDDOperationGraph( rootCluster: RDDOperationCluster) /** A node in an RDDOperationGraph. This represents an RDD. */ -private[spark] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) +private[spark] case class RDDOperationNode( + id: Int, + name: String, + cached: Boolean, + barrier: Boolean, + callsite: String) /** * A directed edge connecting two nodes in an RDDOperationGraph. @@ -56,7 +61,10 @@ private[spark] case class RDDOperationEdge(fromId: Int, toId: Int) * This represents any grouping of RDDs, including operation scopes (e.g. textFile, flatMap), * stages, jobs, or any higher level construct. A cluster may be nested inside of other clusters. */ -private[spark] class RDDOperationCluster(val id: String, private var _name: String) { +private[spark] class RDDOperationCluster( + val id: String, + val barrier: Boolean, + private var _name: String) { private val _childNodes = new ListBuffer[RDDOperationNode] private val _childClusters = new ListBuffer[RDDOperationCluster] @@ -75,6 +83,10 @@ private[spark] class RDDOperationCluster(val id: String, private var _name: Stri _childNodes.filter(_.cached) ++ _childClusters.flatMap(_.getCachedNodes) } + def getBarrierClusters: Seq[RDDOperationCluster] = { + _childClusters.filter(_.barrier) ++ _childClusters.flatMap(_.getBarrierClusters) + } + def canEqual(other: Any): Boolean = other.isInstanceOf[RDDOperationCluster] override def equals(other: Any): Boolean = other match { @@ -117,7 +129,7 @@ private[spark] object RDDOperationGraph extends Logging { val stageClusterId = STAGE_CLUSTER_PREFIX + stage.stageId val stageClusterName = s"Stage ${stage.stageId}" + { if (stage.attemptNumber == 0) "" else s" (attempt ${stage.attemptNumber})" } - val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName) + val rootCluster = new RDDOperationCluster(stageClusterId, false, stageClusterName) var rootNodeCount = 0 val addRDDIds = new mutable.HashSet[Int]() @@ -143,7 +155,7 @@ private[spark] object RDDOperationGraph extends Logging { // TODO: differentiate between the intention to cache an RDD and whether it's actually cached val node = nodes.getOrElseUpdate(rdd.id, RDDOperationNode( - rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE, rdd.callSite)) + rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE, rdd.isBarrier, rdd.callSite)) if (rdd.scope.isEmpty) { // This RDD has no encompassing scope, so we put it directly in the root cluster // This should happen only if an RDD is instantiated outside of a public RDD API @@ -157,7 +169,8 @@ private[spark] object RDDOperationGraph extends Logging { val rddClusters = rddScopes.map { scope => val clusterId = scope.id val clusterName = scope.name.replaceAll("\\n", "\\\\n") - clusters.getOrElseUpdate(clusterId, new RDDOperationCluster(clusterId, clusterName)) + clusters.getOrElseUpdate( + clusterId, new RDDOperationCluster(clusterId, false, clusterName)) } // Build the cluster hierarchy for this RDD rddClusters.sliding(2).foreach { pc => @@ -227,7 +240,12 @@ private[spark] object RDDOperationGraph extends Logging { } else { "" } - val label = s"${node.name} [${node.id}]$isCached\n${node.callsite}" + val isBarrier = if (node.barrier) { + " [Barrier]" + } else { + "" + } + val label = s"${node.name} [${node.id}]$isCached$isBarrier\n${node.callsite}" s"""${node.id} [label="${StringEscapeUtils.escapeJava(label)}"]""" } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b8ca4ee1a6582..6b069754cfab2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -1049,12 +1049,14 @@ private[spark] object JsonProtocol { .map { l => l.extract[List[JValue]].map(_.extract[Int]) } .getOrElse(Seq.empty) val storageLevel = storageLevelFromJson(json \ "Storage Level") + val isBarrier = jsonOption(json \ "Barrier").map(_.extract[Boolean]).getOrElse(false) val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel, parentIds, callsite, scope) + val rddInfo = + new RDDInfo(rddId, name, numPartitions, storageLevel, isBarrier, parentIds, callsite, scope) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize rddInfo.diskSize = diskSize diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 7d73546ae227c..4b71a4844bde1 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -698,8 +698,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val level = StorageLevel.MEMORY_AND_DISK // Submit a stage and make sure the RDDs are recorded. - val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, Nil) - val rdd2Info = new RDDInfo(rdd2b1.rddId, "rdd2", 1, level, Nil) + val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, false, Nil) + val rdd2Info = new RDDInfo(rdd2b1.rddId, "rdd2", 1, level, false, Nil) val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1") listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) @@ -1543,7 +1543,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val level = StorageLevel.MEMORY_AND_DISK // Submit a stage and make sure the RDDs are recorded. - val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, Nil) + val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, false, Nil) val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1") listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index ca352387055f4..5f2abb47413f6 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -123,8 +123,8 @@ class StorageSuite extends SparkFunSuite { // For testing StorageUtils.updateRddInfo private def stockRDDInfos: Seq[RDDInfo] = { - val info0 = new RDDInfo(0, "0", 10, memAndDisk, Seq(3)) - val info1 = new RDDInfo(1, "1", 3, memAndDisk, Seq(4)) + val info0 = new RDDInfo(0, "0", 10, memAndDisk, false, Seq(3)) + val info1 = new RDDInfo(1, "1", 3, memAndDisk, false, Seq(4)) Seq(info0, info1) } diff --git a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala index 6ddcb5aba1678..e335451f1e484 100644 --- a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala @@ -22,14 +22,14 @@ import org.apache.spark.SparkFunSuite class RDDOperationGraphSuite extends SparkFunSuite { test("Test simple cluster equals") { // create a 2-cluster chain with a child - val c1 = new RDDOperationCluster("1", "Bender") - val c2 = new RDDOperationCluster("2", "Hal") + val c1 = new RDDOperationCluster("1", false, "Bender") + val c2 = new RDDOperationCluster("2", false, "Hal") c1.attachChildCluster(c2) - c1.attachChildNode(new RDDOperationNode(3, "Marvin", false, "collect!")) + c1.attachChildNode(new RDDOperationNode(3, "Marvin", false, false, "collect!")) // create an equal cluster, but without the child node - val c1copy = new RDDOperationCluster("1", "Bender") - val c2copy = new RDDOperationCluster("2", "Hal") + val c1copy = new RDDOperationCluster("1", false, "Bender") + val c2copy = new RDDOperationCluster("2", false, "Hal") c1copy.attachChildCluster(c2copy) assert(c1 == c1copy) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a093fa686f195..bbf64be343838 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -366,14 +366,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("RDDInfo backward compatibility (scope, parent IDs, callsite)") { // "Scope" and "Parent IDs" were introduced in Spark 1.4.0 // "Callsite" was introduced in Spark 1.6.0 - val rddInfo = new RDDInfo(1, "one", 100, StorageLevel.NONE, Seq(1, 6, 8), + val rddInfo = new RDDInfo(1, "one", 100, StorageLevel.NONE, false, Seq(1, 6, 8), "callsite", Some(new RDDOperationScope("fable"))) val oldRddInfoJson = JsonProtocol.rddInfoToJson(rddInfo) .removeField({ _._1 == "Parent IDs"}) .removeField({ _._1 == "Scope"}) .removeField({ _._1 == "Callsite"}) val expectedRddInfo = new RDDInfo( - 1, "one", 100, StorageLevel.NONE, Seq.empty, "", scope = None) + 1, "one", 100, StorageLevel.NONE, false, Seq.empty, "", scope = None) assertEquals(expectedRddInfo, JsonProtocol.rddInfoFromJson(oldRddInfoJson)) } @@ -857,7 +857,8 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK, Seq(1, 4, 7), a.toString) + val r = + new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK, false, Seq(1, 4, 7), a.toString) r.numCachedPartitions = c r.memSize = d r.diskSize = e From 3249c7ab49fe9c30c1b7cefff149b490f6211770 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 13 Aug 2019 19:02:04 +0800 Subject: [PATCH 051/149] [SPARK-28706][SQL] Allow cast null type to any types ## What changes were proposed in this pull request? #25242 proposed to disallow upcasting complex data types to string type, however, upcasting from null type to any types should still be safe. ## How was this patch tested? Add corresponding case in `CastSuite`. Closes #25425 from jiangxb1987/nullToString. Authored-by: Xingbo Jiang Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 1 + .../apache/spark/sql/catalyst/expressions/CastSuite.scala | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) 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 10464dac8d55e..a0cb5da078438 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 @@ -134,6 +134,7 @@ object Cast { case (DateType, TimestampType) => true case (_: AtomicType, StringType) => true case (_: CalendarIntervalType, StringType) => true + case (NullType, _) => true // Spark supports casting between long and timestamp, please see `longToTimestamp` and // `timestampToLong` for details. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index bbb3cb516b7d5..69adb8e922cc3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -974,7 +974,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { ) } - import DataTypeTestUtils.numericTypes + import DataTypeTestUtils._ numericTypes.foreach { from => val (safeTargetTypes, unsafeTargetTypes) = numericTypes.partition(to => isCastSafe(from, to)) @@ -1008,6 +1008,10 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { assert(!Cast.canUpCast(complexType, StringType)) } } + + atomicTypes.foreach { atomicType => + assert(Cast.canUpCast(NullType, atomicType)) + } } test("SPARK-27671: cast from nested null type in struct") { From 48adc91057d4995baf0450d18ed118544327699a Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 13 Aug 2019 20:52:16 +0800 Subject: [PATCH 052/149] [SPARK-28698][SQL] Support user-specified output schema in `to_avro` ## What changes were proposed in this pull request? The mapping of Spark schema to Avro schema is many-to-many. (See https://spark.apache.org/docs/latest/sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion) The default schema mapping might not be exactly what users want. For example, by default, a "string" column is always written as "string" Avro type, but users might want to output the column as "enum" Avro type. With PR https://github.com/apache/spark/pull/21847, Spark supports user-specified schema in the batch writer. For the function `to_avro`, we should support user-specified output schema as well. ## How was this patch tested? Unit test. Closes #25419 from gengliangwang/to_avro. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../spark/sql/avro/CatalystDataToAvro.scala | 9 +++- .../org/apache/spark/sql/avro/functions.scala | 15 ++++++- .../AvroCatalystDataConversionSuite.scala | 43 +++++++++++++++++-- python/pyspark/sql/avro/functions.py | 21 ++++++--- 4 files changed, 76 insertions(+), 12 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 6ed330d92f5e6..7732c83aaf7c4 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.avro import java.io.ByteArrayOutputStream +import org.apache.avro.Schema import org.apache.avro.generic.GenericDatumWriter import org.apache.avro.io.{BinaryEncoder, EncoderFactory} @@ -26,12 +27,16 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types.{BinaryType, DataType} -case class CatalystDataToAvro(child: Expression) extends UnaryExpression { +case class CatalystDataToAvro( + child: Expression, + jsonFormatSchema: Option[String]) extends UnaryExpression { override def dataType: DataType = BinaryType @transient private lazy val avroType = - SchemaConverters.toAvroType(child.dataType, child.nullable) + jsonFormatSchema + .map(new Schema.Parser().parse) + .getOrElse(SchemaConverters.toAvroType(child.dataType, child.nullable)) @transient private lazy val serializer = new AvroSerializer(child.dataType, avroType, child.nullable) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala index 5ed7828510d54..a6ae3906c6d80 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala @@ -72,6 +72,19 @@ object functions { */ @Experimental def to_avro(data: Column): Column = { - new Column(CatalystDataToAvro(data.expr)) + new Column(CatalystDataToAvro(data.expr, None)) + } + + /** + * Converts a column into binary of avro format. + * + * @param data the data column. + * @param jsonFormatSchema user-specified output avro schema in JSON string format. + * + * @since 3.0.0 + */ + @Experimental + def to_avro(data: Column, jsonFormatSchema: String): Column = { + new Column(CatalystDataToAvro(data.expr, Some(jsonFormatSchema))) } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 80dd4c535ad9c..27915562fded0 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.avro import org.apache.avro.Schema -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} @@ -38,12 +38,12 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite private def checkResult(data: Literal, schema: String, expected: Any): Unit = { checkEvaluation( - AvroDataToCatalyst(CatalystDataToAvro(data), schema, Map.empty), + AvroDataToCatalyst(CatalystDataToAvro(data, None), schema, Map.empty), prepareExpectedResult(expected)) } protected def checkUnsupportedRead(data: Literal, schema: String): Unit = { - val binary = CatalystDataToAvro(data) + val binary = CatalystDataToAvro(data, None) intercept[Exception] { AvroDataToCatalyst(binary, schema, Map("mode" -> "FAILFAST")).eval() } @@ -209,4 +209,41 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite checkUnsupportedRead(input, avroSchema) } } + + test("user-specified output schema") { + val data = Literal("SPADES") + val jsonFormatSchema = + """ + |{ "type": "enum", + | "name": "Suit", + | "symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"] + |} + """.stripMargin + + val message = intercept[SparkException] { + AvroDataToCatalyst( + CatalystDataToAvro( + data, + None), + jsonFormatSchema, + options = Map.empty).eval() + }.getMessage + assert(message.contains("Malformed records are detected in record parsing.")) + + checkEvaluation( + AvroDataToCatalyst( + CatalystDataToAvro( + data, + Some(jsonFormatSchema)), + jsonFormatSchema, + options = Map.empty), + data.eval()) + } + + test("invalid user-specified output schema") { + val message = intercept[IncompatibleSchemaException] { + CatalystDataToAvro(Literal("SPADES"), Some("\"long\"")).eval() + }.getMessage + assert(message == "Cannot convert Catalyst type StringType to Avro type \"long\".") + } } diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index e07b625284175..711de6532e28c 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -69,7 +69,7 @@ def from_avro(data, jsonFormatSchema, options={}): @ignore_unicode_prefix @since(3.0) -def to_avro(data): +def to_avro(data, jsonFormatSchema=""): """ Converts a column into binary of avro format. @@ -77,18 +77,27 @@ def to_avro(data): application as per the deployment section of "Apache Avro Data Source Guide". :param data: the data column. + :param jsonFormatSchema: user-specified output avro schema in JSON string format. >>> from pyspark.sql import Row >>> from pyspark.sql.avro.functions import to_avro - >>> data = [(1, Row(name='Alice', age=2))] - >>> df = spark.createDataFrame(data, ("key", "value")) - >>> df.select(to_avro(df.value).alias("avro")).collect() - [Row(avro=bytearray(b'\\x00\\x00\\x04\\x00\\nAlice'))] + >>> data = ['SPADES'] + >>> df = spark.createDataFrame(data, "string") + >>> df.select(to_avro(df.value).alias("suite")).collect() + [Row(suite=bytearray(b'\\x00\\x0cSPADES'))] + >>> jsonFormatSchema = '''["null", {"type": "enum", "name": "value", + ... "symbols": ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]}]''' + >>> df.select(to_avro(df.value, jsonFormatSchema).alias("suite")).collect() + [Row(suite=bytearray(b'\\x02\\x00'))] """ sc = SparkContext._active_spark_context try: - jc = sc._jvm.org.apache.spark.sql.avro.functions.to_avro(_to_java_column(data)) + if jsonFormatSchema == "": + jc = sc._jvm.org.apache.spark.sql.avro.functions.to_avro(_to_java_column(data)) + else: + jc = sc._jvm.org.apache.spark.sql.avro.functions.to_avro( + _to_java_column(data), jsonFormatSchema) except TypeError as e: if str(e) == "'JavaPackage' object is not callable": _print_missing_jar("Avro", "avro", "avro", sc.version) From e6a0385289f2d2fec05d3fb5f798903de292c381 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 14 Aug 2019 00:32:33 +0900 Subject: [PATCH 053/149] [SPARK-28422][SQL][PYTHON] GROUPED_AGG pandas_udf should work without group by clause ## What changes were proposed in this pull request? A GROUPED_AGG pandas python udf can't work, if without group by clause, like `select udf(id) from table`. This doesn't match with aggregate function like sum, count..., and also dataset API like `df.agg(udf(df['id']))`. When we parse a udf (or an aggregate function) like that from SQL syntax, it is known as a function in a project. `GlobalAggregates` rule in analysis makes such project as aggregate, by looking for aggregate expressions. At the moment, we should also look for GROUPED_AGG pandas python udf. ## How was this patch tested? Added tests. Closes #25352 from viirya/SPARK-28422. Authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../sql/tests/test_pandas_udf_grouped_agg.py | 15 ++++++++++++ .../sql/catalyst/analysis/Analyzer.scala | 5 +++- .../spark/sql/catalyst/plans/PlanTest.scala | 2 ++ .../python/BatchEvalPythonExecSuite.scala | 24 ++++++++++++++++++- 4 files changed, 44 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index 041b2b54739f8..6d460df66da28 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -474,6 +474,21 @@ def test_grouped_with_empty_partition(self): result = df.groupBy('id').agg(f(df['x']).alias('sum')).collect() self.assertEqual(result, expected) + def test_grouped_without_group_by_clause(self): + @pandas_udf('double', PandasUDFType.GROUPED_AGG) + def max_udf(v): + return v.max() + + df = self.spark.range(0, 100) + self.spark.udf.register('max_udf', max_udf) + + with self.tempView("table"): + df.createTempView('table') + + agg1 = df.agg(max_udf(df['id'])) + agg2 = self.spark.sql("select max_udf(id) from table") + assert_frame_equal(agg1.toPandas(), agg2.toPandas()) + if __name__ == "__main__": from pyspark.sql.tests.test_pandas_udf_grouped_agg import * 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 f8eef0cf32361..5a04d5794a659 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 @@ -1799,15 +1799,18 @@ class Analyzer( def containsAggregates(exprs: Seq[Expression]): Boolean = { // Collect all Windowed Aggregate Expressions. - val windowedAggExprs = exprs.flatMap { expr => + val windowedAggExprs: Set[Expression] = exprs.flatMap { expr => expr.collect { case WindowExpression(ae: AggregateExpression, _) => ae + case WindowExpression(e: PythonUDF, _) if PythonUDF.isGroupedAggPandasUDF(e) => e } }.toSet // Find the first Aggregate Expression that is not Windowed. exprs.exists(_.collectFirst { case ae: AggregateExpression if !windowedAggExprs.contains(ae) => ae + case e: PythonUDF if PythonUDF.isGroupedAggPandasUDF(e) && + !windowedAggExprs.contains(e) => e }.isDefined) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 6e2a8427659c2..08f1f87514b1d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -81,6 +81,8 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => ae.copy(resultId = ExprId(0)) case lv: NamedLambdaVariable => lv.copy(exprId = ExprId(0), value = null) + case udf: PythonUDF => + udf.copy(resultId = ExprId(0)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index 289cc667a1c66..ac5752b41cf28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.BooleanType +import org.apache.spark.sql.types.{BooleanType, DoubleType} class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { import testImplicits.newProductEncoder @@ -100,6 +100,21 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { } assert(qualifiedPlanNodes.size == 1) } + + test("SPARK-28422: GROUPED_AGG pandas_udf should work without group by clause") { + val aggPandasUdf = new MyDummyGroupedAggPandasUDF + spark.udf.registerPython("dummyGroupedAggPandasUDF", aggPandasUdf) + + withTempView("table") { + val df = spark.range(0, 100) + df.createTempView("table") + + val agg1 = df.agg(aggPandasUdf(df("id"))) + val agg2 = sql("select dummyGroupedAggPandasUDF(id) from table") + + comparePlans(agg1.queryExecution.optimizedPlan, agg2.queryExecution.optimizedPlan) + } + } } // This Python UDF is dummy and just for testing. Unable to execute. @@ -119,6 +134,13 @@ class MyDummyPythonUDF extends UserDefinedPythonFunction( pythonEvalType = PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) +class MyDummyGroupedAggPandasUDF extends UserDefinedPythonFunction( + name = "dummyGroupedAggPandasUDF", + func = new DummyUDF, + dataType = DoubleType, + pythonEvalType = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, + udfDeterministic = true) + class MyDummyScalarPandasUDF extends UserDefinedPythonFunction( name = "dummyScalarPandasUDF", func = new DummyUDF, From c81da276bac224f4ebea1225877ea7e4a902621b Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 13 Aug 2019 09:19:58 -0700 Subject: [PATCH 054/149] [SPARK-28714][SQL][TEST] Add `hive.aux.jars.path` test for spark-sql shell ## What changes were proposed in this pull request? `Utilities.addToClassPath` has been changed since [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096), but we use it to add plugin jars: https://github.com/apache/spark/blob/128ea37bda3dbc1d6ba2af762c7453ab7605d430/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala#L144-L147 This PR add test for `spark-sql` adding plugin jars. ## How was this patch tested? N/A Closes #25435 from wangyum/SPARK-28714. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/thriftserver/CliSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 4ccc17cf070c8..6e042ac41d9da 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -27,6 +27,7 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite @@ -290,8 +291,18 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val tmpDir = Utils.createTempDir(namePrefix = "SPARK-21451") runCliWithin( 1.minute, - Seq(s"--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$tmpDir"))( + Seq("--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$tmpDir"))( "set spark.sql.warehouse.dir;" -> tmpDir.getAbsolutePath) tmpDir.delete() } + + test("Support hive.aux.jars.path") { + val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + runCliWithin( + 1.minute, + Seq("--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( + s"CREATE TEMPORARY FUNCTION example_max AS '${classOf[UDAFExampleMax].getName}';" -> "", + "SELECT example_max(1);" -> "1" + ) + } } From d8dd5719b4b82ea5d2294002ada7aa4040bb2856 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 13 Aug 2019 11:09:14 -0700 Subject: [PATCH 055/149] [SPARK-28713][BUILD] Bump checkstyle from 8.14 to 8.23 ## What changes were proposed in this pull request? Fixes a vulnerability from the GitHub Security Advisory Database: _Moderate severity vulnerability that affects com.puppycrawl.tools:checkstyle_ Checkstyle prior to 8.18 loads external DTDs by default, which can potentially lead to denial of service attacks or the leaking of confidential information. https://github.com/checkstyle/checkstyle/issues/6474 Affected versions: < 8.18 ## How was this patch tested? Ran checkstyle locally. Closes #25432 from Fokko/SPARK-28713. Authored-by: Fokko Driesprong Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- project/plugins.sbt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index cee85515cb993..b0372ce8a8e69 100644 --- a/pom.xml +++ b/pom.xml @@ -2777,7 +2777,7 @@ com.puppycrawl.tools checkstyle - 8.14 + 8.23 diff --git a/project/plugins.sbt b/project/plugins.sbt index bed2cc1bc9c41..86349497ede81 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -18,7 +18,7 @@ addSbtPlugin("com.etsy" % "sbt-checkstyle-plugin" % "3.1.1") // sbt-checkstyle-plugin uses an old version of checkstyle. Match it to Maven's. -libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "8.14" +libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "8.23" // checkstyle uses guava 23.0. libraryDependencies += "com.google.guava" % "guava" % "23.0" From 13b62f31cd3f768582fbf6e397b04b524415b228 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 13 Aug 2019 11:21:19 -0700 Subject: [PATCH 056/149] [SPARK-28708][SQL] IsolatedClientLoader will not load hive classes from application jars on JDK9+ ## What changes were proposed in this pull request? We have 8 test cases in `HiveSparkSubmitSuite` still fail with `java.lang.ClassNotFoundException` when running on JDK9+: ``` [info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (9 seconds, 927 milliseconds) [info] spark-submit returned with exit code 1. [info] Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/root/.m2/repository/org/apache/hive/hive-contrib/2.3.6-SNAPSHOT/hive-contrib-2.3.6-SNAPSHOT.jar' 'file:/root/opensource/spark/target/tmp/spark-36d27542-7b82-4962-a362-bb51ef3e457d/testJar-1565682620744.jar' [info] [info] 2019-08-13 00:50:22.073 - stderr> WARNING: An illegal reflective access operation has occurred [info] 2019-08-13 00:50:22.073 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/root/opensource/spark/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int) [info] 2019-08-13 00:50:22.073 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform [info] 2019-08-13 00:50:22.073 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations [info] 2019-08-13 00:50:22.073 - stderr> WARNING: All illegal access operations will be denied in a future release [info] 2019-08-13 00:50:28.31 - stderr> Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/metadata/HiveException [info] 2019-08-13 00:50:28.31 - stderr> at java.base/java.lang.Class.getDeclaredConstructors0(Native Method) [info] 2019-08-13 00:50:28.31 - stderr> at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3138) [info] 2019-08-13 00:50:28.31 - stderr> at java.base/java.lang.Class.getConstructors(Class.java:1944) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:294) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:410) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:305) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221) [info] 2019-08-13 00:50:28.31 - stderr> at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:139) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:129) [info] 2019-08-13 00:50:28.31 - stderr> at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:42) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.hive.HiveSessionStateBuilder.$anonfun$catalog$1(HiveSessionStateBuilder.scala:57) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog$lzycompute(SessionCatalog.scala:91) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog(SessionCatalog.scala:91) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.catalyst.catalog.SessionCatalog.databaseExists(SessionCatalog.scala:244) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.catalyst.catalog.SessionCatalog.requireDbExists(SessionCatalog.scala:178) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:317) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.command.CreateTableCommand.run(tables.scala:132) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:213) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3431) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3427) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.Dataset.(Dataset.scala:213) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:95) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:653) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE$.main(HiveSparkSubmitSuite.scala:829) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE.main(HiveSparkSubmitSuite.scala) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/java.lang.reflect.Method.invoke(Method.java:566) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:920) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:179) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:202) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:89) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:999) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1008) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) [info] 2019-08-13 00:50:28.311 - stderr> Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.metadata.HiveException [info] 2019-08-13 00:50:28.311 - stderr> at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:471) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:588) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:250) [info] 2019-08-13 00:50:28.311 - stderr> at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:239) [info] 2019-08-13 00:50:28.311 - stderr> at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:521) [info] 2019-08-13 00:50:28.311 - stderr> ... 48 more ``` Note that this pr fixes `java.lang.ClassNotFoundException`, but the test will fail again with a different reason, the Hive-side `java.lang.ClassCastException` which will be resolved in the official Hive 2.3.6 release. ``` [info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (7 seconds, 649 milliseconds) [info] spark-submit returned with exit code 1. [info] Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/Users/dongjoon/.ivy2/cache/org.apache.hive/hive-contrib/jars/hive-contrib-2.3.5.jar' 'file:/Users/dongjoon/PRS/PR-25429/target/tmp/spark-48b7c936-0ec2-4311-9fb5-0de4bf86a0eb/testJar-1565710418275.jar' [info] [info] 2019-08-13 08:33:39.221 - stderr> WARNING: An illegal reflective access operation has occurred [info] 2019-08-13 08:33:39.221 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dongjoon/PRS/PR-25429/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int) [info] 2019-08-13 08:33:39.221 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform [info] 2019-08-13 08:33:39.221 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations [info] 2019-08-13 08:33:39.221 - stderr> WARNING: All illegal access operations will be denied in a future release [info] 2019-08-13 08:33:43.59 - stderr> Exception in thread "main" org.apache.spark.sql.AnalysisException: java.lang.ClassCastException: class jdk.internal.loader.ClassLoaders$AppClassLoader cannot be cast to class java.net.URLClassLoader (jdk.internal.loader.ClassLoaders$AppClassLoader and java.net.URLClassLoader are in module java.base of loader 'bootstrap'); [info] 2019-08-13 08:33:43.59 - stderr> at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109) ``` ## How was this patch tested? manual tests: 1. Install [Hive 2.3.6-SNAPSHOT](https://github.com/wangyum/hive/tree/HIVE-21584-branch-2.3) to local maven repository: ``` mvn clean install -DskipTests=true ``` 2. Upgrade our built-in Hive to 2.3.6-SNAPSHOT, you can checkout [this branch](https://github.com/wangyum/spark/tree/SPARK-28708-Hive-2.3.6) to test. 3. Test with hadoop-3.2: ``` build/sbt "hive/test-only *. HiveSparkSubmitSuite" -Phive -Phadoop-3.2 -Phive-thriftserver ... [info] Run completed in 3 minutes, 8 seconds. [info] Total number of tests run: 11 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 11, failed 0, canceled 3, ignored 0, pending 0 [info] All tests passed. ``` Closes #25429 from wangyum/SPARK-28708. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/hive/HiveUtils.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) 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 c3ae3d58a351e..2e6811d5f259d 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 @@ -338,18 +338,20 @@ private[spark] object HiveUtils extends Logging { } val classLoader = Utils.getContextOrSparkClassLoader - val jars = allJars(classLoader) - if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + val jars: Array[URL] = if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { // Do nothing. The system classloader is no longer a URLClassLoader in Java 9, - // so it won't match the case in allJars above. It no longer exposes URLs of + // so it won't match the case in allJars. It no longer exposes URLs of // the system classpath + Array.empty[URL] } else { + val loadedJars = allJars(classLoader) // Verify at least one jar was found - if (jars.length == 0) { + if (loadedJars.length == 0) { throw new IllegalArgumentException( "Unable to locate hive jars to connect to metastore. " + s"Please set ${HIVE_METASTORE_JARS.key}.") } + loadedJars } logInfo( From f04a7669461c4f564c64ca915ea4c768da7c5f55 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Aug 2019 15:36:28 -0700 Subject: [PATCH 057/149] [SPARK-28718][SQL] Support `field` synonyms at `extract` ## What changes were proposed in this pull request? In the PR, I propose additional synonyms for the `field` argument of `extract` supported by PostgreSQL. The `extract.sql` is updated to check all supported values of the `field` argument. The list of synonyms was taken from https://github.com/postgres/postgres/blob/master/src/backend/utils/adt/datetime.c . ## How was this patch tested? By running `extract.sql` via: ``` $ build/sbt "sql/test-only *SQLQueryTestSuite -- -z extract.sql" ``` Closes #25438 from MaxGekk/extract-field-synonyms. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/parser/AstBuilder.scala | 22 +- .../resources/sql-tests/inputs/extract.sql | 39 ++ .../sql-tests/results/extract.sql.out | 366 ++++++++++++++++-- 3 files changed, 377 insertions(+), 50 deletions(-) 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 7bbdd4f3c520e..6decfdb2c2244 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 @@ -1396,21 +1396,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { ctx.field.getText.toUpperCase(Locale.ROOT) match { - case "MILLENNIUM" => + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(expression(ctx.source)) - case "CENTURY" => + case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(expression(ctx.source)) - case "DECADE" => + case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(expression(ctx.source)) - case "YEAR" => + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(expression(ctx.source)) - case "QUARTER" => + case "QUARTER" | "QTR" => Quarter(expression(ctx.source)) - case "MONTH" => + case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(expression(ctx.source)) - case "WEEK" => + case "WEEK" | "W" | "WEEKS" => WeekOfYear(expression(ctx.source)) - case "DAY" => + case "DAY" | "D" | "DAYS" => DayOfMonth(expression(ctx.source)) case "DAYOFWEEK" => DayOfWeek(expression(ctx.source)) @@ -1420,11 +1420,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Add(WeekDay(expression(ctx.source)), Literal(1)) case "DOY" => DayOfYear(expression(ctx.source)) - case "HOUR" => + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(expression(ctx.source)) - case "MINUTE" => + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(expression(ctx.source)) - case "SECOND" => + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(expression(ctx.source)) case other => throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx) diff --git a/sql/core/src/test/resources/sql-tests/inputs/extract.sql b/sql/core/src/test/resources/sql-tests/inputs/extract.sql index 5dd3e6686f1cc..f42b75623d5a2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/extract.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/extract.sql @@ -1,14 +1,41 @@ CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c; +select extract(millennium from c) from t; +select extract(millennia from c) from t; +select extract(mil from c) from t; +select extract(mils from c) from t; + +select extract(century from c) from t; +select extract(centuries from c) from t; +select extract(c from c) from t; +select extract(cent from c) from t; + +select extract(decade from c) from t; +select extract(decades from c) from t; +select extract(dec from c) from t; +select extract(decs from c) from t; + select extract(year from c) from t; +select extract(y from c) from t; +select extract(years from c) from t; +select extract(yr from c) from t; +select extract(yrs from c) from t; select extract(quarter from c) from t; +select extract(qtr from c) from t; select extract(month from c) from t; +select extract(mon from c) from t; +select extract(mons from c) from t; +select extract(months from c) from t; select extract(week from c) from t; +select extract(w from c) from t; +select extract(weeks from c) from t; select extract(day from c) from t; +select extract(d from c) from t; +select extract(days from c) from t; select extract(dayofweek from c) from t; @@ -19,9 +46,21 @@ select extract(isodow from c) from t; select extract(doy from c) from t; select extract(hour from c) from t; +select extract(h from c) from t; +select extract(hours from c) from t; +select extract(hr from c) from t; +select extract(hrs from c) from t; select extract(minute from c) from t; +select extract(m from c) from t; +select extract(min from c) from t; +select extract(mins from c) from t; +select extract(minutes from c) from t; select extract(second from c) from t; +select extract(s from c) from t; +select extract(sec from c) from t; +select extract(seconds from c) from t; +select extract(secs from c) from t; select extract(not_supported from c) from t; diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index 0ca7bdc09b019..a7efe825c98e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 14 +-- Number of queries: 50 -- !query 0 @@ -11,106 +11,394 @@ struct<> -- !query 1 -select extract(year from c) from t +select extract(millennium from c) from t -- !query 1 schema -struct +struct -- !query 1 output -2011 +3 -- !query 2 -select extract(quarter from c) from t +select extract(millennia from c) from t -- !query 2 schema -struct +struct -- !query 2 output -2 +3 -- !query 3 -select extract(month from c) from t +select extract(mil from c) from t -- !query 3 schema -struct +struct -- !query 3 output -5 +3 -- !query 4 -select extract(week from c) from t +select extract(mils from c) from t -- !query 4 schema -struct +struct -- !query 4 output -18 +3 -- !query 5 -select extract(day from c) from t +select extract(century from c) from t -- !query 5 schema -struct +struct -- !query 5 output -6 +21 -- !query 6 -select extract(dayofweek from c) from t +select extract(centuries from c) from t -- !query 6 schema -struct +struct -- !query 6 output -6 +21 -- !query 7 -select extract(dow from c) from t +select extract(c from c) from t -- !query 7 schema -struct<(dayofweek(CAST(c AS DATE)) - 1):int> +struct -- !query 7 output -5 +21 -- !query 8 -select extract(isodow from c) from t +select extract(cent from c) from t -- !query 8 schema -struct<(weekday(CAST(c AS DATE)) + 1):int> +struct -- !query 8 output -5 +21 -- !query 9 -select extract(doy from c) from t +select extract(decade from c) from t -- !query 9 schema -struct +struct -- !query 9 output -126 +201 -- !query 10 -select extract(hour from c) from t +select extract(decades from c) from t -- !query 10 schema -struct +struct -- !query 10 output -7 +201 -- !query 11 -select extract(minute from c) from t +select extract(dec from c) from t -- !query 11 schema -struct +struct -- !query 11 output -8 +201 -- !query 12 -select extract(second from c) from t +select extract(decs from c) from t -- !query 12 schema -struct +struct -- !query 12 output -9 +201 -- !query 13 -select extract(not_supported from c) from t +select extract(year from c) from t -- !query 13 schema -struct<> +struct -- !query 13 output +2011 + + +-- !query 14 +select extract(y from c) from t +-- !query 14 schema +struct +-- !query 14 output +2011 + + +-- !query 15 +select extract(years from c) from t +-- !query 15 schema +struct +-- !query 15 output +2011 + + +-- !query 16 +select extract(yr from c) from t +-- !query 16 schema +struct +-- !query 16 output +2011 + + +-- !query 17 +select extract(yrs from c) from t +-- !query 17 schema +struct +-- !query 17 output +2011 + + +-- !query 18 +select extract(quarter from c) from t +-- !query 18 schema +struct +-- !query 18 output +2 + + +-- !query 19 +select extract(qtr from c) from t +-- !query 19 schema +struct +-- !query 19 output +2 + + +-- !query 20 +select extract(month from c) from t +-- !query 20 schema +struct +-- !query 20 output +5 + + +-- !query 21 +select extract(mon from c) from t +-- !query 21 schema +struct +-- !query 21 output +5 + + +-- !query 22 +select extract(mons from c) from t +-- !query 22 schema +struct +-- !query 22 output +5 + + +-- !query 23 +select extract(months from c) from t +-- !query 23 schema +struct +-- !query 23 output +5 + + +-- !query 24 +select extract(week from c) from t +-- !query 24 schema +struct +-- !query 24 output +18 + + +-- !query 25 +select extract(w from c) from t +-- !query 25 schema +struct +-- !query 25 output +18 + + +-- !query 26 +select extract(weeks from c) from t +-- !query 26 schema +struct +-- !query 26 output +18 + + +-- !query 27 +select extract(day from c) from t +-- !query 27 schema +struct +-- !query 27 output +6 + + +-- !query 28 +select extract(d from c) from t +-- !query 28 schema +struct +-- !query 28 output +6 + + +-- !query 29 +select extract(days from c) from t +-- !query 29 schema +struct +-- !query 29 output +6 + + +-- !query 30 +select extract(dayofweek from c) from t +-- !query 30 schema +struct +-- !query 30 output +6 + + +-- !query 31 +select extract(dow from c) from t +-- !query 31 schema +struct<(dayofweek(CAST(c AS DATE)) - 1):int> +-- !query 31 output +5 + + +-- !query 32 +select extract(isodow from c) from t +-- !query 32 schema +struct<(weekday(CAST(c AS DATE)) + 1):int> +-- !query 32 output +5 + + +-- !query 33 +select extract(doy from c) from t +-- !query 33 schema +struct +-- !query 33 output +126 + + +-- !query 34 +select extract(hour from c) from t +-- !query 34 schema +struct +-- !query 34 output +7 + + +-- !query 35 +select extract(h from c) from t +-- !query 35 schema +struct +-- !query 35 output +7 + + +-- !query 36 +select extract(hours from c) from t +-- !query 36 schema +struct +-- !query 36 output +7 + + +-- !query 37 +select extract(hr from c) from t +-- !query 37 schema +struct +-- !query 37 output +7 + + +-- !query 38 +select extract(hrs from c) from t +-- !query 38 schema +struct +-- !query 38 output +7 + + +-- !query 39 +select extract(minute from c) from t +-- !query 39 schema +struct +-- !query 39 output +8 + + +-- !query 40 +select extract(m from c) from t +-- !query 40 schema +struct +-- !query 40 output +8 + + +-- !query 41 +select extract(min from c) from t +-- !query 41 schema +struct +-- !query 41 output +8 + + +-- !query 42 +select extract(mins from c) from t +-- !query 42 schema +struct +-- !query 42 output +8 + + +-- !query 43 +select extract(minutes from c) from t +-- !query 43 schema +struct +-- !query 43 output +8 + + +-- !query 44 +select extract(second from c) from t +-- !query 44 schema +struct +-- !query 44 output +9 + + +-- !query 45 +select extract(s from c) from t +-- !query 45 schema +struct +-- !query 45 output +9 + + +-- !query 46 +select extract(sec from c) from t +-- !query 46 schema +struct +-- !query 46 output +9 + + +-- !query 47 +select extract(seconds from c) from t +-- !query 47 schema +struct +-- !query 47 output +9 + + +-- !query 48 +select extract(secs from c) from t +-- !query 48 schema +struct +-- !query 48 output +9 + + +-- !query 49 +select extract(not_supported from c) from t +-- !query 49 schema +struct<> +-- !query 49 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'NOT_SUPPORTED' are currently not supported.(line 1, pos 7) From 3d85c5489524cb7922ee377ced767b02b5e8ef74 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 13 Aug 2019 15:51:50 -0700 Subject: [PATCH 058/149] [SPARK-28700][SQL] Use DECIMAL type for `sec` in `make_timestamp()` ## What changes were proposed in this pull request? Changed type of `sec` argument in the `make_timestamp()` function from `DOUBLE` to `DECIMAL(8, 6)`. The scale is set to 6 to cover microsecond fractions, and the precision is 2 digits for seconds + 6 digits for microsecond fraction. New type prevents losing precision in some cases, for example: Before: ```sql spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001); 2019-08-12 00:00:58 ``` After: ```sql spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001); 2019-08-12 00:00:58.000001 ``` Also switching to `DECIMAL` fixes rounding `sec` towards "nearest neighbor" unless both neighbors are equidistant, in which case round up. For example: Before: ```sql spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567); 2019-08-12 00:00:00.123456 ``` After: ```sql spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567); 2019-08-12 00:00:00.123457 ``` ## How was this patch tested? This was tested by `DateExpressionsSuite` and `pgSQL/timestamp.sql`. Closes #25421 from MaxGekk/make_timestamp-decimal. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../expressions/datetimeExpressions.scala | 24 ++++++++++++------- .../expressions/DateExpressionsSuite.scala | 16 ++++++++----- .../sql-tests/results/pgSQL/timestamp.sql.out | 2 +- 3 files changed, 26 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 4834907a3691b..a6d6545b81186 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1739,8 +1739,10 @@ case class MakeTimestamp( } override def children: Seq[Expression] = Seq(year, month, day, hour, min, sec) ++ timezone + // Accept `sec` as DecimalType to avoid loosing precision of microseconds while converting + // them to the fractional part of `sec`. override def inputTypes: Seq[AbstractDataType] = - Seq(IntegerType, IntegerType, IntegerType, IntegerType, IntegerType, DoubleType) ++ + Seq(IntegerType, IntegerType, IntegerType, IntegerType, IntegerType, DecimalType(8, 6)) ++ timezone.map(_ => StringType) override def dataType: DataType = TimestampType override def nullable: Boolean = true @@ -1754,11 +1756,13 @@ case class MakeTimestamp( day: Int, hour: Int, min: Int, - secAndNanos: Double, + secAndNanos: Decimal, zoneId: ZoneId): Any = { try { - val seconds = secAndNanos.toInt - val nanos = ((secAndNanos - seconds) * NANOS_PER_SECOND).toInt + val secFloor = secAndNanos.floor + val nanosPerSec = Decimal(NANOS_PER_SECOND, 10, 0) + val nanos = ((secAndNanos - secFloor) * nanosPerSec).toInt + val seconds = secFloor.toInt val ldt = if (seconds == 60) { if (nanos == 0) { // This case of sec = 60 and nanos = 0 is supported for compatibility with PostgreSQL @@ -1792,19 +1796,22 @@ case class MakeTimestamp( day.asInstanceOf[Int], hour.asInstanceOf[Int], min.asInstanceOf[Int], - sec.asInstanceOf[Double], + sec.asInstanceOf[Decimal], zid) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val d = Decimal.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (year, month, day, hour, min, secAndNanos, timezone) => { val zoneId = timezone.map(tz => s"$dtu.getZoneId(${tz}.toString())").getOrElse(zid) s""" try { - int seconds = (int)$secAndNanos; - int nanos = (int)(($secAndNanos - seconds) * 1000000000L); + org.apache.spark.sql.types.Decimal secFloor = $secAndNanos.floor(); + org.apache.spark.sql.types.Decimal nanosPerSec = $d$$.MODULE$$.apply(1000000000L, 10, 0); + int nanos = (($secAndNanos.$$minus(secFloor)).$$times(nanosPerSec)).toInt(); + int seconds = secFloor.toInt(); java.time.LocalDateTime ldt; if (seconds == 60) { if (nanos == 0) { @@ -1815,8 +1822,7 @@ case class MakeTimestamp( "The fraction of sec must be zero. Valid range is [0, 60]."); } } else { - ldt = java.time.LocalDateTime.of( - $year, $month, $day, $hour, $min, seconds, nanos); + ldt = java.time.LocalDateTime.of($year, $month, $day, $hour, $min, seconds, nanos); } java.time.Instant instant = ldt.atZone($zoneId).toInstant(); ${ev.value} = $dtu.instantToMicros(instant); diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 0f4627ea78649..e75721d26b6db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -939,8 +939,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("creating values of TimestampType via make_timestamp") { var makeTimestampExpr = MakeTimestamp( - Literal(2013), Literal(7), Literal(15), Literal(8), Literal(15), Literal(23.5), - Some(Literal(ZoneId.systemDefault().getId))) + Literal(2013), Literal(7), Literal(15), Literal(8), Literal(15), + Literal(Decimal(BigDecimal(23.5), 8, 6)), Some(Literal(ZoneId.systemDefault().getId))) val expected = Timestamp.valueOf("2013-7-15 8:15:23.5") checkEvaluation(makeTimestampExpr, expected) checkEvaluation(makeTimestampExpr.copy(timezone = None), expected) @@ -960,13 +960,17 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(makeTimestampExpr.copy(min = Literal.create(null, IntegerType)), null) checkEvaluation(makeTimestampExpr.copy(min = Literal(65)), null) - checkEvaluation(makeTimestampExpr.copy(sec = Literal.create(null, DoubleType)), null) - checkEvaluation(makeTimestampExpr.copy(sec = Literal(70.0)), null) + checkEvaluation(makeTimestampExpr.copy(sec = Literal.create(null, DecimalType(8, 6))), null) + checkEvaluation(makeTimestampExpr.copy(sec = Literal(Decimal(BigDecimal(70.0), 8, 6))), null) makeTimestampExpr = MakeTimestamp(Literal(2019), Literal(6), Literal(30), - Literal(23), Literal(59), Literal(60.0)) + Literal(23), Literal(59), Literal(Decimal(BigDecimal(60.0), 8, 6))) checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-07-01 00:00:00")) - checkEvaluation(makeTimestampExpr.copy(sec = Literal(60.5)), null) + checkEvaluation(makeTimestampExpr.copy(sec = Literal(Decimal(BigDecimal(60.5), 8, 6))), null) + + makeTimestampExpr = MakeTimestamp(Literal(2019), Literal(8), Literal(12), + Literal(0), Literal(0), Literal(Decimal(BigDecimal(58.000001), 8, 6))) + checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-08-12 00:00:58.000001")) } test("millennium") { diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out index 61fef6eba41a8..13a1d09b71b76 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -125,7 +125,7 @@ struct -- !query 13 SELECT make_timestamp(2014,12,28,6,30,45.887) -- !query 13 schema -struct +struct -- !query 13 output 2014-12-28 06:30:45.887 From 601fd458143b5f24f42c61e4645b64c6e96f3486 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 13 Aug 2019 22:55:02 +0000 Subject: [PATCH 059/149] [SPARK-28719][BUILD] Enable Github Actions for master ## What changes were proposed in this pull request? Github now provides free CI/CD for build, test, and deploy. This PR enables a simple Github Actions to build master with JDK8 with latest Ubuntu. We can extend it with different versions of JDK, and even build Spark with docker images in the future. Closes #25440 from dbtsai/actions. Authored-by: DB Tsai Signed-off-by: DB Tsai --- .github/workflows/master.yml | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 .github/workflows/master.yml diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml new file mode 100644 index 0000000000000..d29b2c3a00f28 --- /dev/null +++ b/.github/workflows/master.yml @@ -0,0 +1,22 @@ +name: master + +on: + push: + branches: + - master + +jobs: + build: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@master + - name: Set up JDK 1.8 + uses: actions/setup-java@v1 + with: + version: 1.8 + - name: Build with Maven + run: | + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" + ./build/mvn -DskipTests package From f1d6b19de542d8fb6b966d117df4eb5938be9a2b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 13 Aug 2019 22:56:53 +0000 Subject: [PATCH 060/149] [SPARK-28720][BUILD][R] Update AppVeyor R version to 3.6.1 ## What changes were proposed in this pull request? R version 3.6.1 (Action of the Toes) was released on 2019-07-05. This PR aims to upgrade R installation for AppVeyor CI environment. ## How was this patch tested? Pass the AppVeyor CI. Closes #25441 from dongjoon-hyun/SPARK-28720. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai --- dev/appveyor-install-dependencies.ps1 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index 8edd8f495b0fb..85e0df68d2672 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -115,7 +115,7 @@ $env:Path += ";$env:HADOOP_HOME\bin" Pop-Location # ========================== R -$rVer = "3.6.0" +$rVer = "3.6.1" $rToolsVer = "3.5.1" InstallR From 331f2657d9451ac9de85f576953afde187ff9bab Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 13 Aug 2019 16:48:30 -0700 Subject: [PATCH 061/149] [SPARK-27768][SQL] Support Infinity/NaN-related float/double literals case-insensitively ## What changes were proposed in this pull request? Here is the problem description from the JIRA. ``` When the inputs contain the constant 'infinity', Spark SQL does not generate the expected results. SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('1'), (CAST('infinity' AS DOUBLE))) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('infinity'), ('1')) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('infinity'), ('infinity')) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('-infinity'), ('infinity')) v(x); The root cause: Spark SQL does not recognize the special constants in a case insensitive way. In PostgreSQL, they are recognized in a case insensitive way. Link: https://www.postgresql.org/docs/9.3/datatype-numeric.html ``` In this PR, the casting code is enhanced to handle these `special` string literals in case insensitive manner. ## How was this patch tested? Added tests in CastSuite and modified existing test suites. Closes #25331 from dilipbiswal/double_infinity. Authored-by: Dilip Biswal Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide-upgrade.md | 89 +++++++++++++ .../spark/sql/catalyst/expressions/Cast.scala | 119 ++++++++++++------ .../sql/catalyst/expressions/CastSuite.scala | 26 ++++ .../inputs/pgSQL/aggregates_part1.sql | 10 +- .../sql-tests/inputs/pgSQL/float4.sql | 2 - .../sql-tests/inputs/pgSQL/float8.sql | 4 +- .../results/pgSQL/aggregates_part1.sql.out | 8 +- .../sql-tests/results/pgSQL/float4.sql.out | 10 +- .../sql-tests/results/pgSQL/float8.sql.out | 8 +- 9 files changed, 214 insertions(+), 62 deletions(-) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index b2bd8cefc3f96..a643a843a5cb4 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -161,6 +161,95 @@ license: | - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. + - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: +

1t;Sj&D}2QPE93Qo&53Hpxlu5#cL$Y0}{1cnKbj>b!=noKk% z0g@3oQ~A;|F21`zWga;KB1PeG+0)?4M^S~OQGT}J(I>qcEPP8XOR^#fNpZ#0MKf0}03}7c+5rYR_7Z9k} z*OGvyR#|5(@E5z=u2uUXvD?ntbAa*Ma2VrgTy9ElfT=(MSpJ&__(-x0HyOfbo*Bl( zC6kSdYPk(A;X}f(ffii*6=s+RkWpu|5*Zd-7Q;&GNLon7BLFZ-!Ieia!H@`$!Pr`L zK@S!a-&={QR)8jqV4HQbcS4%AAl;87+#Scr2Y73cK|~S1Tmcw3B;yqvyK^D+C^)B2 zpLVObwY8PCM#uO`h`UsBNN|s)Mz79&6z<3N+lc2!hWj4Td zgK0+-3BuxD6_Qn5%>Ymi_Q=Rg6rkdr3b7_?yfA&RX2t+xCj>pXR)9yIT9o;Su`bSq zR745JE7WR5|Jr6aKV-aCcOr&l9-TdV76$BI06Vz9>jWU#QOX-d57PX>PO@1%} zXH<7aIgZ$)PGPVm*zHx0#FZ$FT3xQ$HekQuh8xU*0A?Vwc)7>-h{hup1wf-(%B4Dd zmax_e$N)(0yDPz~zxW7)28~Ft~R}(D&HL(%sol+Eg#?WQLG)J{nFbvmAcgN^)k(NMpq6klv znijQvR3FTuF^IYWMAj4(y`8iDzfA?Rk=f*o^Q43yW&;RBE4x(m5}DYXicLEtz!-&tJ76T zbrwK;7$298HZx;^_cpp|iaOJdEod@rGPNp9!bON!kz9n~ZAuMW#O{o|n<*twvet!* zRU3`p#aLK1Q30uyQY_os+sie{Q_>1a<}jELFc$8EDPj_rk(mwPm4><2c(Kp29)ZQs zJ(8ls+?3ZtYNJBjb%Spi^Z`JvBPD@8j?C`C_O2$aA#LyM1wAV0*#-|#>ruQWJ1>qB zj^;rRdJq~w?CSd41Pazfz7n8TB6h!S#L79-y1gJg`3*S)@FcjQum#5sQ;jTXVX_LW zpl6;|a^$5NtY|Xr4r2`3v{*7VQDkK@Fih?aU>;ElW-i;`dou>zgkmW2_W3XKTAlkJspOL zn;D!0Kx;XZ94C;O_vuyeW(M|zBk+x-eDLUO-+P`~)%IIhTo1_s2H<6ww{s$pg(6_2 zI4s#oc;Di4A?mQ;6&AU~2InY90O9Q^y{51>+qj@Cg-FJtM(OH7o!7XSttsOn6)-#+ z_%M5Ft+`pgdHduNDjK{jhhdvm>nJnOYqg2g3aC@`3-8bnrBmRu764M~6(`z#nTM=~JRWU%<^NB4OROE?R2TrV`1?S5)yuqC|S zq1uYePEGc0ZWDNc(H6O5l_}BpwC-?|wy^S$g*r0p6hag(gsjVi6u=8jC^dnZl&4{^ z4*)$%R;b6f5+oon)@l`eLWt@m3CixFkSS=i0z{G=zEhWlSYW__CGDcZt|lM7+SMK! zgSU%-6vbjnG10Nyh8{6wLP(P~hLT4x7OlpG&)++sG6Hi(%wB2PF>HUtP1GZey+WrLsN)yJ2BoQBo@unbt7M#@^FXp^P<@ zDBTJJFwYy;49-qs6<>3Bhe?z{XL5i+i~v0(Kn+$B5&+55L9B@Zkbp{)31WN7grL`i z1u|z@0ydMFF|>+|)I~!Mzz2`L_z0%!$b4C8S$@zk6^RQXJT=vmpR<}5B4 z#2KJQ9}Li>kVdJ!8N88_@IkhY$Aa?0OO_KzOEa6{>FqZO!#l7}F-4dtz>>MsW*H8H zeHg}o{7Q26h!Qd|Amaf=ICjJy$1aF5Qc=*HGds7ND&=!_;w1{4BE+;qvWM|CrI+vu zVsTc(SFn@@2w?6Y-1~1{5FS86Fl%vv*_mW!Fu-k#N3CiE-sm_p<243}PKX&G8?cz| zmGMr|%B6{IY7(sPWi(`LXCx@d@TOzc;N>CnOM>Q9=jVwkOULHZI8!sKazawAbLt0k^t*P z@o%3w1okTu#8#qVPeDKfpvb#VP2d}Eyb&dg0)s%H>gBNDl8weeGT)arNR`S)vQs?EX1kmKRu9XNfjdn`D3F^Yj3eHsP#hEi_w0d#H zc)O{b%~L{%g`rXLC|0aur#Vnsl8myV)g$^@Hb&scu1HJ-lA8|UmRqG1*hV46`U4QY z&T~T{Cm_oM1iI4*H_mk%7@2+rM*ssrMr;7yxKM)OnE_Dj2Gn@fh?53JgV?urK2ryn zjNJEYMLEpQ)LyRKdH;ObV z)I~79+(tuSwmE#7+!4dji&(3bBL}fF18nW*rbi=Ija~sLxJa;5j75!wkir!`vLu^f zyEtI&&dv@Y$Q+)#08WA?&N2@d0@JjP02B~NWH#Jp6J)VKl7r>3kA1AOpZIg9CYE5t z9zYc0S|L5w9gwEN$huZb5e$O3tagwdI}hl-=&4SZ8b)o~3W?G+pAQ+Riw0oY6#vPIciW%FUO$ zm>?j{zK1|V09VHI9X(yvJUbrrn%Wp4@L`t%i5)(})L2B-O79{Go(jZh>}D8)01`01 z`nFJ6@L=4xx_Km!zlcpfZg!kWWQz}EZ10@>`A#63LzUt~Z(NMY@gAAPQ<%|TwD6*_ zjCi%M6B&BE3A9-R!(*XHB++X!)-rW>cQ-}0*K}!B6OJAi7Of~V5*(&h#F<)6EQ~?N z06_aHO#~`<5n$Abw!Ef0qF@kYo1Lh~RgFeg1p2qLpYxcvwUDM_+3vvLw{pEKDzWL+ zsx(RFIAr##snePUsax)qI8Lo2{y2d|{G z5Cx;bj?88e0Y&T|VH_t4>54vS=&1ms2PqTUQD>^P>3Fa|kHFyl7 z*e#kaLz77z!+a8`Ny#052(!n>JmxX(Pou1rKnE&04_PE(_>PhN!&FKcoHPUsFta`S z(T{fjWzNK#w4#a&rQ+R#6DD5MzTx1a+_djZMO48FAP})y8CK!y7mjp+mBQYnfy_!5 z*U*lJ1W5BaV(e`j(!wbOwhbPU;R3)^qh}Jakwrt4iNb^Nb5cE*OuQ;c95 zS%{K^fMEk9E{|%$aUt-CQps^mPGz;CM^KaVUXQjt3GtXlz+gwu2D2g>`0C7yE6kt| z>5&<$a)!+VK&cFw;8+3}taE8EN(LY^G)XxuTATf_AWIGctrlGG=Pa@q;blc)hcS@E z@<_U(K~NgQTM=Rb6ezqfrpG*@LWrzErdCbz{M4yaBhL8(Zw|Cua5>x9$3+@4A3JDO z$mD`w5Z5o(2uEcESkRz1I;?J3W46!YTy|(m$ex;o89+{UI&?74XK2*1 zq&eAVHSoEkW*9ll03%B#SkaW_VE9_801Li47`_z;P^O{J5({w->LW;o*}iH5@Rx*; zY6B$pRw5WI!U)hi`bbZi!!c6rG88d6T2tiFAnFcU@0ea#PQFqof)U#gf=P>!z#g-6 zdwZMLq(Kr6F3k`#0|q@~$OHpW7CMw{?$pvKYaLxHB2_e=gPp9MZ zKHl&2)1FnSvwJj|vxkE=^H?Yv>KGszC?QQLrbHW5*^7&OU5_iPG9ZIfo#iDz76&5l zCdpSaFEgl~Wn%=6?25#>W&A-JeW@FK4JGFGAXBrKC*G27xojM29hZfe11WWVo_RcJgSNC-$t?a?* z5e*DDWU9g8qfZD4;ghVcgfzl$xZ#FzW#H0U0NG>SA1WvUn^^7pjWO&tuRb?G=EnvzaYhUFh00|z8c6hMgpxW+-bh_h?IT3q~ClE1d z$qet&J`aZQHo*~uiIS+apc$b9h-UFMw8^oNWh|X_&7vCO6=GX2UKH6My$lcUcPC{X2bxkN_N6T zgV8`J+UON}yN?f<_84&i)QSjzvAgfk>AK#Zp>jo{$*~--S07fd{Mz2$wmX+L=582w z+lqIL4}S22$5KjK0!he{OwBJCjH{T15o-k*TuKpJ|Hy=i9RMQ~1vyHKhgkv{yqLJ^ z)+t7WPVtCFsZ~u532MT%ax;zL)1@i-!UC?=@y^)+eMY==n~k&JQi@AP6h z_xU~I+i$P7CfL4L7YrJAw-=c|?nl zV($?zbSfvygYz!4;bD6B9xvNSvqeZ6BreiQ1A09Yh0M>JA>Bq{ zhjb6F_?*NC=h>5wn)IDI`8DNTNVNi=Mk>||UY~og%8i(5oOb(KpWwMAvB!-bjWS{j z&PVkSKJ;SYg~#%T0ZZAH14Wg%6Mhxf*{7UTQ!_I{6z`KR(W~uYdFX6VP5uRXNnMORj88&9?^IN$g$|H zG9I(Jrfkz{-I-0jHy1=V1SwdvXoU|KQ~ECKY#_F9Fvv9q&>HVlG$cj#kv3^k|Y;!_0X~k5UYuN0!u1FvVX9kmz-#d zHay}!gYC>IR(LU@tgsC*+y~=AMo#K#8lfN@d(K9QrAJMO6Ati*K&dr6ISL!ZXDCIp z5ub$@UMG+`@|)c7w*N5(iwUa`nAsBRnwY1wdxR$;OPIvqoYfQo08((mEPPv%c~pkK zXsjS?XpOj5LI_W+I#CFQ<(U^3{CYu&D+$QTheTxw z0$gen-5`cZGRj~zdMjx==VaQvS%qzh?U1!9%to1fHJQT^_~gSV(U1cPhKYb)NQ#Cr zS|L3m(4+uh+@aWc8vFFxEx2Nq^X}nFC^a41zD@^ zU|)A@6->y;>J%^wS8X;y*c!r{`u%V~s0s17ySwZAg&{d1j0A)P*PE=B56;PEYKu@B zR~h-+l8mxho?&@fFEn$-=!oUL0m*(sOxpJLcA{Y9Lk5U|RRxyITQgp;+hOxg?bo(0-9~qaNj-uQkj*F&z!X>)c2<3l2b3t) z1#?8QN5&{2C5E#ZN^x^hQm1%y8<>%_ctp>JfH3mJBP2Nr0{DPxm~RiJ&3*XDJBPvv#J(P|GgEFsXXlUngRXS~BTGJSFFBm>}! zQ`2Y~rjnEDv=4UzAS-zXiJl_JrNV{RJpvSX9u*m4VLaCRu?)sU4I~ypV0_Zx6*OJz zXmx8T=e1^70O2WJ=)S{}c-4g{JPBof)2UVh(}gUVnbgMA4B#;-Bt;?17aJ@JW8G^~=pu_l?2ot+)u^dYuA zr4Hq7x7}uCF@GW;kWxr+!0@F`fmF1@uM$T#^yY8TMy7QYjtssDhd?v7JvZ}`2^Q_n zZ()YUcu~qvdwRS+(=a;Rzt?1*bn0GMxpV}Q_M2|H$?q=hfvxcydE8yZBmpT@T z3Uv;$gc)Ns$qZ(r2Rl*Z_%JVPIos_CvhGR{fXj007dYYU!gNCm6}ED?(f>mG9_ z%1ouDQBd5f<5GL1be#)QlkYOT+Ck><C>kZjxlw*(tVarG!WE;sJ*+k6h@Fq znFoV$D01DOoPyI}89#aGbcZ0T>VhOVL^(Y=?Pyg(s%bI6)sW~@U2B*H^|EO4FB%0P z$mKjZc8?(=NcfWVT$OOGD5acm*^_u2+Bf2d1Y~3qLQ6<+?3#ugO~^7vmPRQ{o9K|? zQqXW*D~z~#B(_OFkW-}Aibm+DP{IHb>oGX(3yM1|A+64WiA;^lUa?_fQbHqh;;m*F znuKUlaK)%8V*=F35Nicj?=1g!cD_fMk$Y+z-LYFpLhN8AT>)ymsS8e2y1)?=XnynP z*y*=RRw&vvc|Sj-#TX0Zsx)xE0~D=;phsK?#HwAWRx^BGf$=nA1lcL06rX{>_)}S~ zNZKtB<#!gYM_tg1lGcT?<=vKUY6RKpG=kuXrPp59$E}F%I$)?^M&Vs@FDyq^vVr5u zBm>`^>`>r<#&%5E{XrdI{p>qRrfDT(46G;i#P8sL`rbd z^gdzi6{mfSZfb@w22iHUX@=czdY&48WWtV@f`lR`#{$G@^3;3mJX!^KZkjMwASrYb zk|7AofF@TQR#~!6a_ry~N_C^{bL9RpqkUtbUd|eP8N8)Y&Kb9#e!pGR5hb|3i`zO% zxzJpi%E@)Gk-4|JH-n6X*?dP0T9Ip|ZnLi6Qn^ z?r3u^)C4fv&6-$Y&>J^KCP~}VM;0@KpqzTNP;$F3)iK^+{l2@pm$dvB8x}`u2O&a4 zAB?LX*{G|Fb1`>nv-%Eun3==Oc>kdpPL;tStWm1p*UnV7Bem`KHoZFjP(I53lPUc> zY2jT@MXfSq?7+l!--1OM6TwcYN*So|B$kICVe+_p%!A=D77a4T4+r>n~L^~1&f^KRkVz#Il}!SZORZ53u?VYVDRAScz;i4*FTX7Od!Es0(kI` zPU*AU!~a~@7o&&0K2X%GrqcymO#J~VW~0mPIv!R^qa9z7V??7cF~OUieLVD4K_q5& zJ!y2QFHZATmJVxD!m_;|P9!u{D)1*688~pyZ=ae^cImesMi~q(eZ&%Mb-t=zOS4X0 zG?#hVO0I)0jDu~uSQW_v8x7cvFTV51+kR^AHhc8)R!n0N+WdPK~us3@R#H$MVKP;S$|tE>}QfyN+l!RH_AmN9%|7|rgdnX zAkv;opc9U*n@$gD=hRlfsW-3i}o2^H^xsKvrS~@0fMFIQfHL z%y_X-6hEAk5gN+M71zU>*6gAzhy@;P=eq#WqeJ6pdt*N0cU<+&z5NtQf0OEHAXXGP zqQK^SFCpMybv8sJTr4feeQ98jWd$zjgHqnF&C@9ho4AL5x=b@v_+$UrAt(WuKSBjb zzy^_v-?S)a9Y@h>Kfpkhv{HSQ!;s^Gq$FyyvBpNu&sB@|$r>FO+0}{X_H}W@P|Y=q zS8A+RtJ$Z(>YLD!#+n5`2d==gk)0-U@6WjoIW(l9%VNP~nae1b(nJ8IY2`^DZ;7Ow zRF`tP|R>d({=3O_?ilq!w+NMr1LO7E!ZN}H@fwY=X?Su0}w-Vi| zeIM2M+U)mlJ~UCvGSk!_lEdNZiTg>X%Z?MbK}qp$P~Tt(FT~v`V2#1c+xw@?6>rfw zZ?&VXsE$7ciCBXeb(i3kfm$IgCcq1Wzm-AxLeg!l<~RknfaOVotdL1$6h&j{ zvE)ejB^wpTO<#s3>e7@weF+_XvdfGgEXwbsQZFSu(jn@#$D>77YKvu;MQu`$&U*~!L0$m z_C*T?sO@pLxU?IROr$0ezQw2yGm8aZ&jJj+;yA@PRn2Xh4^_)#qOs9_Xo6DYL25>{ zelSRh15B7C3hXvaek&IEqf!FYI;I5_sqYnD^t&Py*}@V#_$^;j+x9saz9Nq@D25#J zm)yI7U{Tz|?5H6T8A)HJjurhjL<&}Ps~;XLBcYC{zkN;+GF|$~a{z}6GHxzp2T487 zcIHUCI#Dn~81MX*1cb`37Y$|^{#{J7kvKIlGVz2K1K=2@9_C~uza73JO4>bvrRG`@ zVr7A!|1 z!-ewQEhY*)92w&U2Mw2fQU^o!Z#u7nE7P|&%zqsO)AsdGU_W1lBf<@@Oi+J^=zU)` zO443Qg4=*b4O(J^1jDBAh=tjWIsk)*x!uq!RSKY;&_Y9!b^(odNHb@Ar)Qnqsi>h8 z#V}ZKgtL;W+co${fy5IrDmZD(u6-Ts>^V33xKu#F2yjir{d*ISxmr%|( zO7ad!PjvxlNIK1Kb4r3u6(v#FAP@5*Hq*TLZOFa|9 zx)n=SNU>4HX;MgBdaXu_^0?5*#_2|n%GHU2?%of6b?ckbPW>^c?Uxb*4#e_#nYUJ% z@YgJ33v#g~LMp^ia+r=@{IrB;Mr!!V2rMU^ydBCWY&vgpv{Qz?=-*7PeJ932M%CZW zr;HOhE7}$TEm#uzx1T#Exr!t(OLarcYHDC$!=cWoqdM%6Lsqh0%pszv@T_sv0%IZC zNM7QU$?P&($?KPQTawpYBF7`KXb8x{7Hm?eBpKL{00oPJ25kNcYGM-gg_9I5bVFG% zATef;c@Q#mk*DG?o#4K@Z8$wWBRoFRGEyS67VDz+kHM~s>&+I5xbd)8879~y{_wJ) zO`3Sixx51cl^DuY4?qsoWrB4W{&NzTj0G%s0)U+aaNW*GPPUMJuf>^!wn~J$G7=R3eX!@6h90Z=+F4Y1hE@SE3^QZI8R__$#EZ?9Xoy@SvR}Z+ z=sZme30Ecsh4>Fer+4PUP_)YyF5gEkQ8#rUMrJpj5dS09K&dnlQ8^ew)LJ14sk3UWB1WRxD z8~!Away{?zb+PoWLjon*uzG(h($k=z`P^-QMH-4aHpLI3_`rtYjvFI{;p$cmmi*q; zx$4LIoaZsJcJ+%TH87C*Ej!Y*^MS5&qK^wq3yxx+==Qk6W1C|B7#tvsNGjl9~xIp{ZIU~WiQL- z1T9=Z2;AvTh;}F>c>8f%9S*faD8N!SIHh8Y#JbR5CUP>3Dl>t zv$YkXf(Ol=b(rEIY0Pm#AYB8G=k$vr!Hsxo{zsX~x>rhWnsbla0PpKwb^a2fXoU*Mdz%0M5Z0ZSS zX~kMbd;!XF(1ax9f@-R5A&ROt#-$U+Q=s|8Mn2a+oajqX^RzrEeWob|BI>@nM9SsR zOjYxQx(x5g>a-^Psrbm!t-X!<;awYW2K=WKmP)4j?hK#2{VZp!e;8p&z_9$bFE!d< z@1h&$*!DQDOmMyrF;bcmpK}?b)0_jr?Ld~p_W{6COp5;>Jw7WDpsh_e0aE*B!|M6 zE|eFoNCWjIM=|1-3!_T6){?xCOMLWt@^~k#Qd>^ zgU!erVS`J7P<+!hh~b`E8dlIgqom4$!(Ubr1x2_mLL(gu~!nfk?Z;#R^@$gaDAEE!SHdEgZ4&I7Z&p*z5J+fx&E z(Osb9MvEu|zv=pjJzJuf17A})TjsHFyu8x)BqoOGX2BEYMH2|Nyren90()1 z6F<$g)t{Qs=dSS*nr1A( z8NNk7Z&E^R8&Kal* zU?-Oz%mUnW_$&_0M}l2<`qZ5N=IdDdz(3VSVkgS!5*W7QT!!RA&I=w|xWG;Tew55w zN`6phvAu;tTd^k$TWh8>0Gd_(c>F{*t~`UI3IU{?jwx-glVpclLAvkPs^u}5UV?vl z*d7P}z)f-nekCG(q@qkx+c{n27pOLgpBrV>_TZ4l4_~4_yz1mU07~TyUMr9fKG=L! znDVhq$l~kB$8#(vh!U&+@5MI{W17Ka(IgJ(W7TQs+h85jw-eHY3~*)@qkB#BsHaq2 z_P#L1N_=btksf^pBn{tLAshEz5r-^yZg_b=CP8#|xlS0Jz#9%B7$ir_t6)|}>mTkR zg$VJhkhUs&TkDLxfA1{b<_jrV;K}`^l$Elzd>cmSapPQedG$yXqYa%z-e!IFciLMA zCu){-UJMwCJP;A)F_Oy1ZKj$zv%qv9-_^6Ji9b}cir&4P(@Cti6nEx{L{e*Fk(Ndp z+Fo)@W->Bh>aiC1)IR7k&t)i zTu_rLQAG4u!YaiJ+#x|YQxy1+7NahseH}7ut=3B(n8I-32BD}9f-NGpZWBLfBdA8yE`k(b*WAex@RUk)$u;Ea!N5~9Sy)=2RsZj%;xiWLS-sR z?1L{~H5MTje}=QFbux-alL+CQcJ*mXr3bA{M}$4d7dJd$P+@cF6IfJ|LeIp9ES!T= z(rKm7FCJ{Fj%cx$7j+)%@7gQM5fe5`=h=N;EiZe2$t6E`@4Ubv45H)eLQ^Y_L~W0E z>ZY#xgPRi7pfl*YRqfHlnlz_a<+}TSZvRTCh-CY>8v5`x zhT~kCB@>Nkf}UTkzKgYxMk4yavp)OVKpHg5>2Pk9Ib3MK!MJ;%C*AtH;wfm(GEt)EPYi}5rV z0IiyyE)W3v_02hKdgk5XkH>>8cy|JS9*ka_t`(YF>Xh{dw`3w;lLC&7ngdqLu7sLt zGo7IL%wh$yjcq{kWH0}*k_L|j$YlDm0XS=z_X2hgV7_-)pLm;5rn5*b4TsWtFggD{ zUH|)|1RjdQj^@u|v+@(!*D4~4Ui<#VF|-mhT=0Tdf}m4WHJ0zF!8gYydC{VDU~*Bj zGO{nb$h2Ub18U8ZAbEc+;7>Vw(SqbMGE*kxxG}q5Y#?r?t=H9dQJ2*XEp$%|hA+yA z^$0GN{`V+MpVu*&t@A;tSGZ6G3qUh}u@NS7+0KpZ?ou3+_+yFXgSwv+iulTqm})Nc z1Oga&?x%Pgg>)9aT6hwK9As_!RriiZ?kIAntB-)yF(g^Iwa9#=-*6&v`oA5#6Sq?t zO3jxw)QsWGi7m!IZ^fI)hVVssI>YmXq>YR^H~IYD%|D}b*DH=h`x!SWpv3BF^S!q1 z+r58%TZYrfLxklMJ8gEzrJ7jw3pp}Rc)M8r#eOA@ANER>3;R0M5I<91+@txl{K{@? z(gm$17usYGNlMP!sXii!MI`W3!0&WT6xn&r`p%N^q}Yrvho*WhuUeWULBpWkCU>EA zJ<-390(=;5>$_Dq9p!Vbs(h%fIe*oXQ?2~oKly%5Ib)ZEMQAdWa_oxJ37R|giK$_e zV1H4KB*q3@=?n~eTMkRMQQZ+*bYIhCM|P)Gam#aE+?e7}HnUI?=}rK6kz>z;q>v1!%ZPtL66c|d##eU_ zRa^~s^|_6EC8QyWovc~&2@BjZy@jfdk_J9jbR`yyaNdEnFd2sDhk>%~d4ZsRSLK_P zlT1uBrDc9QO`2TYKHg^+wpU+zXumD3X6GfG3&NQT8@NHrvr}9e-v-u6l9hC)&V<`> zDkO8zRa5cXH`A|af@<=;;c@SoH49<|WMyt2r(kg4o1Y6ndw*@AsIOT5kcR0rv z!Pzp!JZ_V+WR6IbJKZ@VN}#DMPjo-SqA~x3Eh{hAzyzy-3WNPKV1lEJkqoG#XBt2QSw*^5+5b2J*SXeBO|Abr#>N2Z z4SIh2KYe6Id5su@zZoVx!uHsVU>yu+{NI+_WAMutt9fw3JztmF3eN=6jS1%7L}8#y zyDyUOv)gQ$DN5W6@g?VAu1pAJVK|7IrFrOPTSqe>K(Ue6rbF6zujTiy!+H|YcuAUb zYrB-BWSdPu5ieL0VDLxZYp2_s6}aa*9NUM>^H;;9O@#s#s?aSoTZrgyY3<_pLWfrE zHxv{0kH_j0s;nAziqOkZIGY{WuVNuAM& znDw>@OU~pcNUZdh=74pi0mL#zzoeo?wUSl7^YWpboT-%D{c;4`9P{uA8wu27S4qj2tzhAElGTH(xAm zM#7LNzcDxFWCq*ssv`{5vpRV{s@857RR{c5;;w6Qua@7^jQr1JgQLl`-8zglHLh{^ z6=jbYLaYUk<(oNYuK#>gX`T%=HmG$#hH9Q@kLSzh&%ykumH?jSd+tUTEq8JVmYLIS z<3h!8p4Jg+(}+Wn%!=35e>{hPwUjsMTrWesywsf68PvQNb*00|CnnXn^u>D2560K= zChLBU`j=O?M>81e-1PR1{I;bJtUL?}y~>ldK0 zVtM%KzY+iwEJLJ<7>iD*;P{ayvvKC^mT#kx!N1+I^0sy)%TQ43&$&9*enDCnOruvb zGH?j+AtS22{Lz7**l zVC;Kcr~Z=gEK~OkRe+peB{?xcEnesy*vwg2t})(`guS=3rSS32ps{ zfMlC7caa)Gy3L7(h-{4a1f4_yipUY!u^8Fm03>A;GY9JP-_`DQm{#iPK`cfl2xo4Y>DX) zCm@o}8F!jA53-qs2Odfgx052=-G4+zA3KKY<+?aeL0&!I$6r> zG>b?r8D@e=cHxK+0V$t56+)VQp%kmwW_=GzBVbD&)t#$HI9r%=zPY+Ye9E=5Rp9xI zy^@F~sK(dX_qFPYm8_;4W}MCFM^u^WKc>=AE3nUfMsu zN6`!Zeh!i5-3n4ki%r3q?*RFanjopZ*JuAUHp&(7bw7^&*W7;15r6MM+@jh#x~YHW zWUk5gTKDSjc~oX7U8&!Xjr}W@lmh8-;;>b=*oc(kOsSAF5^~>UE;VM~BYt+?o(ENy zd0A43rpD=hmDFiP5>Ta)c9dv*n8nC4jumrZ@>3>k&&_qcBl&a@$K_E4vo7LZeB>Cd z6yad{S2&M;?E_}MQg!-Ft3a7pbeYlVmSxpy$XlCkbDRqY(Q|SkPrqhN-GU6;mQqb9 zwiLtL{u`^MWz9Q%2nCYnfGt#?D-J3#>Dxgl%tC*{d`JBR40C<}&65eCYPzUcdmFRv z4tQE@Vyd#Ag6m@+(#EFuYG;a=o;^`GE;x-QC1k-7Ui^zgfF|%UZ?Wzb6P-)JOgBBR zZhG9;z9IY&B9T+VHK@jTGo(_sFJ{GO*Vr>F$;MV|wyd)I;vm>ryKJ^svh$v|7tU^H z$@sv==0=886=_I6D4Kd{Kmdy>IX+R0gq%Wsu@26lIfxv)8GnzdR7z#XA-Jwotw35q zk*Y6pj=y{;#voTXyciB(K`KHsuhvQ*?a6RzRaSnQxPWk=E4H!?s1v zh1~fuE*dNlbea;O7}}U|ctAp5EEcsI3PEhaAX?SdL9;H4f?su(n-+hIIlgtrX39A@TH#jH8i$m3w4&?gj|k}BnOHbo7{r{6C7!KN0Jb>#DGg)P?0c@3_3%D>Y5 zVkz{~P*ozO4j+5S9qJ)^mykr}UGc<2ao1(H^gN(<45e~}{? znYMr+epK_~HWWAHc;KKoHxV|ps&k!scxQEnaS(n`^jFB*uy@}b`-U~}skmp?Vrnlk zi!Q~D<>8|%;3r4V>bBNIgypuo>2_NI1^M$wMd;#sZws~QxRC~&@dl4XJ=EHAtO?O= z&Zf#`?FYrxXBf4$=wJ{qTbmfN;&20OY@v`B%~4%wC~S1<+C-v+1cRzo-Ofh*C}ejy z_gT=9WfpEUNjO^dnT)}xwaWQW(2`wj-cAiuqa`ZwHfqSooQeO?BYtGKcHj}KDC1zH)^DALC zCo%3Zuv4Uo(zvf|^?3PF_)%}E!18`offzk`Y*8OlXtR3aOEYO$pvlo`&;cpBlvb&+*jc)B>QJZ31D;r5 zJqZpckxKPh7}Nk)3KL}<3GtP|ytd{5J%o=NHFI#A^}UFDLMsM{<#CI2bWy{;0< z1b?KRhGGJ7eWJnd50gzmMU0k9y|A((geW@q(b_>C)C zv+#-rxPrnmFpg&rwY2>R$e`_~e=Q^LTMV2#f%zbSaU!`{=GT$z$!Ajp;D==eksQ0R z1}6fBanQ2}Sy1%k=kO46k4Ts)O3@iciDtP0SR(G8qeUQc{jX!hP-m{hl!^t505mV&6#_*r0%!=UAZ%FsbfW7@h+&XS+X= zy$xPnr&MT|*@ipLKH2dr3xv5_n(Xn>23y9L6YkUE8^~LDb zb&aI4W3Z0}x&O)7s?$2%rfL8Ugw0;-~hHOXKK*mAKv z%UxMD*q50aU5t6DV2YTLa?(mY`){-8(MHmS=%As8XWGrLewaka^K0YtszLD(;HR+W)gvl6A_&nM?6$&7DDlzMQ)wUsk zw&dCFsZ}+sc+C{M9wSQkPJwv2P+%2YL==zVVzdmN={3?=z>LHQ7zB+@){tUoYc)!~ zP}GbyJd(Y}WO>@roqQU)37u=qz3~HGg-9j)ycS-Zkja>MUJP>&Zf2`!E#|6CDZG9{ z25Y8o%q~}x0~@B7>?T5-60&{TGI6_3`n@ZJ2pqIXeP0z{bGVF9wee<3E*b^yVck3T zH!jGEGBW)YcL95#EP|rIE&VpqCOXBn=qap-gcRXvReG1J7Vi}rHWFfrSJxDH5(U%% zBm($e)_fp@U4Tg~TUjyj>VzK z*`s1d&Lm=iv3+gIT8=h{M3a@FZ0;0;t;}W&tr3zl2XzxXHc1b7K|egqjWD2TUmZwO z%@chL+w(Sl_d{X&)#I-BH}n`no~Rq70Q)e?;I*Ynx+!JVtQ#*^`gLC5PCr=NrIR3d zA}437DN{fW4s8=%zGc81oiVSssR(N z2A<(a0?q&wELGzQt4zZ~6VBwLP?g2Ya16kwby4IT!X)NujuuVpi&{CgGbkWf#azAM z)c&m%JeiZnH3p5Q6+a%>*r0>i8K&rM?0uM_T2NvNFJ*|ogcVT? z)7_j5_e4|^S?>@47Ffe!+BsD?WtQB*^rkFr;c{ZCR+Nm-f|BKIvCyDf3hA798?&>l zbKaeo6<~b1+v2gMpk9jp0~7TuETbPeW}xO1i8k6Dm1$@B3Z^_zOTesES0Zd#@a)G} zQh7MQFBb!j*_$0pWDydYw(4k(MgFUs%W#1ueM`0RfsJtITOGNIBvx- z{%WwJuo0rusHz%Rwb-h76HZV$|37v=zp+HdLU`n*@^d#%VXc5xo%!G8#Pyz|HY54w zmRf`&(Jl7#&VcSYfmUlYmUKH^rL$Hol^oLJg4{aV>$lJJ-Xj%cI%A(~=3cmBzQWGG{4}GW* zsghOmY)31P_fWIc2rI2jSW%+@5;x6A9FCzOQk;0ror#-bkz0JsiN=8z;isI_#{cKE zRD>->NGZVgBoEoZnf#nKEd3w>L69OuSttUIQ3yDY+Kxh|!bWUhnrRvhBh|nm$j@%$ z7!sL|9_jh~_t!rHdUtMt@taV1lyS{cC>ae}T!A0JCNg>gN8B%@@9=^dX*U&~?H^FgWRhXCXgQn$d*tY zUw}`{%VtGWtpcLasOR_7$kpT=tsRY~QKBu)t*F&K;A9>!iO^Y=qshvdAfnSY*UJA(|!^?B2E^8&K7orKXCRl6j>fpt(es@`{0=^$g5HPgppVczh zPWMmuy9dY(wM1g3WDu<05}hCFlKQ7ui(t<+2_vZF^5b4ye+Jx&^LeM}TPT5dSMEW! zQeoL6wgj9(386Xjp8H8+k+Gzax_Wl$+F!3-yR(A7%!OwWn%y>6zrB{QqqgsoL3m$P_B2gAhRasYCFEs)43V8e;!N!2(*{6m`Hg0Il$dBVU#RBgQJj7mjZ?7JXMuFZF=m>+_Op{(G zho#e}g9oRFV^z|9QX*^xGe-Bl3%;?5&CEJ#ma!5>} z(XD#5h{0J+W>;z+tr|ZSh-x^0JN(*w;$x#XkH)#CR-6nA2|XF%mCs_k5L%kLU;~>@ zQiwPVh6{I&S|o5Q=7#bpAZJ)HS8PR(SMB)pjG|d_;mz^`u)BVV`Yo&r`W|6xHc(5#26lmKs@LismOYIr^Qp+@z|bKW^?WyHSrq63yCb|(VB_u zJnp)_Ip_b}6CZ$CFK8t0dZ%ao({m-QgZ=iP2AO5V2Pr}|^)L3AUXYqc$&-Kk;d--56TLvlo?L2dA=H%rJ{AZsj^9?4LeIfqh z+ah!CY3L-yenb_vBzeMEr>P)Qgok7iW8(zd))lwG(Y+^SUOuT->@xLXunpU>DCf(4Z)upBu&fPim-#W;-f^UNz-h~Q`cCrWv?=K5=m!ku z48%%mE!1X#Q4)N2R`<&+00dqn;u2Jm*jht+3dd?1US)(oqGFAqb6)h0ylt$9^oaQF1vw%yDTc??G@!}EO>KfI zil6cL$;`a-tZt<^%aMjBZSt`Eyk9%e<4}j|omV9!0tOvQT?z;0oC6Fw8A`2XuB`*Wt->kWDMB{N9Sc|>(BN$akej~Vet4KIC z!ycn?FdascfR7cIxe9EY7aso7Fi5!gPKC64*>2}#kzDKZS>$Jt#>o>N&o~^XvRu7K z{7-)MG%Swu1vg+3QQAxOwN-+s?Je8!qb(QiKOkH$+AfJv+#42C76^`)?5j% zUtV@an9n&xQ-E7KJ5uaw+(zjtz83zcdli1XRi{_0Zqvn$o9wK$;s&I?gpc1*s4|`G z;^-&sACEGHyG{;)ODS^b2lwvJsGV%Fy9C&X?4F$!LIR6_?^BI&-83criMGiQ5~MFh z0mn7gUhG`;^?_m;M1WJtZ#J*R(u)2+IO((HyO=Yc_Cx07&hWJ+w^JHbs z{Jz6~ak=juOiMq6DWy$5+=2SlW2DQ#NjZzyjFz3 zWAvCZ-)3WpvkG-?VK&7*(XByct}2@)!Wvw+PeSjqMHI|{?37nA$?rmVg_Wqe!Yr_6 zm9aK$+}ugnctaT+$7?g-;*}cRz~}Awn}=~UU^X(~RYn>An2UY66zMV^Hjk8$r&~<(hYVEd*55jqfw-5 zEvG4;!yUk+^!PIj#&Xxy@-J_K%htf;cS2XEK;*TG5b4DfJ0${fb)mWZbA$h>XH#_d z6_2)1cK6{AZ;K{0M4{dfpJ=7^+^RzeAovzL2@|nQv|;j|ctqi=Rv8b)aR0zSf*E_G zsGg!GKE)eEQt^Qii)i<&GA@zn?L16PjE3yFUD}vko>BJ8sf@=s-H4lyMm_h#@AchM zy7D)W0XpZfYG$WRR}KbzLI>(*ULeMg;SPct!XM}-%eGxImHYU#LY?F|{5^-c-bJf; z?a1kcom8II_xDb>S4esz33)=2#%E*%C|si){noL;aKt1?ZPrhEZ0kc6i`mwwuZ$Tx zv6OhiyZihzPc6`EF|*TtxW0`ly%_fHkmJ-ejYAF8JX{Z=8KixK%n$t-uIq7jnGuav zDq3&}}p*b0CL_mKI1(>{+6}1^TI>k#68P)LPcdHJU0u$ztfVsexYO z54b;6PgjOKe)-i+$)Q6LJ8m4pWDw&``N%mK+Z=)qc9<#MT(5Y8MxU^C4uDj=w8NbS zPI|O~m)W@mdx(BQ2Gd((2uPMGi4`v+PRQbx2S2H&hb}&#*2mfG6TtV-g$yYC%B-GT zjN#yCEt(dMAS=97Mn-5UmJJ0cFrVlacER-;;`3<6tC`s0)P_{tRqT792S5hYtp-F2 z+87j>(Q~SF3dolrQwc0#ByYs9LuoFNZ|C2B1~2IND7a!E>y^gkHTVxtP8=APo;SX2 z?q}vAcSCwjybYh89nP1F7yV1$hMuMvUV0Tjze;(C?!chSZLT&2N54^(V1MAu%+T7_ z;AaGLjT2g$S}$c5QgUrA$#D4xIFdOFCTGO7--7fYIBMCJD~RPE;61Jl4(3KFT@OP0 zpd-%pui3a46s^Asv8gLTm6~`nFj$3q$5p*^1G|Om1DDT~KO+m@BAzUSK%AJurx+ZGcAz1+^StNm)SN@L~RM8jxcbcyaL29 z_^*^eLG~L7j;JdzqM4ENl7ogxfU0^24`*K9vBUhOqU>RyJ@iHKPPkNuLq z^?qDXG1^b9iHvF*Rt~J7=1xFVW;+;TxJshdF&j7@BB-ne|8>g3TvH=0xLU+Rf zEJqx_w`Jj{m_iU+zE0j;lt#$PyN!w+zO1^JqQFKu*(V&=UVT9;?kuJ0x)zSV)gCk+ z#A`6FqI5zV{*rU#K(*U}KArDa_sgB19tHI`xy=$L1Ero~571b^a+*TU&)5dq_y?1> zU`+N;Bk%>ZzKNyLL2pE!PE!rFD>yE0AEFSSF1(diNmw9)XjI}lHP`6^v;&XgF8t5?g}E5v#?EFHiF`tM; ze-%4@myXv`17)_Quexf98;^czF^B(nGXRbjV-^1@^4oS=tr`*(Grb3TS(}Ck2BJXc zT?f%A3ch0;%4NkYEpT7uz-Wwu$xU*anv;z!sF2&_L=cmY=o3da^1SH;0Dn$X+>*Gc zN+Z`7#+v<&70!g+jOyCZFmt)u!VxAj`On?uF{jz0mI*(tDKf3BP>=wHk*}qY1ReZ- z6;d*zbcL>zj!&O8e_OO5J$T!2Bvk?1+`iRa0k64W7ek9LynldDCS84}_&lGU5nWhc zj)q5}I#s;KJi0H-x5MK4nLFq@n%6moU@$F3-tD1b<^$bs))wJKHpwl90=fNrL$;V- zqmw0+yTZ@z=JIUObge3+t^OdhnncHrQJ77)(2&vj#?DEAqA!|x*t zdEp43Hur*nlYXc4Ir_&{=OI^N!6unLD|6Lqifsfe8UGL5h5!nBgppu7i{z2`T}}T( zDQ%gu)E(R|jH@JBh}WCovwq!#KEakDUUr}98q?Y~{P;Rn=ReX_uq0AksoGr9oG4~g z5H55wF}XHA(TOO#-BM>Q8#?cV%Bq!$1hb%y;DpJs%n4c-@<$j9GhG5SB*jt%8;$iN zxt_*hH*Y6%=Kr+i|1u7L3#jiYMczi5@%RF(d7tr^a?*E5|6zHCVjL*29uXO_nbep< zD<76Odj65kO|H6(A2#+_lNhS+@EeYp&i(K>bNg20f{PF#2rWNq?7;hZo>6x@*!I!; z{+q^VFh)g>bF8b1>ymlDzJ@m4vTk_~q@1fq5z-6qAlE1Jl< zw{UQPDuc#367f=LM<$zfK{2>wZdj45YWh!7ArSE0_}by9Ru=Z*`Q9Y2dg=g(Eu`^1 zq@ed*|MmZj)Bm;&$r7@mn+&biL(bdw&n9gv%Yp#Iv))?!pG4Ot0jVj*Awd$l8~ZBz z?S&W0(!a}rP5<=4dL*qMKi}s+qjDuz$EIz#=v;cUNuJ$KzH|61Pa(!dkcQ85TE$a) zLqtpGg}OvQv>iU;e9o_xL$EM_9}ZZp%e@6#^>xX%N(0{ZW2)StEq_LmxkrSMfxGq& zK&GN+ip}XjXQ{*7taeFafB-bbQYwm=lp$!1%ojRzhTm-{LS6x?c~pxMWHGHT2@sNGlwxQ{KZU3~9+N0^7Uk!aQJjg7H73bA( zGGJP2SBYjKKT)HJt`{JZ6xg3FWOg%vPsb>?XmcItJCwpy70ySDxC*Shmbb9D7IZ)n zqlwfwy9Y5|2tq8a^(Mtoxx!QhYH8G2sW>KE3BpWyLJ^TzWVXqbmZI3x7l0J&t!$A5 zMHy*I@E~I1&%n|0>p~}B|vVI1Z13gzP#P+PAdrX%{}U$7eWHorJD_ev+Y{`t6D#!`JIsT;#9Z61_#@&26}si z!*Mml^0-hJ2YfX1;N7+2X(7vw^Q0MphkK|Iq}7mD-`-<GpHT63DW<*$k~s(zIXL< zx+C+dyPW%|iC!agy-Rc;0qyz+rhs(EZ;MF9472?o+x?DqstIPb@R zyWatUf7=7QK_K`4dQiF-J!tjtH?9gEEmx0kelu;v6j>A@MQ&-|YoBW`t%O9VAe*C{zK_K%B7e94qd8cc6 zXL@>Po_#q11|nl=GBWT_WuVU~Tj85r|Fhf|$vzNpI`e?IcaLcF8g7*3$ebTXOq_rE zzUKI`##ZovFqOaAdwbP)o7?+Eyx)lJ#m)zuZhZaFLdpK^?%wSu!Mms1lgpw1zfcE~ zIh`j+*Y($L#i~D3o-L8R)>#T%eLd^SPL~%3=L9%)qi%G7X*g&xAZmoFX7gLMTV5{T|H#sh^mOv0h zm-X$x3OyA`2ey*aw*gk-hKoWoHB+l=_#h6Egis;enoArC4L0V0oAm&Olm6en{g#nD zUbsOL1^Z=R#NItXlAY(?v%(&G;0pNoJoQ}MuK5!!e;KKusSP@jpmU|-oD|B-Dq-j& zXOgb9nqNy5CT4UQ1A2}zlY!WnsRo_<8Gqui*x$=-hi=4&Rilqp8#O!xwH-~?voW@6 z6NgK34pe}}w@73IsMQLa79REZ&LkduqXue{4RC3aj~EIN-Psz}kvV+)1@*yKEL|DB zr5nB7?ESkTx$8VD;JfQVziG8)aPuG!{EE-J^)EBJ3?(a{^e5)O_=RkuoU5va&U%7K zz!0^#AFnY#?%7e~5&yg2zT)odxOo0!Xa1%ro-KTOa;22;7j-4MJI*okzkOt0)wc-8 z*NAUE#ouVb)BYc}zA~(>wQDm-aZ7NA;M$j(H17>otf*J`Of=y|4Fj5_j=Y^w=qx!2kzMf?!Ele`!aro77_klv4bd8l;WPK zmL#2zr|s046iYb7OWn}GoT;3FP~#(Bio8@eisDyg)5q4VxNeHFUZLb@lG1X%icai$ zHPugR^RKs5y8cx*>*P0%{|@%eb{O=wEa-7kr$wc&rMzui70GlvGLx$bx)pP+t;-q< zNnZnCz%pTMgK0()HNBd|^dCuR3S#~!O!b{)hAh(5c?Lr=V=9%cP@~Sz#l&*}9s{e| z96gqoxP*;rPPrZ=1pHO}!-VlwT!;INszutwXoRNiy1I+aY)C3WT)2g?>{W(PHVz>| z)?70sVybaS^>;C4fKW+h@11rII~0=_tT#juZ(ZH1@l8xs>prX1crBcth` z7t}vDi{dq&b~JP&o>vifc;KW@@=qV)1blPJ_4AU8+9$XXEa|>CZsR{_lPl!fYd~`( zcR7UXLwUUDov?V0Ar;2xLwv>W% z%z_Pnxppx@WS>6VSCve6Kt#6oKSewq5-0I=k%aEKw<{M}D7a;#&g0bn@CJ_5A$9vt z`98Z#JUeKD5)OY_%9O^((JB^ezSDwwb0yRZ(gKL)zw@CowN>bTz+jl9wfUrC_&*Qp z)*JA7q3C>6sSnAgI$tBKFFSc3PMJiMlHRvyM`}D@YurwY$`WnqM|=sEcQlHgl3;Pc zK1!dHD>ZFOwWhoC2k4YX4*>zdgPX!Kof9YxW&J38=N=aoR3g>F7l- zq`$QXs-NOUu2}6_Dbbsqm+`Z@8P#Jje2h-LDz#rExYJ7LrGg!lvSNyn-I-+pYPirc zrFz_te;|%Z@eH1y1dUEA_i9I|8Yp-$H?K3N4@$dAH0(5@p2tRAyGFrTEpOklGN}@v zlYu3AmN)f2#K5mrPh1Ef)f-M;v{#u!bPxelAu%J+(Vdtagoseczo$oT!|_3>4Q!yr z)*}t@Yl)+Omj+1)OH!hukJ%5;ZVe|7p;b*bYoN%n&h>}Omg^FWy>eNC5tB)~D~d;C zVK5sObBP6(qoBlJrUmbUIP%FX7Hc@zo)P^Gwb_*M>u40I{yu73>8pdGf686z-#LxH zIzy=IgS|n&v8FI3gRNWsR`qB+6E~|ex5ufz{iOm?>e^%b7gr@2Q?>qEhX(XnjyxgvGuP zu1ZTfv>;9uB=h~XkjSp1RDFJhDqcY*4A7uA@KM%BPv1UIMT;bw-=Zs((9t{<(|U2qxhU0>ld$ctF7z8^x%(EYbOfqXhbY{7b_5gtWjId({&jp|%g& z5gRnHOgF%p*OL0I=jF}_GL6JX<7y9O|1p9HW9l_{Br2Bo_^+Xy^nuS!?JgcbTSfQ*w$fwp{Ych-G8EPi64s*w zl`c>uHc$wN{&`HCsXQ5U>yMMQ9*bxnit^1aiwN$3uIKdmwc3hbZU0Q;|0QV7*8~N@ zZ0||^dr09jkFl{-`h@-z2K6^BKb-q!-Z}hOoScpJjir@x5CgP0QG4)P_AX;^Ea zIZX)PJoDyRyJ>9uI=|i~LdJ*z=}bOA!l8=Y^)6NNK2^_uK(DDel|Pk6#JnHAAuMY1 z>ZZDM(;=D^OKzNl(lqb9c?FF5nw0fv?AxEX!KZ;YeDdUS$-e2nuA>}C#x>^VN&ae2WiiPC$F|o>2Rd@fZVBNXE*8xQn*pZ1PrE)7i%arAqr1=;4OtU|v?T%O z_f5m?xx!fmkd&4qXHuZ_n_ZWNfzArazlWxphfYK-{^g{z8QRT;Mq<)^&q0V(I=bTF z*j!E=8F&74qKQ~PCM^)KZ7O39Y(^Es`SZFtnJ(f_Tft+S;eK0E^H~kOS~#?`{C&NJ zYacKj(bm3!PCHW2I};!~ti=$r0K@CpsAZtN^gJsnT|I^7;h%Heun7Mr8L?&rA~jV_{K!>eIaLsPCf117NX= z%U{99FETcLZehYP;OA{XfoMUCzw_n;6$&1pG-z>){5_2?dJl-|=j96b7z^eP;f>&k zs3u$$cO$&1grNJ3AYl&v?%@*-iKEXFaVn)NGIv2tlE1YjiE*=Jd@&4{_qpOGlWcDk zJDk=t{6{jHa0_2w-oW>-=9OcCKX06V?tP9k5;J zhwdo8iK%gSU+{b{_`Yd#dZV(p)72?nc2qO%6}4lVgAAox+ugUb$F7&jt0boCpY9Nk zqqy@L%2LXGZ?uzVC>%ixyYJWdBNvWgEAT{Ea3>cocylIs+Ho~`XFy7YXS-%e`>5Y? zNp)A#^{d8l#rzkAA9N_1A3$2)Z*V&rCX4PLV{&$QSIgY~WF9N}r{(qpq$JE`F}c?u z1PJ=h5|BcO0F&lHJiLPdI_3tHF36WZ&yQPX@C2VpBq|9cyPMekQM;R0zITLfQ z{KqP3TxIB6Ch5k>@PJxJv=?(&SXEmomdQ`IE$_-V2r6brq8UI}A4rG*NxzmUo@&N1YU3@Q?zc2FSFTW=e?b zB}60<5${jp>)Yb3i>e3HLvOdPOA%#i=Nntr5e^h;R}qD6^RFE;MPm28v!dw-YonAF zDVt8j48I!YHkjXI?@iK$apcF$YZHKDM(z|aj+<2VL%kBbh4(EMG>?t!!-JW%^oTcpJN$;5Lfx&2ri(Rd^hUZYyu zijbCOlsvWLoGVU}6LPDFJ$g$FJXSZh%hjof44SZ;B^9(!LteD|YA{WsD4|Fgga`6x zMkw~>VOPF*Y|?n-hso2ac8%yc8C8szEx*2Y#agF0S%XK-t;yR3j=t#81nN7@x%!uk z9OAqK%3*IZTq|F;XZR?(<$iWb&8Z{jlUyR-%e3?aVFqI<8Cria8u4@$_gBnjSfokj2b`~Z;UXlU)lz~M)$V+0^GePQusCiBc4KR-+*BuH zlSZFR$PwdhY7TfG@ZE~HS>@`M6;*sC{EDNcP3d8t?5g=D#&cIUqYIBw? z>UtaluNR5qB0g3XPe^)@ZKgZde^D+5KFS0bwJVW3RRTkmcdA==+~QE^ic&*0iCC#F zbi!$m?rgDsXN(PDCL0bJsfUPa?bPwfQJCp)+5PJQ{~FtE?jI|MuAeg*yP(x9a2n#F znqBp&$SsL<#*KXQH~I50O1C>|!6jRP1&Sq)6Q3hx$wHS>*6DJ?VK(1) zmuaW>$6M2aiY>coJ1x0sMpd59Yty<-xS!oih8o8Z>zT*!9-Fn(G3zSv7V(&3lc9eW zaXC$={m3d-To)%@$Lc4$!z)2wW1I*N)`fyxx~U>so1F{H3bv6@DP{bVXuMlO)zvl{ z!rAniuA0pEI^JiE_7 zTUKW_$dQ^tb`;btanwA{I;pX9U{J0&f%~nbnXTN$s(K0}q~67^G{pt4B}Oh8{h}so zpUA&6(=@sZJC9F^Vy2?Gcs--$=Gvm?zY&6d+I}YS{1j&9gzoOY(O+G>FDK?Jo0ND| zg@t7hWiv+~SND+*qa;*NeA4LocONjxq<;TpFg6hAt%Z?&-yLhilbcM=c=r@hZSs}f zyV;FD*%3*RLgjPyKmN4qKWM%`Ej-QYuA*?}XP)ba{}q%rQ~tCoA$czP?Migp+Yj0Y zrF5MVzx8g2Zasy6MJ*QXd z6zb{=_AfeiF8FOo@ZK1IM&8}}O#uYrqmoPFDR)@+hL`Y*6k%on_8@)jmxn}sW$(Kcc21S9G+D` zFA3oUp~&v_akg1zx^^5LY?{c9;UJku_31zks`zmOkwMmKR~8fT?yHoOaB#AG%VbJP z-XoDIA$4eK%5ba8WUjDfwX4Z=xAN)9FEM6V@h>6RP!`!{9>0%qVv7^-V3`|5%8WYQ zHJB3@MRtGFI!?cWP`)k^^40dKGU8%y#Q}>qbK~eb!Z3&U?{0R+=^n}D%4 z4N)o8sWFde*&Hu!fHIyaRWu~(=U#P&XtYpykDL{r>5PJWF7lF;o9wghdsSOIO(Jil0~5JIxPhoZ29|bJ!{fpGSB__wcrHa%UlLC zkcXojDpGC@qA({xx5GIeMfCKVkt*@qOYLp(oABs(V z?NSFkjj~Hj_$W>{If)Xr)UCr>}!Ff}iBZil4M!K&}z-BrHaC z7N3YqdI*wy=Sp((Fv|}IbjW#Kn<3Y%3pe9zB;>UGhat?1RUF{mEl%DQWXF0HbCQf# zTuhJu*9*Y^Ulm^@rW@uL+yBtK>*ioFri&PHJ>%LD`hi`+dO=7uNtdTWS16_>=1~~;!G~N7ywEh=yGryU05pXLor(f`@8V97&sKu5rqR<`fTX}I0V;}hVTHa+<2S>Q zs0ksyLR#a_&IiX=Vr_PZa?0?7YLyAaDrT`?3XLlb5*IAb%Rj`ya+sQNmT6<_6d>_b zc%nag$8*uZj9=P!H`!t~=%t{0inxJXP2yuCRH;{}G@41JZ%!SmGBFCqMvSrzk}3W0 zu*IT~yKP*%$@wHOLUlj*iC}ZtwRco^1GMm_GON+zg~QU*5_sBeuRST-6w`d&AeqL7c-#&-y2dnyU&}&Zp143^$kg?0 zNjlAdRzP@Ir7#WKr7dx0Qk`$6$_FAL@z`jA$Kxed3%P#!z=apW?pwihA22;KVD>Ss z|9y{9Ev~K3+9_8H*teW$Js`GbJFm9xsj#W=XyuJ2nc(h?M5x!H{_N=h; z#xx09L7J0`9LalK79z+&w_6Uu$Mw6F=n3u zpbHoDQg$*mDVOnk#kGV(14;l&-;t&1f(jQIvOJ?29e6`cL?f$bTLnO)u^}mQIzGAD z3vSq-BY>|5UOwS9eEcHv(sN?d1YMF^E&h-(V5aFv+cG+UXmL>Qk~a7M~M z>I>*&&=eID*}6}{G-kQs!Yqo^D7?QqUSMjt4jgS zWCwVMR{?@*_xn;_{)ct+|3Lo6SHM;TVX02SDEdpwjh$?Gv{BubAAYaIPI8jJmFU-n z(RDUk9<|jmVAomjYbayHSQLmGZe%Ol|R(uT&34L=TsvoI-+5r8T&RxMM0C}@3 z_-#-osHqfdy`DL=S7$j)vr?!)Ns=L(JM(SXwDQK^hfzL=TS7&54Y!(}Xs2oYskT{})0ytno zD9!e9gD^uSwi>)6eiCX+D&nlIVvMqjFuyGBiK1$lngjn;*Dcm22D8KasyR@@rWts>|Np-+ycICgo^^3wBCnWuf&d<-VtsM{OChY z*?$@VYcFAE zUVJ{BnY7|ErTHR$%Bi*KFl(FrXUU2GPI->A8>b9r-e4P_+{X~eJ_`9$2)02Lol7JX zV!ui3F3XkZzkBL?dbvjRMp&B9#`OJNvB}XWOmTwGEtN|>d(6%~EljD+hnPO9|5ars z8V6n`9tnBGpTYso<6=Y^sCpm@7%&q?Yb*U1@#u&VS??sepF%I%$mYwh-_%&+3!kd}8>VFz z80go78}K6g9q)fAFS^FyI~8hcopcjiurN(+{!QqJ{ieKa7n=Wce7U(i626dquoTwKn#c z+J-s#HgbZ6n|n~9d!uk7uYZ-3(^v8(=Y zo%XQ)qRUsu>WYcZjDCKF)aYsDC(np0*UPAg&uXnS#PQ5d52PwcoTPnYT^2-9 zpV4``|LvtP(A{jNty=`mW{5JXYFA_YPiZy5$XVB;^K)RP`aZ*Yr<=>?L$2VOF8X~W za+Z|q9ZX<>xmNg$g4<9JO!%XRrsD(KT&$RRgo=r_kT`hGE8DhvvnUrj;Ga&YLlgz6 zMWh)yg^yI+D{#rquJa>;8oiY~oh;frkfL!)+E^h7!^QpJP&Z50+(0>8!g1T$@pa90lPKxZ`7U~ ztDIt&{3_ZOdGw7o;a@M4>URu+kooG&cV1we?0obcv~lyHSE1UAXjVL#2t|R zYH`|%e)rCPG4A|%|EJQ{l1}UHQ4f-T4tT01Uj2>#4qxE&0HOY9pDGS0y_VLIU2<_ztQ#1o z%z!I%N{dP|FgMUYM?;lg7H$Tr%IuE46N1=M$vp)8>>nb%leqN+<_ zkUGNfZi1czlwnWd4}dbG67n>|Hros7V%j0NmU4kIV6kry+TYz5@3b*i9B;{>EFVnq zcTAlLM(StLQR5Ik#I&<><>WX~>|o@w5lEu8`1>tO{)5oh(Y6e*A2HG2unmZJ@sxde zxZ3T3Zr)Qa?>}uk(_R#S*xI|GGON!^%HPf<{%qpw4SWq^AxagVD!Gq(qa(frOmd;H zU@bR@lao5F6?Er!&*$9V>~XOzR1WQle`$fRI}{p3V*;FG0DLsrhcXTjAp5O35%U&C z@-|-{G>6^Cmui<$^!ZAKs~M!aP132`VfF7PQxjFUy}Tx~R@$h+@OKMdJ`?9h2(b|l zL#a8K2nKwo8HYi8ES7+rN=1J6y7(RU!%O_#=zF)2=Oa57)enfR+(AqU4RBr~DE?3Wl56lI6U@bP75xu=TLX zA?_#4KSzC*3{|l&6G?1M;}g0g-bL8+)LE>L&I~QITY#U+5`)rE?7kejf1sIec(b4*8E)m3;trF>6FAXz=rVP zXEbPJ1ZP=pnHPB56NfjcSjLjaDd;<-kJ z*e7NH<=mo$tC2w%VN0IPrA4Qm_{eNox9z-Xb7y+d@>Jl^Qy}7Ug;+XJ|WuwqDU=gWL z)d9rrM>N!9_!z{0cRxInZ_dJndTb>&8o+9^)AkOQ6H`=yp%^yhrt6xFfru!Lca}hO zzE~XVH!m-(ahZg=6G3zWkfTW^SX=)b_*0GY#XrbeuHAFSqMlhv!uKJ`rwcWqbqxMr{&gi)~hfds@X0p9< zM#vwog1Rf5bW@8<5y?{TyE&f&5aQ23Aw#jfJ9rW&T(SB-e{y*-QIaTtsr2U}Yr|FQakx@`F zPYEzkS$fv|;2WnmA-7&<-)wfZ7EIP)^Yh|1K}*dyD`Q{I+$XuB&?TcbgEWM0MKb;= z3JPpR(r4yw%suf=Ri3%QF{*X9w$pwqX9a3P8Wz{i$#U^HmdIhdBb@kc zO74jN`{ljp+cqil69fabk~8fwFT=DhiRNW#ft!ohhBAa+;&+ujwE@#F+r?AUtHTFJD)dqnnlt_4 z(V)#uwhaW&$1rrB=sGg2-7t*U|5z$rckAxX1fBp#4v5?uc$sLH`T>c zgr)l(Bg5e7twf0)>IVA|c5pxV4O>^m8`uY2xGetUToFz;o}|^tbx?G~B)(MZE)Sef zmD@vWMp58BvK!*HH$6z0{B^zz|8q=qf9fGqY;NugyaQPcCXS67VwE|(`1&cq9KdGE zKKKMjOiwLJF=*me%mcgNZUuduy*y^AlBos)ruBa21f+V++#hij`3(rZ`or~M>~U`3 z(U9@qK8knC@T;XEiG?`3ZMw3Z#{r9!nD-2O*LXtidpH!`w(*6l4%FMxZ>Z(dFOaJB zck_#jXLcS2r&DPjrlSTG6L-XkJdw3_SQM}2J-=#oNI@GBASA^s$5@N;|Dp)yyF@yKy-*BQk^Q+9lT>|=f#C!fD1nvu$PQJ#pWx-c0XkDqNZe3l>UN!4bwZWZ=Ofndk^{D{ z{8{BEZ#t>ZkBffrYa_W(_5#vOw*aMUCzz>3CzmI^NUTmU6y@m5o@-vt+M?03TtlEq>{_C(=X+6zr-JM$W zHVlGx%M?F1j!O)TA`!01)cO$&mb%OXP`+9F8X}!=K2g6E6GN!xz+_R3FYNX+?^Cn= z*9Q0*!R}8k(XrjoRatB9+bN;f-?^QiIunB&`y)D1MRz^RWl#z(pmZ8}UHQA1&X3$^Gh`@SB`94U&eS98aIE0(jW59iXa`y-P-u%V4AG|F3ze;U)3}E>aZZYS! zu~Mk?uP9$pAa85!IJ$Q0tN8qse(5Gnh{oi5`(_=y`La~VtVrgAu@S59Y)_f2#V&+( z`vhG;GewpffuJ=p09s$T>_zF!Y3-yQ3TnG*wa_nEG4|t00TNY{$73;VAdHr=^z`wr z$-e?(8{zWJ=lFh0eg2LqtjE5siR=}?%e|a;^3z}5VZPLhSo?-_Q=RdJObi zd+u@8=y4M?DFT1GFuIH@sHT<*Cq1Jg5D=SV*dKfULaK_P4U1=ieN^eZCk;3wDlx)H z)hH_JMFNgTD`QU$L;AgeE}3b?PgIq1jnw8dn7|m!WGY6FO<0s`v z@HLm0A7i-*a?;AkODtS$r(PFSj^5~L9VZWT| zTIOzBYsupm8nO(bRq0Iy5Q^XRMqtyJteEZRT{!A5jXO%+5oOKS!j7OIv8T_c{dvoOD*B9qI2*+I7Qp@SU2AZ#4}wmzPcddbLMg%;v67qw4+TVHGt7r>_#v@~ zOP=tY?j(Z;@k|=yMF7H#JE1uO6V-mL<{`O;plD@76JQ3*zNuE>3ZS-;z_CO%nyQn1 zZE#;RelzZ~h%bBT7F9eM(}O+nD$jxd= z&n2uhNg8i&?iTXZI8k1+&X@=16DzUiFUl^wD`SGqn#cz+tL^p1#zzZ-hsNJ7lf9eLrKKXAX%C3WOU;7I&cadS@g_n9hoj=Po7V zm{sITDF@sa_hCRYQjy3b z9sfPM-R5H$KjJ%iCNxu5*ELYrO>iIwul=e|Lx(aG=uy6= zs52~P-CUlS+saug|E8&;$B$12_=&C$6~3X>?GtF#8^*^;VjRoBmzq)lSNmxhqClq^ z-WbGKlu5dUUxnqG4#04Y+Da>ng+Uy^?3l zc5BVjw@4^wmCW81uH?c|l|;9LTCN&1K>DUTJgv_rvfAE!)tp>(-aaFoL!Pe#(+Vx@ zvjHraibgzpCm88FfJkXo?$BC}cH!Mx7;A3@2c}z(blaZy9ztoqN3UoZ-=lu$j7l+A z#oaq{IHcRb5XOXlIKe=6eF-tUs~#PR>W_$Qf>)7gCp^S!C|WTPzO<~~YF$QgO@zx! zJain`yzWHXic-{3VtxF;^k!L3Sa~FaF2(|%_&3M*jz#98@QN=DI`R2d#%`c3E&f$T zvIa4z{Sonu25a>DiiNc@c5ka_*G-u5_kY1hoh-qORkyemU9HX6voF;mRpm3yf;1>pT20)Zm{wy$CaBH3!RUVlIsmvmNxcgi5`^*3ViDq2Ra;V}RU#d;Z%NgPoAZP*wO! zv;r%O{W`fP6~IhbYY%{^b`_!%erN&|gLQQ{T&D-rZj!`z&eaof{$}zQYgK1@1%t%K zZRy_t-7F^Ak$11b+Ak8nfJw+Gkk0$C2PdWYz!nR}wmA5b3Vft5!im%5E=mKCW75fmmZ~R0DMIOu8tWVgk&{$uB!ZE6 zM#*#k4g;#I%di34L6|;qsAYQdVCK>At9TnR+2jj0WXog@(2L1ZD|KRkC~vPUm@pw6 zC&QvV-LgtKG1(u~lF2ngc|xmXQO@1-13yf^i)+QYkjtnd;LIHLkDlE4>fxtu-AEtAXu zHoFn3Cl{%yiJXy=1mGQnykKvH#8SmXEZL>$Z$XXuQ_#po>#aT^Nq0HgCl#)2h2ijK ze$~H>URu>t1#W7sG*ho568mo6L$3&utza|w%{H!2Nh!{#HQiB zY$V4FQ=Kh+%ZWPuf!wyYM2Qq96Yg7O=vxKAX$0Tw1iv@Q!0V#Fj1A{eEHPvo`f~AI zMs#yoiCI;-B#jT?cd5w^9%k&o0EMD)CedBNtJH><;=3z&B&@WLa;m!AHL*1cA79cY z`fRiLc7Q(4vPRK1_N0}QisB$#Co`FNg;bF@lZ*69fl^&!BXCf>*f+onxqK!WtNuc(!f0Vb%vH85xRK+E z9r3i>pbdJYla-Uhzx=YCCu=aJtZ?3t>S0EOc>tyYfdO()RcaqRRsT*$<3BTc!Mt4G zPkxb_yJQvJ&EAx+W~kn~@SM%i@eQ2UyWL>qDM3iuu6jZzZ;F;il>UHDTa-@TU!`M3;c5o^PR>yul82z*5^$RwppjagM<}TzC zXQQ`zE5A`&5!(=rureBlKt;m2L|T>e2P14mt_ow%5EaTYk*jg%b zhIryNGTmoByJ+)i6?+EO%aX=n1>xSt|1y%T6NV6aK`r*}v1^bPjF#tp1|ydibOTxI zQmESIOogH0U+mI<@XU%n?jh-HYK`}Le&4`Dn*;!@Sa|z4;?YIl2{DdWS*~g_(K~#@ zOC+^!IfS&Fd244hyLl_qTN-Ua5UMgczI35}wQr()Q01a!Zy&J2q^RJJzmlVQ-$ zM@{3@qe20J49;0)Q26pAS-weBwFqrcqfF~jg;evR!JTAZ18GRJ0nN=_7HLSfG;vQwql^ax=>#nix1R&olEr6DNGwc^Z*b!FP3VU<-DTs5rC)=ILm9)9 z$h4Fvr(YfPR#aAB@b@VW8>V4k7>s-!k|EP3q{y^qpri5|?hlO%v&h4;hI6YP4(!?} zqhir4e|_4bG>7NN4=TRY9KeeFwV@N-*Dq?yn9F~>oW8?o6(k*;?8ex-0{3Qu+~|ap zXX~f`n?!|}%56?Rd=d7?eJ+_0R7u~0$#(;LHG58f-r^-91yZi-xs#B{qsd5c) z9M{)yM8R}%wR9l(UwHHXzF^FYJZ8k0fiv@44e}B0VPAJpT)Or9;$h=0mrk7R;HV2Uw0?6VR-tzCCXOqUlDQrXjg|Sa)t@QN zw4^_omJ2Si?Ehva22JtDZ_l+>{Zhmwn;!!3n++*YRGeJu&C|UaX<2SDc(KmuT(bHt zI$24FZ#y>P_~X{4!!n#)AK=D3V0v*0&0`(ySjw&Gi~n5fgh7>*)1+vLa-v6KBhxbA zvt!I0NVy!trQVu2*XuM>(iEa}815_YkZy*l?K+$|H_1VeZ(jT2*uy>NVn$93<~qWH z+_U*5i+@;dq>LoHYGu%^6>2CCoX;SV;14E4cUM|h3tdu(uI8~!b=mA83%Y9Um&6K`92M?)EkA*8-_yB;%ng5#Qxqs#XDM;&Aq7p z9=lM^`^jYc8KbQvmdi(>9P#5F23~pVeF@GeEvCrXcX6OjJvw1i`hl?*GWK{U#ntEb z3X0HSj@Pbg`ed9ut*mOU@gd|%fH2Y!|0F;c8jJ`R)_gbpra}sL;%+t9GitLixSLdj zTjd;HZG8^LAmV}*q|##HP-1De5i@Vh5TXD;D1g7_Ycr~NYVt-rk};82 zd{8>R6F2bqYcdVwuM&!bq1}A^Ek6z0`A9X&q1G&Mc22`wDnao4f`O|6qfBH0o^+rw z#<_B^M-R%5?S@iZ4K(2G@>j``RQTWFNM47Jyw3g zx8v%Y$C~rVbPBHvA=c~;1{lIU%WOMngL5KKOyfs(nz`LWGNi8whvlZ%|JddHBbR-! zczI)i@7;~k^aA;?@bV+Exb_jveJ{>mxqY@b?QX8dbT!&09NPHMk7(7uk;JMk;W&{< zF2X-wiN+XulPM8cpdO((tlHy{DBXt~$6p}CF_Pa-esm$sLQWQOo)-hw7CV107~wPS z5X4;Zhxo8l@cFsu>OCU_tz&xL5qhk2QI!vh&=|h$WzMYcyO3svVEexXIo9u8v=5S3 z-1S|p8>~ll@Q|MllAaE#Gy>Q+s1g1V|BI%x;EHQ$7U;}igS)%COYqXdmUiSYmhkTIo-;d<`j^3|5KYLtL z2WeD!h2q^7vA!1FQ#9jF8UBViEIyZ9e^7;um~_IJDF!+S#FL;fsqJ+m!(nX{N}%z) zxl?O2=09u1VVlyk#--HmY-hpoOCno**&E?27i~?Ut!AN|ft;32>fZ%bcczt&P@(O9 zJ%RMIogUgxq^(2&wXc&xTX^Tj9b{*p|I`xN@_Nx9bZHLh6P-6|H&8~ZsEmzzEwd1{ z`mx;7F%S4*Aq)Vm9ur}kl>2c*?alBx6k25wN4WtWi^fzys?U>bRb=jGA8!GOEL|%s z*>Bi=A+&!OBvOVTpN#37`{E+$5@ofvn=P%kD#Nz;KlRRsStX0pvW6u(+o?TC@X5FNlW&BlSA;i7U30)idjO;L z$GaOZKSw4psLg7VUV07(k?{P3Fz44F;3ec&e&nZP?Te*5UjBj5h3F%1yU%^Y z2VcpXE#k50Kq7A1BSZw|Ct?%Yd z0<~u6+q3t#wZg}Olcp$7VL==l|J0MFL!$Yye0*l2nI8gsFys8(`0hVe zD-qRuTpYsk34M>rqHB_Pw z(18pCMGyM@v9L#7^HY77Qyev5($+k zmS`4WWs~^*L-8WeyCfhq<|-Jm31LFN2Muf`4iK72CTpC9egL`hx;!OyRUtmkFOPP% z|IC=yXLa&!{`>c4_M_Th@ZS+c7-QeQ0~_!NVF^=o)C~JnQz5kVw+q$qxAX;EuOQrc zlcz;qQ*=Z-KGP^e@2Bb}k(M7`Yo7jEKPfows@@^83Pi7@Wg-W20ESfShq8zY*Ol)G ziuAV#G~>d>MuGmy9qs5X9ULO7S^NMJBY`{SH&d^#Rg(D?pWm84H%Fq4d$|zlDSZ>3 z;g*DEQ!jbM>*@BvMmjX3JppI?5EEV`aA(3rpPKQI+8IAcDpT!g_cK5CQiWRizR+sxblK;9P z(Qn4-kglZdiuxWuLVo>Ac0K!<7q2y>Mj8N!U*G@N3|WIl2(s-5U$T2>{GbM*x?kY{ zB=->KLE6P9V?5&1voHU>eh5RhhP64*y)`DN^TKm|68eh3kiGQR#?m+e-RrwGv7?XchRKtT*4$cmqiXmvMqU^_?efijfMSI#BpR?M)9V{Sq)H8%!+-uO| zmEel~GCz`Q1S@h-@9Vz%J`-i-O0ySji~rVt_Fb2^7B(YI(9Xbl_Xtv$bn~rJn+{pe8R=6f7wtv1*6%$S@)<|_`KMT^okfV$vuL^IwsZ$ z7j7pNTh>CRh_(&k`_8wsUm|I{yv0ae;ofkIqH2M9Q*rw z>~sEk%aE7xDzDyNGQlrk8BR~Gk z{J=67i=^15IDnpN4Nq9z==33x`OA*!8aj}=N^!ZyAFs>PHCnHSTIl*$%z|m_yg(vw zTHN++|Hu2zj}dJ+kG)rjB(QGnzxTEFkj=}*vCD=1s>2Q~|Cs*no~iEJn5i=1Ph=mp zv{6N^(clv)v)7#K{ezXer&T>i|Buf#f2{s#03?+E`>`Q*l9yk4B`%jOA}WpebP1|F zOplZwOCi<)?Vbt;43=4Mtsky!@3lW(YqJxyW0K_Y^S1Zb9zNqzV|c`#W<|p`{&5m_ z%UaYtbdr6O5J9x2oq?LS!@wJj6VQ%t*Gyd_JlYMo3>X%Awf+8NTkI+C3CXi3fVca} z**+B}P+!5a1!}lM3L%yksD5*PP7>2DypM)JLAA~ zLVp}{#cJd?qB*o-1Twr@F=YNE9sn^#bRJj1a__gii5kQI8Bh2lGa8wrr*50CbE;u@ ztf6f4D777=z|EaCz|k-F#7_Fa{y`HD_CUe&NgPN;2poKQE&0Nsoo(=tkwAjnIoxP+ zpO1L-m?h4&7b#)Q8s$llw4w92Awo#uVdHMu>AsVMA9A<4G#TMDa*<}G_;w@+(_jB? z$m~(dzIOz*-KgUx+`i??Va|pdziB6$@(A7KhCQe9VLb36N#|laPU1uDc9gVWZg3x< z*ktMnKlH5s-4LhY4hw5p0zWpAf~f@SDz|b%Wx>l!SqZU4`fKz+a`j|CAtdq33}4<~kS|>vfBs0ky^cH5(}w}_ zQU3^2cpo#Gr?-Vvopl!$?9oFd2x(WsJ@@o&_f$K|{tcC=!T6m8!;=eHZNsQMjqr`F zP0Hnqw7R#(s=SHHCe*{)!Ub`bBMs1q*qT-9UjF;6qA2$6nvK-|J>6v!jx|)vnFpa z)-lC6%Z-ppG`w!Ih-bID2#@??)e;uEsFaK^?Bn!%M({CEk$jBw>X z^TrSpL2j?qGSHNE7yWy66(_ihw}}oN>fFAiVkIJRLZ@1DPEA0A&1P(Fz1%AeV6A$O zpJ>7sXh3br(co6}lVOKMn;lp7U~<4f?o}dfX3)i*UV@aPQ0mMN(q*{_qcp=Go(l6-&o^ z(f)-?bJc!q?SK>YIM2?}W?vBUqK$ZgTv;eM$qR&CwS5wFm3DGG#S}Am<^AVVJ*zCh zPA?T#d^NnBeM*dLQn|qsUJU(zU}|=D{9*WPzSL{?-GF=kR+jspMg&y&d-uh9YB72I zC)J1{D+Cm2^;NA;2y$LwlxS+~Pu_C;PHdOw?g&m0$iZGx z=T5!1;2P|P?y~2?L`&oR>F)nzu}r5DZw7v;deXg`)BwA>Tz8 z*)1gyFrj1An7-duIgtkLlQY|BU(xfUPK=a;w)b73*6&~9xD?qKyAPLrLJQ10!SL$D zH&h+oBRp27zJlltjOnh?H-Xm)!?Y7@#*V@SX#m;O?UMf>`?{vUo3h?>N#;S6G`7y_ zsEb0h@-)|W!j@+ zb-5R6{?d4iOX+U+S|=dMNoceA5j#KV5AxfiUtQ0v+>b#_g1$gm+{z`V`OanCn7@}B z&0RD-0Zx#XyBoh(MYP_@ortW8&ItT4})?NOO!-z63=vlQj5ytTitXrlGA@tVcx=$AuWv$2t>Q+nKKclNA6Tq z^)U)b%c*a}uB>aOjV+W7Mvs0{>y(-r%nHb!{$aYPCTzGCb|+dwmM&a}_l7F`FJ8}- z;om<=ZhhN`ZR#X_pYSp`hWHguYK|x=1sDu-aE8Z9nF%DDHTSD_D0vBN+0o#%s=NmD4|bh+{xD)k9zqp(-3RsmQNuk z8f}*Z;}WGV4?@Ln)#%w=l z^DynrkEvQ@*X2jJhb{R8IFs)%c&Ap{c;f{OJIrd&lBBy*#Dj|>RKdzhhHI)V++RjM z5+_?Z1HOa{DYY%j8aQN?~4%7;41n=C-bb-BOirp3StwUt^Nrds+AvCw+?Bl?!`wKvzTa7CRk<(3Qilu- z#zbqPqka~`b>BLePI-TcREbU|($_YgR8h@912ln^Q8+%S4B|ma#d{AVy#Ds)^hJyx z=9Nm`lTPq^Hc1Q=O1UF0&fr9sNHM`Tt~PNjDi^kG%|^%6+FFSjUd zbxT6#l_K~?F9tj7iNdisavO$-+rwwE)ZE&7{CFBMGwl{_E!x&}Cj9&YiM>7Rjepaj z-165y@K1}3GH`TOlG|9-AG>ZJA44XbAE`9s2183k+&PscVq8da#7bC$pR?<}MVV?` z-~?saFp?#|hHK3}!u4%2ev*CBq5P>=bPwKxi|e?#>vL+1@ZL=;MGw+-!8^>N%EU=M zE6Secsv4Y<9>!}0f9fUfe`d;PKKi&8EzJ6C{l5zUK(YW9F4$}d#Kd0I=mOi4G0+8< zh|LNZVYwPzAjSw-m#un(kI2*vwp{31Qb{n@YWxlI{>#3X@rz6(jX>kYToD7Wn2GklPZ++o zv}u6$Z+!Y+{uZB_v-^rrDw~sHwJ10Z!em(XXJAEI&E_ne33FSQCUi_|oE5)_$kr%l z9t_WTEf4!+-*~q9=tPx!twe{k_0KBhu`zRAhss=lyH^*q56DctCZs3!E)!yi4yP(2 z)+AelQ5^eitd>A>=xh|j$V^qRO#PJT3`kDTvlnEw!AM^agUXymn~lnK7l#%SS*1gg z{aEM-j7-In5S`=V*NcBEsr3-rs@SM;hU2usMfMvfrq zE<9n_f6sDj{U0lnY@IKyf{)@u>eLU0|q3>Inx>NnYf z*I@8LQYWYP? zoMy;i;}A%iTAM*YJd$z?eiWfuk=LNF3RbZN>)l~9`@w?yC91yu`p*R`YK3!TGM9Bv zQ(UHPVuNPcr(;4>4TmlkJ=3LfUcAAMLBn=w5O$M2Sr&y?DmA*e$S{eebMp@|6dEd0 zBHF3(Xw2V!3!e3byT(>+NSqi$*Qv{QwW(hGu)`wbm2|dv7MHA0D^iwU_SR_hV?M9P zTlc4B)bY^#e0$Q}l$&UiL(0a42{~Unst|rn3Su|+r!SbMG4id;TPB0YVQ!KH@XIOh zD2+uSz47h$-a8Yuvt{wvTl~r)mI~v&_faO$(Ews`h4GT4q}MH^l#tm7B$`C~>>Lq$ za*!~1G37zGcAQX{pQJ=FvQMsFS9GcRy#$IN9KK#(&y9>*t6$v4ba=M5mdyCF3S{HJ z$i2e0iq&0HL-FI;?Y<|Sk}Xtd3fCjw!yy=V4w_;X&zpW1>|qLiENe!!w{!b^BrZH0 zDe?vUM+g-EcH{n^B!vg2MB#FO*gVOgU&$DryXNY+`briV@^Ab^?kz#pNGV1oN3a>^ zc?bdG)rXxC11#Hg6Y#UDDaqnX<}v$BdIa^QP&k+wkTs-_1B4?Xt$$SKi9T79NHn9h#IQWEePLrF(Y>PK*|wJ@-_d z`1Gaot8F3k_%zKEM0JJ)5HaOC#L{c+WBv(~C5$5IevUWgi}6Ka99+^=l?7FtL4<;g@C zjnc@$j!aJJ9;sf4l_uAD1PkMAQ1kA5whJ<4Sm z0bw#i@Pc+|B(t34@aCEqyLCiVvdIZrj8I^C7jQBS{bDs1Llql|AhR(Ve=Aw{(#5NH z9NbF&5EnR3I5I&c!6D;rJZ#)~RO;E6X}$KVdiG0_efmF}fgY@=%!GTiZ$iE}rOq>K z=a-B68^uv;^#L@RUY>EBaxwM4*Hc#d%ufn%FE`4vBB_U4;K%Rn*3*kC+tAguUZU%N zBPg;CddbF*R+-lwrdCrB!!A}h(+3K1bds1J@qc+mAo~+E%Dv^JvjDv`UmT<1BH*II zl|5sl=roK&Y$WbxC9U&M{kYWON5Bv`iH7ang-YOsn(4IxSBboF+{8`E_bxXsnLXfs zn?=Xc3KL2Ex3u4|G=OdMJAqY|0)e*Gbk+EI5p1QTB4<)AvJH0%o++^U{=fNQB2gb8 zQ#;Cs;NL=vSKP($t3^wPa?D`U@c+a&zZ0{e&u17F2sC{{qiY=Se*5$C0Lxvv5wh;M zVvP%(czW?O(jHe)f^pUzyQi*bs9;}9kL)65BZ}fP@nSf%T>vK0;vYgFACh65SbRnH zi5hc7G@D^it9a1v0vTlcUTzv5Y+)S1wQyuFA?c8db#4`8z3 z6kFKB6)?c`WM?48$4lrglu+~w6;sJE!lR9hz6)pf9p!!x;x6+|lHwXk2>%so2UZdpim zz*03h5FQW&a8*>{Z1p32Gnh{iDXvY5B0!e2t*C_|yfoe+kVp*8D=e0g>&T@8-cd;P z^{Wf~ES|}iD%+&u3|M z*)9H7Wp1iVXpXiaWjC)p4cJBDrLjxqMLt=y>OxEO;<9Rrw7$A8o@%|BA z(>4w?*`=S3v_G~7A=VmPfMBq!h0tZ$XWITBF*+0qMZR7Dh;sR#pNjBqgKlQS zJgW+MN7zvEM7ogWcQA|d?Ia-fun}gOKqtmH)KM1A+e}pag7fz2hsQL_vTDSq7nWwO zM0^ofTV;!aH8{ItY{-2Wibd{`v9Y!2g4xZ)5^rBO3;BpMihnatKF_ZuxS~ijDu$)u zszdbZGky+Q18KeC#s6UfBT*L@RDuJ zg5V13IRhVpDYz;_;J-;nXK>d2!W0@iPAclh?fpx;3ngs)p8ua-aB^-<9!)oL%$-m=K@Niqt@CT+yJ&>%l&CUj~1U^0u(`2d2 z{=tSbdN}5oxW!1J;yE4lz&#LemClGXu*mvRu{6&gHOpj*Cz~!Wlq6Naygp~#OdZ!| z?lR!aT&DOJ30Nv$V4M&Uj~0agtPC41m=$y(6+zwz>Ej#A zAR38UtUxKmGMM_J1)wsbb>1RZYaS6t_9Yc`Z}r25yxB%NTL0eHW;AOf+eTCQkd7eo z+7X9i<)ZSzxX$=tub6`HW~uoK%Pyv7@^(~u39nK_Toyb~taCxbNi;k>f7-+hvH+-S zp76%l6aO`B2*p6#SYK!zWkEg(a$10 zDx3u}Epz#0Y|s|ht_I?aqVut@2DDZl7*;AL$`<%{v?5V6X9xUzi4=K6O7AGpHUnaj zS3;!#Sb>qP9(rUF>&pYU&_(81IO2+wvts4$^HwNR>55>LbpsK?8vx~0jl&Zf5H=ID zXUc5l6gMeHW#4*CrV_BaGGM_lIAb?TUy>;x)riom)1@RS*++H)uEX)<+JmBYp%NQb zcKBy^o880|QzxUIl+MpXBm)@$VU)X!BDu(6T2V3?5a6i1j$0-7hYyir^$l_^65{FiX)IaW46@xLlaN&8{1t$BGQMCT%NN zYJ-pnQTR0|T;WUY>_|MPb;OV|iDVon}Xgn8FX3YMpPu#F~A-fKSQTKvv!2^NRUyBrP5; zk~>jIml8=qu2v}yNYAtv2A#U_VQ0-Gj-XzKwb#K%9@jv9k8GTDuh02iS85Q+CM@tG zPZ_&?Emuz_Auy@Y3Gc!UOEc3MjN3Vi=*jP~6&*_+og{#1ttssVU4l&gB`iaMPsPQB zOfpp(-hlP=~Kq-eq!LZ`8cK`f%jXuAAZ7X-*x?_LmLOAln0&yM|`*i|%+ zm>NizyjL_y5I#CYIz?z;`15Q`QHQ!JI|@yul@d-yGObdx^)G&T-8e1?GUqNNO`LN8 z&&b^_HrfvhPEW+Nb~QZElGmMwyy#%N)72LS^UW~I$+T2g?haCAFudn@+IrS*s2z)y zSUR*VN?c(Xzz#!iB4Q7mVe1__8`Yr=!ev}Tv*{(U2ypry6%NhG>=QU}JkW$EH&`uQ z*jStND9cV&VtEo8NUhJr&ewpM%<59_Z1U;RjjCDV&45`ZiI6{VCW}_Er91;|VtUVS zv6vQ)LCevD89;mnxuOqmqGt~~WmRS_^hg9}&8H6HaZNYmEX;K#I-LY*r~qfhJSTk( zk`hA^73Lp#5uphojQZM-QKU90A=@Em<}4IJ>N^6S(+$5tLRtw%%B z#dHodKZ8$ZXqK#D@xIbvrdFYe7hxd>5*eb~Sy9ziF+~9NXFGT%p=X?BJ}*4#7o#{j zCVBlNQt*w^_LrR#h4f&<&Mc21Nv_qi5V&C|PB0L{8{93ww#AzqgrJwcj9mv4tu0xo zrz*h^;EIGLo*QIi+r1QQ?rtJu`P{p*t#LI{Tv;Tf4%(85a{e@4UY%~XMkgIKh`ePd zSkN@5#U-NM2C9tl>?nMu9TxlHisfRZl4f6J3=T@g%E>S;i9ExApl*hacSL=*tSmW7 z-j=1!g7_!-;$+PxXdc9iC8D4pD?7>QtY{v2m!Amc!yHlgk&1sSLrPM;@kgofxXKjy z2n6nJ1^}&r;~f{!|mKL665({iY@P%%Y@lAOO}?5#u&h6%K7TFfmt*@^4pq0GREn`_gI9Escm8 zG<+mqWFP*>k+&mBsB%M_ofy?Bvmu`nf*6x)w&yXgF&r= zY~pprSiWh0S@)B2Sa49Im=jj0B-Hk&U!(|KOZQGOTw-`Wahb+eM;&9=p@;0Gg)>{@ z>B9*E)Gb71P&QIj15G9f^&T0uNj zhS!B7>jz5h?E!I2t;?hS5Hr&C(62L28`i%I+1HH;_MwZ~Nt5-?a&RlwzZ=MoR*Wqf zqCzFss7=;Zq9?*flCicOcQw7i=xN^!>M+;Lw1DO*;o z^=Gs7)d!F*>4)^!jn9T4W2iafN87Mdr0fl7(rgJsM|&`T+Hu1)kV26YE5ZKvI%3?( zU=n&%R=Mp?4__uv5Cco_JmV3~{(3Tkw7%Wa+WNI@sG`crS{m?-k9x*y*+&y(KaSO# zPaxSQC{K6x>+jGR&{z%tuan z%eK250m~?hm+d9r?=b_0v>6omv=n+iNUmc@qO|T@-&;F;6cCU!E#8(FPEpOCcSzV4YJLi`nHEQ(b(E$u~%uFDwb> zii)Oym8C%vARoXeQOuVF31KU##Xp}W={O{bPS)~EA-G2cvv&~&`cNKH4oRXAG-6-| znj?bCz)bq_NkV#<)Hz5ziBhH&rR4>Dv&-RP>Ks-76td<}qN|D`sT zN-$6rfCTF77UPtK2J}*s zG5=+#I91^+brHm}R|QtDG4f`y>x1jFc`0iE4MnB*rZy55TGY9{R^Zu=NS{6d2h!WhMcysZpDY zJ1S6M(72fja+8c78tXggw2Xr9n-+}2a?e7{n+nNZdCg5owN&Xop9|Ws+pa!Q1$aY+x^P6RN_) zQQ@DIMN~Wr57htNX3^53ad6yOVq^|%-#CLnK4C5B_Dvj6l!5Kirg**$;{(^Y<1Krt z5&eZCcGwH$KGPzxWSjsNwacp&8(bj-5<3RA`e4OhG!*uV@!NUE_1_rV@WuV+TdVf*Ce({N4OT7x}c zd0CXe)vC5k#r#@apf;;ro3kiJL&dR$+rypT<~9ajDHFN7Q5t>?3J=oCc`^_$!4cj) zMHk=vbLL@;(Iw%AkPfxEePyjXo3lyQwPK{$1|`yUrbYJuwdo(T02+Wl#v2W2{|&} zDQbF|uE1nMS`olZRu10p>(=wh#;tgb6 zQr#XY!YnMFY#wG+^oWS2fmu5*q;cJP_U4|FD1vcR{c6BO0RleMa$*u>$*OtALErEF z%QW}K0p#-v7J5Bd%+eG7-m8<$u(v$GS>5=0Xb=06A4L)cjdo50Vi5~%NQ#@%-0&?O zFKt`mW|oc-g*X`uu}X&!iT?EXm!^0spU7H~Y?|iUBv8y+i10Ao{=mwU`PQzM^nEt_ z)^*1r3UTEO9f58GTR615?tJ|vsYd|xFX$6`v6JrNVg?Mm*&^s- zq|s{)Da3}2Zwj*5u|s&;_Q`X(d?EQno#tl1FMY&UzhN9*M27ZeX~8`uWYTv3N0?A-nI9C(2w#>_Z}G|4Eufn_v|#FxpiTGKG_4zhbbuT|x#8 zD>4=sZTM_N;fIdHC(y$SnnINng#uvFI9Nu&So_rZQkW3FF5cFetLij(p>$?FTGLdL zcuwFpa$&vZp^iod5eG76LWTjWv?K{lOOg-NNtH}>*tE!rF;y0lkM#UEZ2pWPlnNn+ z5#t{&zJ+`W&$Hl(;@a(=0RJ^LnAqS5gXIZY0$_pk3B8`qLmnJTJ++b& z*p3PfKBA$Nbpn>BRIC(tu>ji{_T3{nhN_#s%W?yNZ_Uxry&1R=PfsVWnl2mArWxoM zi)raqOd@Q{%x-}+^bG;Fg1bgPlOpkU@G@InLvI%245C2RWL(rQh_Sq&T%k(ToNY@& zqT>j4Vh&Ec=f`f$;H$G9lN2s*iWa%27RLyuYmHtlc60xWi_mqn1A`WfktHjx0AS0l zvaXS!`^6$7wjGcx+kw9YyQvNpAEV&%8-}i@%4wi~%mKb^c{&g_oTW8%2-9?61t+Tt zAsiZD>YBudFscWjq3BUBNJYyP@?9kgTaUK>oX6r(l(EF}6^{Tu2s$b+7l1c-47Q?5 zFWPNw;}xSwfpkOUKuZP|8ITIDuap-Vx6%1>gtCJ=-(!~uhSoVTDr^T$60nJ~s$`kq z)#w&wG9IwWB|2qRsy&INXQxSiA?B4IryJWsV^f(~YB+?G!zDWb6ej3+_WRjH>Tq@q zdOqzKTh)9PVZix@=ba~_hL1E6Tb?0>npnFWx&I(jYp4PhM+Y)nrsEtC%NA;DYpYnb z2T$y6Lz=hY#|13r2R#=F=&Q&u9ZCX)n6)I*DLfcR=pc>MhH|`87kN|HpyWdvMfBPd z53*mX!<`ebBAO=BH3;3Q>=Sd`Qt{K{dn!fFk;mNTMTypZ-Pc`H9vSvl82NsWDxkfU zu334=yfmvuHY@dnI-BrXu4ya4oE7XN)C3vPb z^Cs!BHmRH%rc`a?M+A*+Rul>HRe5K2sZSy7unSO^qx(R-F;&O?(s^XoOr&xz+v8Zd?>m zwKSTqk#zCI=$Mt6p2PEGjq4xKn+B6NLYr;GhUwukbmUUDeVtY&w@g5GM|S__Pos~# zV&Nw8pX7@yNwq8Z`tCj6NwN3xH0j*a-MXW99MmoV4dr>>U?j{f$w1B9GCU6z$?#D^ z`=4{))~>16`=sXVV(Q1f#d=bSqPosYbPVSK7_uuiMvXn1BBlj}{&xmk*UPkg(_A*j zIh%C{8EnN=5{uLe*a(hv5d;rjX|Mm{fd`)t*D(`2EPz=`LbaW<+4j}K6znA%WJ&#@ zRM8t@Y4Ce&2@LZ{@O=Q2RBY2K<+ZZu9H_nJ{!4vZ05~W%l|58@Tlc0=2J&>{&4NXJPSt++LGbKX_$6m6*V`gQ^{GeXC z5Xg07EM0(;ylyj-BeYHnjR-Oa|2FkTgG;@=UbZ7^2(*<>HIxyJWzRZO#qy{p=(1F) zX6TK=rZYd12{i81#pC#8$z_~$S3u_L6#+ZVP;0SRK9ZYh?u!nnUM`A-8O=5y1EAr1 zKH(r~K(f_Twp^7IW7E8_{jnxILZ{eyNF$;POUcrL2k40?#&tzN#nZ;EK;5kI=kpFn zw_X+Nq4J1+stEfRUc7UUvRuhZgkk^(hJ4*fH3IOz%2kB%J#rI;j%dJ?)cM{WKLr7z zCYZ@Qx*zPOEQ?ESu-P%V~Rz7~1y9`q7qk#h+p~Woz4udAMZEO6dNu88{(Zs?qxbs-ZR5%b5ev1S! zZ)n5RlcaaoDUH0i{-w3LN8$(q#{%|V6Krs@WwTw;D$?w3jw(g0dnzc5q{ZsdP29X5 z2$@ZtA~u{OZt3SDZn>;cY{ooBuac($GczF;nc7|^T7$A0{T>4 zez=6@ni$P~m^0e_Bx;rg#+z446eD?F?uHE|;_Q9mKnbuY$ z8jR7%Vp<&BSO&Z;4DT2$g(oBJ?=Vn5{7FsqV`F1Pl+Hd!FP_AabKrG0!+9)5f@IE0 zgNfwRc43r5X_>Re8>$ZwMf~&&pb{r!#S6G+d~sp~ZK*SsFR~ zt(9nO$xaO|y{Bopl_QVIo=>Tt`NO(0vzS5=I% zNEoS=S@qOq9E7cs{YhuI=+NhMHy0)k67uzc%neg2{Ed#{AvS^D*9KjUGZ5jjuAGW= zGz`1DJ1kz8f>k~$Fi|HA5+_L{gepS27Dz#4nn5AcYXk;dt% z>(R8Iw-;m@+B{YR5fY(a$nGT z%`0e(;tl-P6~KN%Y~MGr7U~}Qy+SRl`%$D=z1fKbKr%%=5VoL?o0O;ko3j?<0f2ha(4>$kNO}--rD5NMTEhV1Gf;!JIEV6EsSB_r9qIVzNSs1P-+vE38JU zDtVn&HoC^ZoSZV5Nb_t>+jCM6rpS~O1nA<~{M|CeLA%o3hsoj|Hl`=7xkSmm}<@9{YjD73I4d$(9vSPG!Y)nCp-NS%sqkUX+rvvF!BN zUYs3&qZX1yvWH17wKGe=i2_<%F@bL~yDb*-3#XJ-VorX$b{Kr7|Ln0z0&EED6w@p#f@-8%H}$5DE!7 zu$rrNfiAdM77OXsxF4F7#vFPbmU<;7S|o~!X!4r2mq`pR1FLwh(MK5 zM3o69>3Kf=g(rYK^a?`KI2t%h1W+Of5RzAqq zl1)%N1=`jZfXzy%IakX6C{Qf8LvSco+}(>yaJS&@?zF|- z9f~`pP^>`pIs9Lo7w3G0x$kRc&)&1v`aw-`%lVH_)e1`(G3Rlh(n6DQW$z_G)oaYm z3Uy0NN@s%-(cwAiKpd>75%!qkZ2)z<43Qf;Vb-0sT4yB2V9jnG9DuCMQ&6tV@Y^Y5 zva^*d$#?AaiYdA{DQ~OqZPgtVJpJpf;ZSN`qzxzepmvlhh=(j}TXMQq$2n{<8UriF%>+eTWb+|sgPb17>^1tqp zI8Ue#bDn9X;P1`V8Umj)^}mzx+V8s#j#3*Qd%4Rox00JivB1=eE|P?gF+Zs-XT09; zWQQYJR(dck9-G3tm1aX<83mbRV?sCNo8c6aK2P1Z(V{=IY2Gapxf)nouC*!8=O%U+ zd;2dT0f~Pe1rTF>j8?1?i~Sl|)uCo{pWqU}%x08kCDWvC>*^mo>H#{JbYRPX z805*I3xQZ*3Q^zD9syK$S!Y-gv-g+DNQ|P;w6_DlDwAColf@r9pIh9P70H8q(F^95 zt|G?4YTQ$`pQkxgAk+K+Ei#+oD-+LAV&rQ3XC^>53chyP>ftEr;V4Kta@%B=vQF;PBrXrsM;oPlV0| zys?|!htsz0T>*6|qzp@Tl$+}am&w7N3$n_`X{to+qKz|TfNy(DjxazOC(qBUVe!Is zGu%VYYKe+nvPoT&q^HmUirv*8&k8Pnh^RAjANKsL?iis@XEKJui!5!NL z1;josg)zA=(Y>J%S~*xLt9i?CA?FtpPms6gKYBx^axQ(Y_AZW@=g@Qi``~6ne=I1f zqb|6Z+f<|CuY~%U%S?IfEM;PJ!r&dPKoes!oq0puG_!~SsUz${5`n}%j6vG5%A&>d zM0uLZ;;w0x4s~}65h=b1I8K~8^wwD`1u&8 zX6NG`aLn&s-OxU^4+>c_E`pYxEz|ZrMXe0Ci~ZmE(gg#INiTgY9uBywyH1&H-dgK% zl-Pf{M220+5*LX*)8(aXX+u2f-!G~C@m?`<;()7Vr#KkEe6=- zZ?h>!6dNs1&*s_mq7H-`b_z-ivbKic$3)U9wm0%d$vXGaMHG8m9)v!t)k|}kQ=^4a z5UdNIx8N`#+sJ4T*6A(L0kSIK(eYS^=Dp2CwfyrEVbHzDF+?IMXr=lC+vDY@kpAC$zj5a-K-NNKyQB1X--@ZI z`-kq|mP5D$lc$B)d=?ouh=>Y=MAskA7ijWs5=R?9%LyUp+t%t5eHdjD*$i6ayr~O)5+-J~ zTb?4B6-${{7L(!#euNZv4dv!|Cv7jKU@mALpz`BI+Bo^U3L({XGK?ZYe7kQc+knmrkK7wp zWWJ>s6q2-1muG?69Paep0+QK_xY#LCY(XB1uik+k3#B|V-PL>BIMx^%*oA_R49hW) zIP^*Khz3%m?{rBDh`k2SdRoak=Hs2jug6v$x~g`CO@HZj`kv+x$?|4SHdq48Eqx>M zvA1n@h3P1O^=19Wx`xd0`lLdf5jjF{ejDslJOEHPHee8k`}a-CAah+!)<_P$5Ph9l z{^TMi$bQ7syt;e^BR7QR3jfNCV9r^<+sf*zyYIIuEQ!YClDPnQY1vO+Q9qXyF^ufD ziFIR{YEG`O3O!}b{Xfr6Uy}4mA+LZj`2eVtt}I#qI2-k6PK4U1U%%QSp0|+n*{#CG zOB;^Y;~kZRg6fadMb%mOkH_h(hv6FoMPqR=+xvVrHPH>B9vX!m0D6W&!n{j4>S3GE z{i^IZ1~~Revs|xKH*Qi3`&6g%Qc?9Rvm?(+BEgjaH{jo&fL}-_kAo$5c&1-0nm4!P zY$x-$LRG?k%Duf65^Q$w(~R;S&Zi-tn=iV~=Aq>#RCjL@nvoK(ZHuW1&qmJFJxvs} zyT?$!9{85yGrI?9BXr;-7zZ_f15X`+m-_TI#FDpXYtuMdk+8b(oMUwZ>;tA<_UH z3uTXypF+)#Rq7E0GFVSu?=XM$%J5KcitI@?eD}NE&ASN4BBwK4`-GLq2-a?-AHp{N zM+yUnV56%QSml$jMHpF|;lS;`5Hyx#h=3PBahdMpR0!?3WO}w!{w1J=8Jy40Ivb_G z4^5Z(N1+Ex7_?IomK=bkVdCszG~t0o_&6S#-)?e&U;sOgF&LPT$u<`0aFaSdvrY%Z z5twqJl6&z9Xw1n}ze~#;3S&5xjnl#zd6X<@Z?Pd0%n4OeICVUgpC%%1G4^x%m2{}V z=pm4l_fJSS*HbZde;T3==wMdt@xOW$27yec7N?7B7|7- zOFAxI1+R3$o%xJMFS9yDf;%-IGgVgwNZ%%iMV9OH($byta_N|pT(RU!JVFW?RkE?m zS4_#uNxVKKX{#)Hf`VJlytXeg=}8*iHyN*Ff)X!Ph1^LG^L#gy7^~!4P!m%G=MN@^ zDp2X|3~ZTT4|lzF(?>ctp=n~2X9z#N4l~hOtKo~k9i^9_y&m`k?Vqr9`~gb4;%5vj z(5d&KugMvo{RF=!kFn@q$em+6r37u?7Ab8gt@apRPP=g8hiPjO zNaxQylpIU8ZBms6x%J~XP}qe!SCKthzwo-9zC7u!B4Mr?!(iM=S6T|kwnjZtUrl}I z-}-(dWi}}=eZ^yiZkhQ3UOMJ*(2}y4)5;Q`fI1L{dkr(~+Xy;+m zi!>X63ht}>Q>*PUJw_hGNnxzd&>mfT*g3-voB5zEt29TD8AimfX{$jbh=-|;Qib|~ zsr!n+_TPd3J8lw>6Y`+&-!W*4)`BO?S}c}FU=$LBjA1-_1V@)Dez82xqaGmVyZH?OiO8(g*$Kuw^oW;-&pR@J2P_DD8 z$M>Gsf~ja}a+Lk>Oq#M`vU)ZN4@+aBqKZ4n>; zO#v{nXz}68n~^7CTQ4BEQ=d{FMJy)I6Iv(46YXh&7HDeydVv+;5=-&M157#yKL6|u zh1RE%+wA_#b(j9~Rq%t*1d4B+;~Qzb4q2;w&J=RkUBy+N?D-pTYA)tHVGdUtJ9H+# zr(Rn&>qqylvp8cs%HkOn2kSP)Z_KWj40I^gpvj`q1m6!`9ea`xUyhE?K1xx_oX-Lk zlU(^ceyijr&@1CldASk^3e1>B<;J&>n+?w+f*hoiW*8#}U`$e$xTs^At#4NzXod2h_PVq-50lhX=-9vL=}<39&Vlb_9T z4!}Us7QinzqubRgBb13eh-18cGjI-DmhgmF4w6cG*7e+V&n9VaP!@nbD{q57SgHuh z5(9ewE3cgsRf9URbjDP2>Xxk_!E)8lezX<)-uHYdttf0Hy7|wZ`T@@;SU3DniGfm2 z9ke&9Di3A$XP~x6cVR(xJ}QvhUAhqqdry@GNTL~p!h6VklkVpBjwlBG@zX;F|%8Om<){M0ZTxN-q5W{P6_hrspiiM%Tqgn!1Dp$ zNu}=8;&_a7jGCbwNHy#!n`@tz-pNg0`Tbd4j+Ex7zW#BOsK02dk1$baTsIFs zDx6*dfB&j`!BNAz5{0FmRvL_X{;hYzja)|wE85e=*$Eq$>BpYTGxC^P0eRnl)JaeN z6>&}!uy2lhFjU%_p@NKs;3i|bkkVx9)GDooN?J&y;elWka6S~ zYrW%JDs|uQK0L}m$o#tsDmKD`U zqFhdCzL5)zGYm}?79&+~Ymo|g(EKoD&t73~al~<`Y$2Afjjx0P7|Bsx<6LINxXL-G zgP{)kF~g5bKKoX}wP{hU8A;hXFO|h4ke8*leALD8r5kA^Aun&FSAL~h@Q+a?CN6XU%`FtR z5_{AO2&#fpg$dum$8)e%qv}5XE}zLIxGYUwh%hSBhW^1qGbRMDk%NF(CGXs?Qftb6 z6M|_)W#p71EO@^bOY}+4QdWy7CXuSBjW9TV0xIyP8?|nKH&B}WGJh3G#P4?s6T^o+ zIuCsLSVqJv?ho?>VpRpP7;mEGjl@^jMfR02|GPVRZcm6I!!O^>Z} zaNCi3DZ@f-wR;+}VCxU^>Sj(yaoJ1yi@6l7~xPLO>u&=v^$aay%LmrNL=H8T^OouskJJ`%mU8l<*^K%HL|} zdphF}s0PJw}ft}UQ6js!v@1Pp0SzPK9ybwc^;ie^nAkCJ$+Jmj|2%;X*y z#1YR)T{A~AJ8*A}dTXE4!p4a?-(QBu)2usGEESTU9-`ajU9agL-#<8Xu}=Zi<>ByF zSWd zX9L5X=OzZOn)+waKl&spe?}^P0ahFp{K9o3$T?BypQ*5kZLlYcuxD=X6m3@ll(b++i3wM<~h_PV>Sv1R@#W8^EMFZ`jKGcY*E+NlI@fb&4Guy z>v%0~aUt3j@&b^~vP!ExyvLORsYpas$omcI#uaZ5sCUhO{ZA(D`eiCL^!qlcOGdk~ zI0t{;ZzP`rssp2*0)satLS1I^3bL(M32{rO%DcOM`yam?{P*{rVBG{D_!xx?u;kU_ zc4Z}zI~`bX@QS0PTu7Q~N-(xWrDnVoeKC9Ur(%L<@8e+5kINHIE`$eHN2Rx*&SQS$ z^X8njTEEiP*s4tsF4B;9R}-VDk4LG?Au_lS>XMV23PNQw0#Z*-AD#l>B6)%8@2*rV zFm$|$)0}E|M6h%yO&loE;lEi`cIxbX3s%rD6~wyuD0NpT9dU+I--prg9+g~8Hq+%c z?@0DGV%ai-ZW}PQ&z3nRHg@X>b?!VAZwo))ln;Hg>5~H#RL)V_Pma4&doD8!Rqvyf zk@#dw=bhrdr7gu0CPCM>!cT*A+Qcu$N8-n4FAkcbO)3LoBM0FF@8>~tTBU@#ianh} z9i;KfDJ6$Ess?!YYK;es^zNQmmCgIq~XeW9Y4cFRLr=k6S}4a&U7(%loOAPhWw3G%3u*LfX-Vi{_oE?mL&c?QIK#U26zq2^xgRsE`i|St z+bR!VOH8W7w>$JIQC~A^wI11}`3Wa#l0Q*V8`9@02+NLyrjT+@tg5v%)lG{Q_OGlF zmrslnv6fdQdV;mn_AOXd&~44Px9QEk2Is0ezi--06EtQ% zjy!ZXJQ3SzkvZ;JxSul~ra3}YPk)&cQzlAuUvCJvA%m`24MsCjoe+iQ_#IEW*6=v* zObc(Fei!I|>oa4K(DMV`Byky+0MLAK_J-h15a|i|N*+gr-t*K?j_SJXZO;2GsIj#V zuS+=51OmSwp1hS+E`0trdn@S$A!CD<@42f#nzFu_Eh3m1zq#t{A3&ztXudIFt43e> z5dK!@c*HQ%KiUnw`|Luc%i2NL@qDXk1Tu;rL@(a~$sH-+kEiB-jqa!ig)=#3$zh&vUc z@v`J5s-${?9eLOboJ^VZ2a}b!F?T|DvUglV95E5viivx69`TM8Lsl21QT?^$*wB>5 z*@Ir0-kd$NCL?WC)ItZL4>gSrP42)MDKjRiGu?ipP7i0LpHB%7z4h=T!@-K;G1YU^h34_d=qv(%X;B!$e;N$J+n?4bd&dKX_)~!w7Y$R*4 ztMmN}elQ@aqx(`%`zi~uZkXNf+WmUlJ{iVTm~I?P3HlwSvNSvocp_rzmx}SsXp#4- zITtISVFsN4H2N~7gBjnPm)tG%j!e#+GhPda=~AX1DM+j{p71+QmQ|Ku2d~+a;L%*=_~sA%cz^>00jsEfV_71s z+tE5pc0D?+KHcGV=d76DU}su)m`dM%{RgmwF}8aU-b*n=e}vk(s@3{U?oGN^1bR#@ zG|2@!K_LRpGOISIDO&oIei`~a$|kjB_a7^lpXq-l{Gt4p_d~6sB2v&jT_Wwu^&aBv zS1|6X0B3h7T6?Nbu2mqT#w+?7SGLLKJeD20FODOKtz*D35MuEyB~OmK)90G)*z);_ljfip$Bkhva4mY+}oJB}uM1VASp>G@EX|{X%wSFOeKo2%l?9R*+U5RmN17 z4ch3m&ZAacP!fjX9VxAwa0b}!U|JI|z=+kAjN%SU9ByIXAR0Pi7ifg<_BI`IoGT== zC{3%3RIQGG;V^&WMahcb!|bPd^~tJzMqtr0i*|0;x4&2U9@!vqw-^3{on_FO-_cu4 z+;S;d>8*&=?)^8cpI9hAZ)4x?BoS=#j3l8LE(zPGyM66y^$R#swQyzOH7P*FE`F?F zTZ+7FlXY)CN)p|>$dax%$qikKDYd{%a4U^D=_>p8!%O3gL}f?Du-3$_=MY^Lr{##X z*+Cvt#bHpNCz#17_y3yZz|>S+;^}HxQqi#RlBagk)`<1SI#{&YGAV#COv*n%=?oZm3r%DH zZWkw!EHkeMa>h&M zGI$&awm$o$nD3yWRZ!giwr}-k>O8Q7K5dLTA=?TBW)KrGPeEN|1c==`8N#>5Chi$>OF}1dqoI30SQSY~S4W z^s13p=rn#yeg;2AuL50NIL0$@ThR(0_8Ug6cE+Grb+wGp+il}*`Q!m*dePMae5$2Z z)I!`VX`C7ibH4@^1WWFgEOmYpaXoQ7P5+cbQv00-BUgiD5xwR;HrVK)oC_&K_GNXQ z9Pq~i0@J3U#8|2k-kv-%4DGF>O|mm{y}F^?Q`c)09Of1Cx9q?@)!D#3)t_RGzHt%C z$hx(ASr@mQs8&xCIj1GJvlkbxV>Rg4ZlNKeX7?7AWjBUQ_kOi(faP%NK%4`@*IjzP z*wFvhW~udjw9u>+gYS%c6gehXSq>568Jh{QX97-E6{_jeJvGT1im}mL=E7cVZPb+r zmr?Jc*d)TCKyV=U==CBH?8F@{iLMA)A@W`EOV+-S7*@d%*gqgjG8ENlCt#Kdm`mgO z0-rFLqkSQf}Q*jWs1NsvS1H|1OMh~>maqI_CeUI$zrJ^3zZAr zAa97*GxUIPh>w{Kz>0t*2W7z+I&UGwqVmqNs7!UPbpIBbf`~sbgM&d1UnZ|JYVnjS z$Pby5Q_fPNyg3{Lo!EDdX7kv&`qf&uvA3>6UJ0v4Dy{Ex70;#DTTJjK=AF$ zb0_x@p7?(4t4zOmyPrYFfRz-5$t%7gMyJzJtD^>*;;PL5=_4;(-7F3r25kbj1rC#H zr1)DZ?tB*;9wk>EX*WU#a zl&+SgQA$oJO;j4kD<79gEPrlkff&-`qWDsFT?(SqsKV2J<~wH~(R5^Sn;TYvI?dRX+1dSjwku(a{(kFyo7ay&pPm?FCWZ0ny3-6m51FBsG(z z<4jB?1d$U>*=KU~d0sN&;9Asp_c~yjppwyci-p!HVW&^NhD}>-x_Q6LwGL>4u7m{O zgrySe$!oDN8V*fq=Dz5A+s>!vQlr%>J^U{^aW6;2dg0|+i-y00v5yOIdO zbHwpU`fff|y!9{_ME>bxGQyR{!6$qx3`^7SAlPNsW9Saq{gSsA- zNUFW8c|3oth6rrl9-L$&M-3^*3ur9B3#&?lUDKlX+z83h**}Iv$l!CeH{31Y;@DqB2L3m!}mGV=3|N4Q>zj+u_8qFuHSk3?qY!`oY0{!sFWH4M$yi8+-LSL| zrC29U((#guh*>Ddo{0YFp^k51UO#c4+9y$0kza1Trx|LyiFq$xCSO;1BYb08rjF{I z4waoh0ZWkxF;T*EyQJ?eC5AR+JX@alYAYGl=IQM|(cFYwi$O=a?`tf7TWJWin^PDa zp4ol;1jI9#3^?>ud_+e|=n|6Ln8xTmWZ^qI{YljbkIfD#ddU3^EH{p*5rEidki=?M z$dv21Ug;))3qBE9V}4-jc(YUkW2qcPYN@W(0Ga{nE`}Dg?vxivz~?*qm8qWjl?Y|A zXb2*$oF7H6fm7{er)tQw!2xR=wWgi4a;N|#`H_e`A*G4Dn(jQ|Vtz6E-mVT7C1l3QDDCi3Q?Ram%%W($K1lBsDCC9|UU|{GtPW`J03ZqI< z5J`_?y4`1j)(=8A{w%*#nCPFRF2_?S#w7zooVx^5cAPoUlAOI0E!`Q3%_Pn9LOpYM z>tVg8j+54Hgc^C?G7%<0RCbDGrZ6%|MxdgKyUoN(fzy)JmzLTfQT}rZ&vHUux-NYp zv3#=IPV&mg3}-$jY^n|qF>JF$?IhF}Fv85G{|Y*Th@R-3ss%&aQ4?aa{jsBIcsd<) z3&t8Yh+u~XEDIy6wl(_xV#8+BsV!`yweT0_gHiDc(lw~>O``?7amWzvl& z$llaWMh9`s{i6=LSz`|PGQd{P5Tx(lzF}@doCjBNeyS;l>Ihp;oh;hS1cI@tJX=>A zSz#0WYqs6!M+`b~DXAohg1Pc1<~lzZhz>*gY(@;_u7s0*EK*|3bu4k+tNx9jWJ!)M z#Yxpecv#OKY6h*9!L%uhikCyi^YoBJDQA6Lfy8*nhVo!W7^z7NI|dhPF5t!U5t2>E zRN9~J;p!NX2e74{oy+t|YfbaQRu&_1bg2JLxFY`N_drvQf5ePUTPKNAo3Q*u@j%vt z{A;G5>GxoKX_V$o*y#^)F6YoT(4C@8fdVWYuqGpPm2?l-sAh{F(FFBmuJ7)QMr1edAz>FQJ zBYf)c!Er}djUf4bX&%zJm2+~Q(wYu{5m_OZUhRGG?Y6AJN(UwqLoBoE=%8Rgz9TU7 zutu0C$ScZ0P|(X&vhPd(0)U4=FFb1ztr1FFuFoyVTvvu#YIiE!{w8y1r7eo{&wz^j zrw&<^+4PJbr|PJ&F}w)JLq)K$_Q+7Kbs4B-Gv5TB`}Me)4RP*%`Q3#qMbE(x8FKf}Q^F;i>nY|8WxdpJc{F z{j0Sv!KDyuJ75v`C7QybhGW{Q+W&4O155u0~ZWqPa&zQQ5f*P_s%!^Nv|u5Qmv0 zznS)b|MUG<)OMo?KKa4&9a<>a(7?ThO5N>5eXAi%YUJFpwQLCJxMs-1oNP+}N-)vU z_ZDLIP)0QzRg420i7T%OS&HV_vZw`mCdOHHU~QxkMjtbZdx3r9ZzkYOaoJ%W0Ew>q zK}3D_ca77P$YuxC55p<8ZPBFZB9k<(J*6u;N;fif>BLc`i}soy_H&}6v=jJB#lBgx zMreRcI)xDoQW92f!_|Z(=-`e22=7ila>Mv**rv};?NvVIX3rB1n6#JOwn z+;-}iqgvD!u`-WsC@9gcn<^Xzi|dj{H!bPW9uZ~>{xro{nVYZ01$T$Vo>c6lHjG&O z4`L+gFtE#2gAI7websi4isy{a9a$wJX*yi|R8=>yUP-Rz4E0|<(!KE%#AK1;!p5;t z&_%YWOqe&7w@j;U-Lss}Zzi2kna^YOXCEC>?qp3EFyzcRRJPfeu=b z1EKyv)~-}Cq(Q)P?^c7Pe>^jLZ5CnO$!!Ff4m~$D843zWP&(J6Iy6TZ8CORw+@FEW zuk&~8Es_tyc#OHjJtMVoh+&vH%)zcTW0@%<7q#k*rV%-jt;rg7ad$)Fqg;$D*noFf zLk~m0DKK9gX??-r-{sOl&FpJLy+{tg>5#gS(N}Pr0ICAnr=t@{_?i``NIFq}peWvG z^G75{o_(DxB{Qw^aq>c=eNS$a8DQf4?Z`XF@d}9qPk$g_zCc;wtp@qb)qAC| z-bgM-z3sYm)y_a=p^D3j>Q*Fc(|}Tj*jhY3j10LdqgM7yQ`CR&iywmHYB}p`E zPBlzMe)Xkxd3|7Is70Fro_JJUJ;m7|i$}F+*qQbcUO_Lb3S2$=zG+8p=$=EL8|uya zMl&F}&Aizn1zdyi`h?svnrf7Q= z|7|dJ3>^ze@u<_~DyS?+d<9sINN3?bQ+O0>Lv`Yt)gMu;V`a-Tz#2z-NYuy#OFjKQby05IoC#HTAaY9R=Q5q zV6*(O#aN>U?OYQa@8Btoj0)~pCd=iHTd8R}Zw}%Px%n?_3v$gJ(Vom&6Q~uZ>BYF} zvQ@wy7FN9R+!|70LWv`HT*L&QQBz{#z$SpPO76g!FK`P?&8NWkffeb(E?pV|r2#)2 zzVW(5XR`IL1%qJFnnI5vi#j0@nK65@G{^QgfjVTmH+gk#yB~o!Ej?4dh%Fr{aX~YW zTAnLq9_~>k(K^Ke`H|Fr!?aNYb_30~^w!3!aF;8t!oiG<*pDg=dSkdd^esVl1Qp zD?R1FBE-l@m(d@ac2B=AE87U8bOyDSC33|kv&REJw6$3tjuz7Ck(`ut zMr8Ig6UU`;OXU(O>WU-gh>QC;Ac_JdE^lzB#!c{39Mim4?OCvY8{EWE@!_KQ$#pAw z_M;HD*)G=`6@TPr$V(loW&tRoNM`(8EW$Zpd+~5YUJhL~^2nm*M*zibIo%I-N*M7O z_jkq|^-WG*bCUep6Xf&vUo+wIfPto4?UorT?Jxuc&@yTGq_D^4X&Bd|{ zLeXa@Na@@I0?(wrLiq@Wg4-CWKdp^Yuxu2aC@m8&dWrBk8DSf?i(B?S%)j}2rsaoh zBA=VweTep;b8Pf>IVRORiQqrT8JGIPh3Te9S?HhM;us&L!*j~7?<_*(Z~cF@+E4xk zRBKwiSlb(w*#Az$CF9=!W#qq4V*TxI%-p#;LEUMD1ByT)vMS6&x2b~j+~*G{08@z+w|wrC~YReX`2e@fR((^82UwA%;u%CkDu zOve8Fb1v`UrTi|1TBtf3-jZ~6y+V0RNded?bGzw1LH!-xJay=DzH^(8z)iCCZ_73x z6vyGuoT1oy+EAgW0o{Yp6DOQ^wm&6zU*}XFQh8|uoJ9TjiTP9##qFr~I2^nz8wm?8 ze`*@Pb0;EEq_Lz1g=@UsrrMJxmdrP@T;gI@dkGhu0mF#Bq}0Fep9lOC9d`sN_ z{BA1Tr#JF>RK8tO0zb*Lfd1ke0&Kz@jDMOG?dPOn2r}Ca!SKM=j;_?tHsy8@T zeK)nnFA;-L_MuC|8GvIwOR@>aNLDqo3Y35MB3wgti?1HI|3#+7TBd;TZMa8ciq}8Y zBystV62M3=4T&)sHFPi8ag-1FE)u_dvr0qe7xAtf1K728CV!b8%i3{IHS+(Wjmn@} zs3+@i+P&dc%IEcd#6{sk+W#;YQE?c^vi&6~#(|!+J;GjcT=X;ciHb+cP2iL$WpE2Z z`}B&qcl7|lYKrCdJ!Rg+wYPE~bf=ft{j9h~mUUVn`FTP~9rMWH!y#(~SWt*~wsn1+ zt(VMzT^>zFiXtIQ>8734>(4hh?itzrAySz>08g-h=z9vO=3(cX8x5NW91G7dF|lL! zkc5*sjB2U%{$HCR!#BH_?$%&D+SSv#FfpXCCvJ*8S~RnpK6QQVHPcqbZ_moG^AWuN z@ONn^zhGDSEZvHA@F54kTq_np`@?cjAQ%DLD@A8PmFjah7)`F-Qs6t*Y6A&)Bj+L6 zR0VJC<=DHNr9rCLyFKiOvefRJ6|9ndqsK4w+*V(u;+6au0P=;Q%2j+=h~05qHYo0S zx)2cm##i<;bE5Ir&x8H~Q`2n@!c^@&v`r6Ij$36a5?*0(_U9;7#_=d}K$m*H@ez^) ze%xO|W@@7sTPdA@Y^m;9L0y(_PFAL>!c)YBa zB-Z8f)daUCV%#k@2}Xnmx*oe{cU~MzEpt012)A zA!I8vE)N0tj`$i7j<~Lwh1NxB_YgAj343FugQTi1z2nL^-)<|h0i>#cX0i7V_utIZ zi`DFZtQXZ`nx+>;4z8d!u8WIWOegU4J7Q^+Brm$M%ImCPJJYqXxBLWC!U89&l|nv$%sxc#O^29ELS6=- z7Y{cT^9)N_Snxc$BabR$fo*hTXOgAYMglTOX0CDHo2p;=GKfH>wzd-HCxVM>+komK zi1=qo);@2f3P#W0wnqwXqI$VXCr$OvYU-sD3{-3|rT-Rt@otwn2weGVf5{NtB(2;? zgp$a>l0vn#%_TLD!bTRwTZE-FM4IHP4X=ZD0eg;I7ezm6Stycm1Rs^(3LGhC6L zvTM*M5v-n(nFNX?%xurJ1f_w3BTv>KVe_#GtL~77K}YSmw1mi5&L2uu4~LjZXUm4G zjXo?=Cd4M5=3CeGk^@)-HO3#`*~JqCen*S0{fb1Ek+HLRYC5F(aV462L!CGoRsUMq zbq4zD?8s-%k#xW|U!xy!9_R*mSJ8DGdr8{VX0pGgIPraa*{W#Nfo@(ocq(yMLhBQ$ z>7|n>vO3YPZk*~+HyhuHRiTlRftl-nqFudl#%H+I*+^7lnTw{>UUC4ELjW>KVutH} zW7P?S$alX;VP!93RyM|-D6z5gn_vtKZiqsYr-*>dq#FV+7DyBdT5 zF1SBx$|?mzY@7yPECg0eQxi zm~2Go^vqq7_H0+Z3&oE=;Ckr%5982h73~6Oy!nP~~0_ zsCem@s^X}G>=oO)tQSmKg|wd4>=j zU^ed4!>)*?{>JY~wZ%lt)FLej>oiMJ@p5K!rx%k5p{kkFPpFCnMp*+(mX z75-q>261Yv^zG21fW*t=yyBbXCWFACtjUv8|XYHTeac7W`fxr+UOCaMF zdu%_KYFO2$NB|TQr0& z#6@eXou@$pqEsMNJf!#JLjjkVKj^bKKanj+=w(9QITO8jl%-7HbiO_q!*Pcvxk<%R zm`YEaqy4o0 zz&d$t(pJ?RFF7!jB8^*Wbp{qn72E@Ag84x?yHCHFYP36(iRHU>=A8~cIpQE2KA0V$ z5@8(nN8>W2g$$Xfxt?g!cEC*d&lP`yV>8^SBu5uQvglE~npK0uCJx54z^!Xbx~3w8 zjb8oJ=q=u?gQiR1@SI!>e91XJd*R?*GAM312ie!MYRaYel`0X47q(AlPN?gc4Mc`a zEeaM%gVf=|$WEwD_NW?1L+4~Jw`)35SFhHg`;l&)r#)~No-d{go&R2!jP|e}wsd?2 zXBlTN2rpJhzj4Al6`VoO>lxR@Ww53k8?#ZC|IPV=ApweFZnABqj`Bq10V~^fiIl)*gf0rs1dajxfgCQ(ms#s9sXF;JJ*P4-jBNgxJ)HR%T@}w{t2iX)@(%A#z*iTjmwd2# z6@Hf3Zl?}fL2X>Il!L|>`gRn9VML65T9?`%uJzgchIhH)qNANr1#4weF_&Nd-I&NEJ?S|g+8KEQg1Fz^Lr|mbmJnQHz zzJ3GavSy9hV+(m)?QHkAf!V$77Y8y&V-zT>s>!6%ffRjG2taLlZx^-Q6YTNOczO@; z_3lVXw??u!j35;_Z?@+Dn$};1!y|Wax-#nhRdyhUUwilz>=Q#e__2i z_6QZT`=w5sBMlYqI~Vr&+7&0%CmhwmSL8K^$gPp`-*@zYtEFZs8zf28FXQul8qOh> z_gd3HZVd=1ldxnVtlVh80oY1|1Y4A;6w^a8+0Po-VG@!f0hX3MWm^-#h3TMm{aI3u zDly4muJxpPdwueVsk?7qmp8Hf1^oEp#s;iK!5wtEJpqf6Auq{{6g1X@QUE9wbdmfZ zt<}<(9T}6KBS^~?KmQj)!UK66b=LPui|@D$>o7F#x!}&~Q11d+r#t5;=S1g%fRu9uzn8hPLl6KyCmv32$fFZeOKWhPf4CmllAtJpVtP&DsbRE zbjn){mpDV;nlA3sp{%MB+yLfky(ln7pa;<3Mww7*!W zd{`)S2|&{}sPGX?qeYP-m<1E(RWm0iTVi}mA>cm z3;B#%_0Kvjl$dSH9bvN@Y2x^A-)5eetW7!y{b-Wxc_HiMbO zv#fq$7Uufdcl|@l%3TY*z^;h>m8<=%WFVZuVx0a>vi?na&dn=N!P^2M{1OF(VV0*e zmZuJq`wryRRpi#g#a7!@pJd~@t*VFX!<9$dr`7aOH@_>lk`vPV>GfyRN)XTXPrv9Z z&* ztPk7P|IiR>?X_-(tNQU{?eXPLr`CCur%9oQ$+)NMxTk@f`vLvOE&YUP{oiy!Je?op zq5JHTm42256U^%0)#~4USh*n@S=?Rnx0h z)2oZyTWr=+6Kml_&E%E_;s);N@%YJ?F6xtX+*H5ECdA2H0?SPTd`wQk*;C=(r<-S= z=-2vBm-@#iXvZhbqj${^SL9C*0xRbN3i{*}{zj^Ave+Nz;vVK8-iePN=W`zBk(=9* z4-O8})^(u`QFs#<_3wN1)u$%7JrcPCcr6w59<=_wGkk8b4a^h{>vi3l41+1x6{}k_ zyGs^b&cTmwP8Uts#M{2laeH@hzqwjeww+%GY_2>WuJ|tX`4Fx{6!e2(e%E1s4R=RP zezzrlx6fXjEpvC?>FvBz-vWJ4(#(aJj2|eNG-h9?ViU&2#J1CLNE%CP z))s|*FE#yYKbSlgqLQ2vL0t9;U8)I9r{YIMCoS;25G6riFAXV3>P zz*bA=<$JHVQb{YtBr64fBGNG#R~$U}Npd&W*ZR2M)|$^S)iH*ZbKU*CGFi3cdv}P& zy(iBk?+VR^q)kQ;ml7>DC|7+Cb0y{$i zvA<>4Vy9vkWA`@Jt3P&&&C}GC=n#rfQrYOK{d4KTAnVpdiSN&ng|In0Uh;q&Nb5l- zK=)oh;ZJ&b4hx9cfa1A_9KJgZ(Z3FERYwECRXHd zB8GS+sXEhH!|00O*@d*ik~XGP*OMrV=#?g9xEwvmt*}a7wR~zDlhQeq#h64! zUY0)d3=E3X^qJPAsy(%yWu+5nJ0)lfIC)6QvdqyBfZErR@NFF)Y94NbFiPm?+J(d2 zGpUW6*aVf(MGDi)g^}#>NeDz#dwC+sx~z`kwkRk{nWf>o<3g%dMI5A&p;sfS&>F$@ zC><~#i>i)DGsEelZilPzzOz_#+rvl3!&vVXnCh3e1l{jZ_)pCeD)PHSvC z2|GG_5&IVAS{VM)-IAqD=vJ+h*q|8L@4vEsv!9sIg_zJ?wa|V0v%Z3_p!6 zuR06vIJ*bVf9_)5>Ol8B=^}Y_Sjl9p7qAstkmDi087FyA$$|0S()8Wh@jcqHdVy*) zV2zLOsN8gxNVb@k^{TLCl1-crSnuk2Q3f7PUze$iho(W!A_m9rf}HRoNP%D;m*Nzm zNaB$@K01br@b<;#?~8!NP{^0?!S<(jmt6=V13CLIMDj`s`GQ}Ch>ykFxY9^TdvN-5 zC8q5q$yp91>Ly@~H@00zwe1Q3CG~eh^mb*GJTdD#&IWxUPEs$9ztz5itC^PP-G3)r zV}vJO=TxAviRVuaz#5;XH0-@%{h%WZ;xjRJstP#yWDy~ zk|X&3@*#;n(}VYsp>>@Bn(~@T|BR~pw9ixG*t=!NtCeQ1sdc}J*@s{bgI6gibV`Ba z94gEmR#E2MG5XvkANhES9O6>9-GJ;Il(E2^11RO77idCGr>}TFkIG_OE#7Q&5&O`s0(9Oec%eVE{E{guYw#laYuI{F?+;3R7D`$m4d)B4()D>wiNJTouFt*O0bQzhgwjp# zop(ljAX4uXf@1Ww-!_KwpEu$+LE;)fnLGvk#R2otG7!B&|YQ$~I@%dQ9l8SM~PC z>1BxX-#sqAYB;8!5r*|l<|Y0@1)osJS~`d)@EY`y zFMB$RU-tRk_uU}MYbdghWc&~LK&bQFZFq^<9xk8#NNL}{AUPu=S$E@T+k^Os*6V&K zes=vjsuGTS{lQV$48{O`+l^1a(+ASpO`D*L56StHG`!BgbMIhkC)*R+_%2MWiVZGG zklH93ke;^atW_Yb*zs@CDcuw*-BQW}cpYG=fN44r?mGAjX^D#i(c^wMseU&FLbnu; zcXUunvpzv5@nQV1f>gYTJiyni`rLEBFivNN%i3s zvC7kRmia-CcD2Iy^vi?=+;&iKBICDhTGfw5bu%`sn-utn3GA0>A0ju{JIJ~0#H&;{ zr}J-EemIW`)E(*ZvnpmE-(<&$mRgKaFy02BuzG*N^?D%%}Qj33=Jz9V{{{ z;t)a=qUtUB)b-~3SFO7_`%qw!O!D@)`OLbB@U?ck0DV@jd`Oz0tStRA~{-dkY2 zq@p9lHs%=|At*#pZ5^)4Ki*UmqVE9ukTXh?PdyokM;B_$MjEh-KOaO_Agu706y69`z zS{_$4@f43*TeHnlO4vD_wo9Emg)xXtv3)hcEwzQ4)yYJHfGb;VfJZ55`bYp|mSTTp z{Run(9zTHX*x`3IA~PNJ>+-oB1 zhy#^PVi&n0vuV7}U+4odSIl_KH{)%U)8o~Fm1G~p+ezC&JvT$G2SW$L=cr)_-$~9{ zNhXgQp*L2oDWVlNo*nScb#7t8@1ExP6Q#U|2K2ti02QhtSZ>fy)iqh%JC$HBqn>J^ zV%f|m?o^PJT(GzBVA@-bz}LR&RM~RbOBA|E{NVGk#}Iezp}+`XHC18E8zamyBAMkn zkj1?*)r(i3cJ5>`>X&`o@{Z4PU+e<9)maSjLfScca)SaF$w@sXgVk1}x}!zuz1;S* z|8CdMQpy#CY7E}w9q_U@TY2;1wm_-m#?vW|{CER0guR!o;fdL`fIu;7! z)nQtftaQ-Wi7-z$WLU%3-+Pzqidebw2D^xSbS%|(FBLOY5He|+b7O5aEeKXYRC6X| zu@cR6g=aX4Jci7qe~elIqPR>{GnS$ZB9AZij$cGj6gk+BKz%hpAlJ?t;PHhxe`>y$ zfX#txESm!eI}qzlLry=eUnLhnc9*qS_d*ms8wja#qhX?Gr8QrcOUNo|U4nlCEbz1I z*v&`0d(FK=qZn#!94BNx?*qdWC3ppr2wH3$QC6QhBLD_;N+EPcF~CLLcC_D$hSJy8 zm=}gEMIRc*VkZb0A0=_sSjr}E3e5`3hOnB2u2+`Z9@rfif&?M+07Y>Ymr4P<;Q z+qm&P;dOx3%7wzp6m@s@%H($Tj8UCtL_f)e#^A@SvjZgL53z_@o`oGEp)udMsntEi zpIVRaT3b=h(pny{L+WG{g%>C;No;Lr591pL?F73>-H+lW6bTq&*_$mkG}ZAEt^uP+ z8QT836>-97XDR+*#EY$9)-LGF}-9u*iPUu}NZHe0M!S~NK(*nOn z*vFa@gcyc=v6zTl@Pj7`ByUo*%HHiEEdo&6x@i+b@;vbM+e`Hwh;tijw_UF*uG>eD zf>VUgJ_)9Q%zVQ!Q90yFiv!TcZSGiadZ`}0u0(1Ktv>v^6=LV?049@sI)rEI8Ouw? z<0FCkZ0XboyZq1rIVIyOIO)H)g}_3~Dt@>u?!Z{8Kme_09CFbmbl-*gBeK_@{<| zITNJ&&L{9pJ3uJ{Rj43glA~ZsBw=|rAFfJ{vl%K;(}aox*a%#I%k^a^unCb1_bTXu$YFpC5nM*o~s zU###wT*Pts_nYv3cSEW6OF^xm$a*J8-*4Ul`(e1XgP^(Nb<5=It;VJ1O-A$%u^`x9 z6ycffE%LJ{X$vW0G>eaC+flKgYlWBPxtHa+H}!nUm3bj0so^6s`_ zaJZkGSz)~ebAgfJod2b?EeG;sKdYm0*KNp{YdPv;L?9YxdqOa`vR#SDj=9f_`8@qx z&Z4n<&kt*4-;zniE((1-7K*vyDr=Sw4sITpm66EMNQ7EeL%?Pe>Kq|$)x>B9srSIT zp>O|!D2^cNU7g*h5}xbW_H2?vhwdOd77CM*AfM5A59=_qf_}o*4{_!T=_h_sw%@+4 z=in@^7R3uCNG%=zvOAW{V%L5!B>83N8bRnlf>P8gu>UYo$>S>aCrD7C?FxoZ#Bk9UM#Zy_F*caAn7UgvTC z0;75G@DkRxZpk-%R&UnWuM%d}U7_7gN$-@@`p$~`hzd;#g)1cqM6aY#ZZjSMy7)qJ z_vQFH_PC#8!3(Dc1LX}0QOETI^4E4N%f!3Q4nx_e(eR4qce}=N<#)|?{(CJb+3`+R z8x#n)<6c-=Z)Jp{3^(_>PYbGJv8CsG7mxUgRrXQKpS3_-bpr?TH|AfC>*Ohkb(R5#IFcu`^b zpDIhp6gD}o{pgpmf<*hb=Gq4DYS}ioNghJvt2&H`RR-pw6KpE{vAraCG55?L6k1P{ znoR2ug;fpH(!uy<-iorzg|L!5Z*y}?pL9p6 z16xM6cis5e9+V(m6XS7VO(xY@0LgWLo~DYa*yFIdV4_)IyQ`E5qdXT;FcD6Qr_yrhEf|VXCuBxmWhd{ zW49w!S8jp#G!zuxcZr6Nr2Te-2qO>_tDY4nobPzx?kqE2MpgIVv(!P3_3I17_*rp) zm#0ZICTx4oIk4whqZh-P=`5Y)qh{BCgJILYU`i2FxpBSc9SkQOoPFpPayZb!2H4o` zc=Co?tjCRH+oyA1LC;`TUhFKJ=A4~TqsWri|2$ej6V!aQ2^UPKDy;w1|GMjD16y6)*T6mWmiXRrGLC`G=go0Wtp`m zk84+gxwpHlh8nkS{U1vovQn%4%Lxq=v3iydb>xEHRebjK+cK6lVKvoOJ7SBCDVKVD zRT4GuRWi!cSZ3%>Ed-TwI6b0m3edpd{3{CnWQCw5JCWsWs5PQH1LBX+8va7 zeI@|8Z)aDKDf7(>3{RV~TVa1aDw`(Oz1gJW8&eHo&N)=S?${q)%8J;y7Cu z?HeWwCFB+}J<3tdQn@jgQAVWz45ug3fp}8pqP_wzC>7JbA)NJfEGj)fUb^N;GQv9T zMEPta@xm2uen(Mj-bb>4w+MtMol(AVFrTbl`d%VV$gqp>VU1dV)AT}iAKv>cZtYe8 zO*P$@s|2X88B$M2In(28tXU zQns#GG~siZ(MS~^0}LHpXsMXtHL*en6MuTzGeu-ttBqgBGcfiM}}o?Qy6bsdGCYWb8|SDjJMHtpAP&O?$;f2+YYY z>I(sEbdRaz)pkURj#j-p3nKuRs#VFoLTv&CZ58M`5Criofo1C0IUnTO-Hf@tM={RcLLST~Pr86=_nP+D>eQUaRqUd|N=YFG$JjLz#?$1kC1 z^gWlJjvS{ycA_YeflztrE%uyBD%Ds%!rOaC5Lxe2?KeWb;rf- z1UXB2T`ks+xrw}3p>j1yuN`FQ9>gt9w3VN13t^DaTeKW8%)Bi71jknmsX;g8MUACF zC+8&ks++2TVqTu;1t}8pr!rmce!$Z$!$K8= z)vJF>pplNG*Lc}8Rw92>U~#n{gUEN7bDl#+C%io9=s$@OP~iS4BFP!GaaPHlO2Zb6Wi1A6R*bPoVRjFdsgvUIFm@QE_7Nf2)e6pW$GB(B3=%K z(_-8TaqxUz$sU2Fk5GNUXL+3+Bba2RP29K)!x=LCft}-@X0+0XrEIbm1hSkE^84O$@`!U-i3U%$58-?OZnY!0 ze=7B+?lG*+AqQJIVs^wuP-L%fY!1!G=R2F&N8tsu-jc2Nxn)&c5B5gMnaCU-99V=W z;+j)ME~lKm$f9=cx)S##-i3Jo9=W@(w#7TdFW?k{pYi@5VrTaGaw@$8r)<|2|4+1C}XC@}e{@6rQz3r*5(dvkB_FA1V@d zl>3v21c>AX@t9yI^0eJwykcW0N^ykceH+V?1AOM>rQeZI6X=^h-FMqd^>u+<1#1_2 z2WDs))H@1@a{K8V0Z{f{1(As6R0Vm|wA(>6(`YPC(C-9gG*q^ zXpV}oIk0@;yGK9Z^IE_Q|4P4!f5!{oZ!{zM6?UsMwDM;Hrij>3d{xOIz-W&QoDq$_ z(OvRXgei=zR3kV3C8N$F7lxA%teX)0wql)Qeo=-uI^rY!J%>30e(~E~SUPdgTPco( z!(p=YUyo>=$96mptWH@z4g*Su2cFfuV6x0cCxZ{4-%jyS_rn(F6T-nOr0_Zlv;PRm zb+tzV$W!X{l7V;mV$189oO;2|w-=T{F~n`S`@57V{K9A^-}G_`F(tt+vv&=sS#ltN z;Ejs<{*JpYi#!7t6RGON6XMxdyqpJ@SoC7uenB}(Du2(JN?;DJGqpL}6gTHZ&TF?= z<7nAZj_y0^ha47_LwNmj^95DM|G5uo@3+7qw_&ntS&nR8npg=XqmC_V?rGtNA_TaU zd4t;z9YW`O@koIZpgUApfeBUmkXSs-t75Sdf^MqX0cvvi#0|NZ^yIsnp-aBSIWQ;j zydhRVOaR{p&Ukkn$G{Gom$qtRd_)K~S7-fFf|#UDX(v9k0j+DH1gkfU<~?d#BWkOT z0HoObQ9t#LCCC|B;tjQYobCJVdZ8}(&*@Ey`YwYwA0XQR%ri)yuKT=}6qtyd;90-r zYNDS-Vk6oj@3ckobkJp=EcYvioRnvpt()A;{K5`HR61J&^Cmhi=V_>XcGZR|mevrh zO13|JP!&76j7h+wm#zCy9u|?9!{Jbiw~+eF58RP8iTuN0i8_c7HpnWJ?&U(OcaByG>J!G6zq3W+0T$hm6jGVvD_{WH}N^FFh~Pxi-0 zT6Rr?oUtUAq+C!0cbg312!SzlwwQ7qopX^fq;Keq=~9sQ&sP93fX9cpuPAj^4) zQYz6AN*oIBPK;17?z2Bq=-4CR;exy2l-Kf+992Uqk*tWiX8C1NN<_9G<{n* zd_Ri1dYe=eJq-ht%TU6fTZG0wxbx-J+FJ)L%^O+m-5=r+pF&@Rq(-+XtCV-fd~g9=IOnUtuoeHPZ>!~z9hh5OE0g3mB2c_G9*vW!11JLkepw}d;wc<& zeR?x zDd>yjDLmD&7FB+#u<=xSymDNHR87h_|~Fi z_o@?F1Y*$t48hHJ#pn5qZ#``4z)KT@P%fn(zLpmTu^WaHI1#AIj{pLP986v|csUrZoN_H^fRnnf`;7k}w|qflpy? zdbtKX+;IjUYLbpR?a+sTpDV+*C_ei-D}kwbSEkoA?#H2$)prJ&aA6ns*Dqt0>c0j4 zsu*^X+c3nUp1J;R?|ZJ48aKP$Ej#+);OH7w@35LiCOlyQ^;onl<=ukSt>Lqe+y^dpXc(_aIuU|-gFKmCrP*P;hY&#n1SxiYY}B>4p*%m z4gUI03yl{Qu=Z&`f0n;w)y&_`OHRzwmJi`43v&BA%5&p>B{8y^N0YfT1ElK`p@&bz zbMvfQ6arOEoVrICnkzq!j2>=h+ltp<%jr7R2<#0!eZ?D*z2`X{Hh`Y*V^*`0eoZZ* z>(zZl?ph?eFe>BT6!*+ZQstdg`&qO=g5L%e<<@rhBem_` zlqTXtx!jKTY2&AvgYo$WFhMlOyWuQ6C5HZ1|M4uf^(U2|tnb0uJ;bseC9>@h9a&f6 zO@a7T&fnw}tHUVDfkdVh?A=>E(}}rJpW|tUM^uGNFM&M$lbvsAeEvzZ{?<16;_R43s(zs5hp(IHmkb%cdMX&-o62rh@N|MZ?I zI;m=~vA|aFJbiSmi~s5VoTsLXJkpqLDn7Z^=q7;yga*!Qoidh#4$HdOjiJ!AX>w2v zJrUhisxyyFOhHs;tG^wFF<*{Mnq0XVaV!79E>(9pvR1sd)dkxtrG)YEn=A4>W&YH< z2HLMeP*#F7cLEU(bXAv{l5DAUrlSGxY-R$S$dHa*jxH{su#a3h%u2~GH-k6Srw@O5 z-#ncdcj0gU}QWi*(ei zQs>doD&6hojYrOqe$3_<+o$}yR|yl$s;e7^d{k-q&OL)COPEE5DnDQh%m;s%dJU1F z(cdezyV5xLmGHZFLtM)fetEU2P}9tt(N~|o@H0V4HZbY_IEjH9W`3Z%E`5pkbN|4M z4SG3Zhd2Vz_<+&&zs1`l%af;fCE|k+-<9=p+@&XHi5ebQ`B+SloF0bb%M{!%ihir0 zmHTaO`jp6&q5Fal89HZ%6^C#M|ntp%SUsExV!Zu98#VMg^`2 zvBYBZ1PTPNlm;V7qzB7wl$1SVH=vunZbKQ=uv=hP?k?Psq1qX9SXwS7>f0V{6(Isx z6b~mVw}`CD6rq<(=+bd#^b;#)@6R*wiodyuI!)Y@)HfwR9b-zg? z^o7LH>0>_!BeU~YBGe{4@`R1N5O;r3v^qeb#{o1ofbvOJ+>0=f?+bR~dPY&0+wsjr z*xSrf@oRa(r<>8tkZwZ~^#s*BisAz;LK9s?D0z>;4IDYOfS(%Z1BZ!B_JBUyz-%4F z>rjRilxhj&d0=B*K4b6!fs(67vw|}S7EK_n>z8xRT~;1ndatZ)$j9ErjA(f5hDzD_ zo`#X{NxMdS#`2?Lz{*+hwt?Qm^^sd!6s#|;p{GM}K&=d_1f5P$d{Gwj`5k(Alg$Oh4fBv;4Beqt5So z4cD`wX-0>741I6x3>7NxaWTUzV36U7XGc-Rk%qwmsAt|GLK>L(AOR z{1PAJCE5R6g@_)h--{e+Y1^N-1(V#b!hinS0N@(ADBkr!pF0A{;{%-1UxxeILdHw~ zCXAN2VcdiYa`3y9yl>Bm;!BJk6O-IeoR^NzBC{-HZUh<6`7q|&>^Dem2H~{muqh=g z!R-9$37;a71jWy{-9yA&PZfAY*(*xQs2@zGZq zxRGhEvO$z3t<2%^)F3Ka&`fd~YNHF}CK~1|RjVDQEDQXG1kjF1PLQXR30=#RygK%T*ur4auJt5ydgth&S!~7C+#Klgos6np!bP%NlZh}a z@1W*mKXCVJl9b;<^IQHOj*2-<9aR?>P}vJXEw?|*xPI8Ux?9U2)QG=GuNZ#ixz@Qz zR@_F zjVx;p-6DJ5(EFhD?>q0P)6>2qskXpM`WYB)<%H(4uxcT`9}fET)rz+-t84=`+fR*& ztWaya_~JsLgCp}RVFvvxe1Coz5-L=IvLtznn%8Z(S~*OReRlytT21dxb4R-nlduLa zJc(u5CPnCqmQ1=b!uG3#oHI1|`Qi47PEj%w3Fp zhZ#p;v^?;~)>Rx4hsOOFyj&=31F{3+do1bf$tnFM{##?3bJACi9t=2*J!cMf*-8At zHoGwGHV=NoYG%bV|2O~n-JX5ePs;rST94IZd1lsL-QXfT@@!^(eS^Xfmc-pWo>?USYbwYA zV577I^2GNHBn~CZ35^?I3S&b@l<(YWY)v(t2-@P^87xujOeU@`7U#b-%)5+F?*gDFZ6_NG0RDJ@oa^=|x}k zLOC=v*)GYHT(-x{aM9k7U`kS0OUOt+nT!V2wx>B^^g% zvzn^ClonPERBXZFG)RH_B08JUKc@kDC^H*Qp2ZGG$Ekt8S*zS_U}Ce2dM@u@Vxm~N z6%a3joJTo^fmDd@Fr%mIlhc37ZQ)1hY~YDV;w+kn#i)VynT5>f3z}RJJmKTT1ykWW znal`14WgjMr1 zN4k`iuN&v@Wr`cqJ`?;gP5o2sUX8SWPqg_>)| zLltSYMzC@n>W1Sz)ddBMEd*t^WvtQJb~|0kAJ7?RjGILu>JHcH00_>IuaWH-%!?$H zg^re2R{vonU0KBvrhbg?-~r zohl>knYNMV0MPJB-8aSRt9rBN+!HrLlmO0<-))?xy#(06z;18OU-1gD!47p1*f*~| zAnd@(C4b$F(i6hc4ci=g%9iC zQgylKPUR;(ZYqbzOYDgP)QHg}rYu%@$pGP>f%2T29SqtdN2jg{DCsbn>yza|9; zY0)KhJ3KMG>f+cPqf;A|_To-JJg2Nma6iUGd~1^bfX!ZQ630c-DaIRgWx=iacocbN zs*y=GGv9VtwTi%aFpzz;&_N1|pu`POF-rfmcjiG|XZ>5l@aLy9UiC3vJ<>c=3leC@ zwY_h+A|N93ln|v-5ZcO-9rwy-B-Dfke@+S|?|uxY`2|)Ou>;VS>D-(c59)AbsHxjl zjukd1n^*J7GjlYrec=NIN{&(nFqTDXpmY+|`PT2O82<5lv+*26$7nqkv{K#Typ&4DcGdaRbJuYD(Okw43gGcf>sR~PU}!yM6I@w`(ophD~|4Jl-uj(2GJ zd@(@xg7Hc(;`X{utTK*Y~mmV~=6RqyDLB4bm9M9LSCfK3*yCB_Zbp~o!P z!wC$NQ5%iK6dYxeqBgi;H1q_Z?K!LRu#w4kzEb5PIMeonlqzikq?u31-K!=LAU!7+c3>B)b4Mg6K_)Q!?2D^YF1qWC9X*v#YN+rh zs$G)vUvn62F@&*MH>XQ69bjY+C*D^-oiT5y96C+W9M1TC<+1Y{JQ!}rHxE72b~rZ6 zN9gf!rY;4(Oqjp(HcL)mFhI!>3bTvDNzTEFHK@Z*XWOKwJm-3BXld;@Lzn!Bow_Za zl=dk>iED9LcBGP*qX*`GKsJ3d(} zFfV1etw*)@JmBqJI{&|E+#v1YC@8@1QQBwtu4{?hBRUwh?=N-SUybEiHwWL@{zJ&- zbIiXFI{$41x*EV<#gqW=3uANh_lEZT?r%gyFpK2#KL4)n`x}1S_XouPQUm@C{yUOQ zJ}>Z3Z$dp!!GFO0L#}vpI0HxjK(he~{{~%x+lmjjFJy@u-3OvHYnG{}uAd z17*p3eA`oJ^RRy95u&R$I(3zKJg1$D{EsgDFIGNTsi&ZAq75^yB@G(9@y4-}y^K z4;@O9C5D#ae_#c3kb_%ue}&~}+&$#=sjHM2c$t0yMW-Q?f6fi#Ik$;{a?j(1`z;o~ z)6X$m9+8``bm&RCFIjf}iAVzj9pWa`f2>`7?oskr-2YTF?mQ?qQJZVJ`xP#K^59<| z<~K7n47Art+o@JZ{O{~n*FX@K4s))kpZJFa)DV_#l=)8m4+MsFLGTlg)Go{ZLq%pj z6G+XO`6mPX2a6MovD+h@t=>7NS^~M4;K;#CpQll zFBcaG5;#maOh6zu!{>WQ;DP+OK&-56kSqH$01{aL^@H58KBwb+20{WW(jQmO=NF71 z_kTV37u*!Y%Jz)#-w{0{hUCZk4EqcIjOaNR5DzyOFXY1Ud=Ck~VGx3TQ~C$!`Tse| z@9?kZ{}+Eo^cw@@%Kb+g5C=PmosFHH9mLD}8{uDYNQ~{7hJTg)FE}JUD-z`I`N4Az zf01bh3C~C&@W0m&r_pZ;A@IKy3&de${I5FtgBB!PR3!@l06N)H{f)~T1$jPWds}8h z6MG|5W_Mc$z;oaia(6H^wl;MkH8M4~v=g8?Zfc_;wKNf+(d1MBDL9CkT3AYZI+?0@ zDykcMS{w74&)q}e+IMAkp52MVl6-;V{2$` z%Eu?+ZfT}&Z*N1&3StIh4ov9YkRF+ox=IeXZ-7`ijrIa5F|Ad&vYA#Uny>}2WSVrg$j`i#@i$o`#+01XX< z1OK4T#nSBmaN9Zms|p~M!Seisg_Rk^0>MT~%KseY6SZ`4RyK7~GqiQEF?|OAk&BwC zD$D;q1@zf%7@_%C99 z12rv8o|AHL{VnMq(7#e@i5ZDGsl7A$O}C|;`Tv7*2%3MR{I3H29sEa{zZd9#3Sf$X~o>Y6oF3CkQ9uMC;@|v!RTHqWEV~gTVg}$29lG literal 0 HcmV?d00001 diff --git a/docs/img/webui-sql-tab.png b/docs/img/webui-sql-tab.png new file mode 100644 index 0000000000000000000000000000000000000000..912ea35410a5428250dfaee40821edf8129df0b2 GIT binary patch literal 383887 zcmZs?byQn>*DVZ`;#Sa;O_1o++BkPF6a5~ zcc1q?=lr#^Ge-8vZ^>M9uC?}06(w103~~$v1O#lkPf}_K2U#oo!GXpNe6f`uRq7A@i>nSM#DtVSCijb`#$?z35J10s^jhnI>07dOuDMA+HMKajON z85kM`&8V<9zO8>0dNPt$rcJm&{IE_tg4cKJh4+MnU^tLGfPp}J_o4af5UB`XIcV+! z3c+{n;w~M4esLEWKtI1qE@J0H6Dk6CWN*AVVh$APs`R5D@#c67Vr7XGZ4yF`WQbr4 z$2vM~)OSxs!(~>65cf8^4a}_4Rws)H$8%2|dToPn`MzA!dfsG#ohRGgr=%wRZbvK; zL95p~Uz@5d<{%V8)xQ!_P@J;g&@&R7`uoME0D1*LAIJchnyn`PIKfdGFgg5 zTGsw*vGH9qrz?2q^_>spVbVX}?PqWSas`D@OznQ8o@ac23&yrkXI<~YTI?e{8+fpz zg%QZV{y6n87on)*+0g>L;{F(xD(K?jm!J944M$GSZD^R-e^+yOSy|M?DX|fbO5FCcXuH?! z_e-ar`=VzNw#!|DZepIVexOj|VQ7CqltoE-Sru5YK8<1GVo~Qd`X3h`2>XItiI_S1Kh=Ca z#RGIb{TKJ(nJfjl`hh+JCb{EIS z^_kxTL7);RgCM~Jntli2M`9EH+z1u9ISIiJf-kfGPLIKsfr zWdHh@ftubOzwzgbH~9T2n~E5;-Ep43k<`en>8)}1-tOTSgilLN{*`c0Jw{oVX~a?kzJ z*qgjLpyivPvbaR6tbfVy7Y#rA zQg2g_kNAKsAuz#ls!BC7k3G`cjCbMf+$Spy`6Yo`JmRLyHtiK?HF$cZ)q;RC)WMi;i><0+Z6+Of(x0T*B#c2&(os^7CQMUKlR*;* zI@VPO`Ihp4W_2Js*NfV7idCtWlg2KCNu~#8ELBir-~uc%v)LdFir;9_$OMyt>ED~) zquh(&xQ5b(`h*IHvP4PA()3dHevYD2DvU3P? zrDLCK@@p1q*J#J+U}`q%(O!{glBgBY#}KAYm$)7LQU1|r!(~J6a+z{N2Z|Hd6Wx;o zF%&T;u|eOF8+;#0ACf!jbKjkvEv;3^=n02E$rI%h^ii2{P8m;WPApfXjX8j%f-i?9 zhA4&l#@ARY#vG5MkbPI8!mWbVQ})!XmR=;iITX&?#V1AVOuLEQ8_kpL!b*r4iN=O& zhdYU@fKh_eNMcWvMYN2!OtVJDC9tmjhK`%r*=^IUj4XK+v;$MV%o6lp?|-YS@lnHV zD#)t9O2_r!jq600oDCE7XJuq;7jN%3ZTrK8xke|L)2&mh5~EU7x<1dXYLnhW)6FD$ z#P?M6Mgl{okT+$DO_`f9zdr#KX5_Mx^s^rr8F_%PL>-%IZ)u5N@~YV>^ku~2qftZ4ad1W^ zeoM;{6?^&PWl?8C?`4UfEpKF zz`u1!oPpN;OpHxFuPx5^S6`Ft5y^$I4>_e(+m39kpRUtvue;p3EN<(J=!u>@`oJDQ zm@zNOFc*k*Iam$ms^6N9mvv8d3x)f0II-!dpYNw9IOr-2%abI!`%|3RNv67|mLA^W zKQZVftt)gZEGB!gF4i5Y*XKOw2ir)ScZY60Z0)qT+?VY)<`Si{ENFRymYTxqy=mAY*m7=p6vA8og82G}SHDF`VVihq^p@-NuxgU=+vg?E6@ zpN5xpCl}1Y4*cl+1`Cu;^-kgQNnva!tt9^bW;R_je}o$w2JE`*f)}anl$ZVPP_t?F zJyH$}=gyo8zFN+f-54+(G#?yzq#qm32k^fo_r6wKoIEqzjb@8vkJP*7zSev|_EUSf z>>b?`H(DeCm)o_DKK2;d`WL-#6I5CL&O2<>9kg(j^1Y2h@H5#?0wq_e$fhi^vi*kH1GYo?oPs4iJBQf&y;0vc_-z|eqzT9W5fRY2sw^lI8eDCiL2bK^J zJ|M_RNoaT?9zxMm%zMAkoI*UV56oe)GF4wA-{|r4r^LiW#t8Di39PWBr?CmJwM9pd ziGE{5EQVSoiH4U{k;7JQlgWqv#Ebqin zq0vEt%=Au{u-pDr>;4R=Vdo2U--@^R>nG$a4x|l0{`!+@{`S+Fruoq!te+qV9KN9n z9yhdIfmICz(kE(`sOc3>rR6!9Yg5fKPYbV|)o~YV7RUQ&4d?Ck!cRJxmKFZGC{)P|XAQH>$7JUgYX5u|J5r>M z5p0P464-hEGbk|{th;B6%PuzWyI%?JrQSCX6ga%z(Zaz(fx5S@2zn~yL&+ln)85dI z=7C9(tU7b1!JG9kf5_>v=KcqCf4khuQ{-r`&@*Eb8@ooX$7SVUE-vEa&BY86$iwmoBBBDr^0`<@6_# z398i&YIt6KLdFl2qW$kBO9Uy>S?6l$;`i6rDz7=`Rc$7ZE2=6RuKHe3L@%|vowv7l zUu4wiU4$=M(&#lf!d6SfYlR>hHgmapC6S@$x&zTR!J)Ntks|D(;BI_gN{F=cE&XxF zM!2f&B6v~&DT>cF_fH2<-}a8pbq+a#D`S9Sf{f{3eS?6cpZID{7hfWIX2U2JXWO+* z%@!B$Y}ulBG2Qf)!NbS9xB`8?!NE_=T<^@~O>vFmzWn_-_<`2QPUZo2_9W7XLsqCSB1E@p>|n>VI{pUD&=-O}0+Y`@^te zE~gxBk|f8##TMiGkNh~qMm}F`2If&2)#y5QN1-&gF95DPQpfM=oh(7Zhnl_a^P0>- zCgwoVCQpll{?N3et?oq0tKjalB}DO|G9}R#B3TiVaAlkGej0x>2nUV**~L-EupT59 zqQ>$I7Za>Xr}&1lD^~RO{6wiAcQA?MUlRTeQX_Vt3aq4hv(X3CCk_T`_tjX~i)@#R-{gJAuPjYFf)`X!eB zeq0R_NvZ#IP}_w}j;kdKT^1$cmr}%+0EO4PB}BYz?tH29TY4hm)n~;tVg{8e7gGE}YyDM-Gr72HpypIxEz?R(AsZS+Dj zy>h12_3H9*xdT-(^Bd1U4Eh%|KZEg%%SUahuJ2C$&~T$aN`r@mTR5pyMb$u^23rpe zlQi+qMyj^u->UPfy#8*2CgfLEO~?lsU;LKQ6w9eO#xtrqAB};{jm`X&m!;HjjKu8>rJpH#46(1F=X_1R+3@V;@;aKOJ#?McB72WPo6J=fezadGa%gvr3iXDF9J=?wJ7PtPr%S@a zhka#xT#ht>!Ja+`$=47^=6l)-2m%KVrw-t65%Zr~+IAsjyg;T?ChURdR3GH!<(2;- z^*ax!b#s(B7b98gGm*TXoFY$NR4gwWr=FX#&A+0X!?hTT)VotUWK%Ee#2-X9+jkYC zB#S%4(1p@mUo*&z6+4Z#&wR6J`9Y`ZQ%CB0!1gl+y0-xT2npWq=IA<1FDB+Qj<1(A zZb+=maV9w-i?vTRkY;`4&s}-*Wi>yeMU?iT5M%k7wFjYjDbM!tiEeC4xz}2=G2H zs;~4mPO5>Q&Wv|xWANYOHhvQ=?&u1@n7W~0(Q^fg$ zh{nwJnVFnwC}m_=b;iN)_IhBq+4ao9aQpVQF(;{m!@`CLxdV)!!<8E7Xd|ktZRK)$ zXW>w}KxsR32ls=B?7QFT*}$)+&w6gxwsjqR-GFAkP42PV5|BLaY0bt39fx6vO_2Qk z++ZJy0>>P*+Rm43XOzVNE5;$Pynj0q{N8|Kb zP6y(mwp9%-8`N33nQzvhwGX-|elMjwVO7^$U+H$L0tO zA7}22H`a|^?3SQ~*o}!24sEDQCt9c!?(i9Js5e-(=NX}?-D%=mYAhTy%0#z-V27Y9TQTE8wPe%J%(mM9q|A&J~fa`Q+eyS4rTZ($e! zXF%PdGKLw_@rF)rDd1#w0Bm{f8GTsJT*`Mjj5f`34(jo`xf`qBB0G&r?j&mos{u&fG z0I{wEk(Qal_!S0{?=OB)&FE(y%x>>vvRueI*iklidiVL)F4nMwf^BXKV<(L3sQUUo3(B+$?dxOV;cY#E8|bU(yX2@j;2VIfO3sL^upT< zEl5uKvY;|~U=<@>Ls3V^m9;#uBq|Ho8@wLhjCZKRDL+j`i*Tv&_=2<}6_W$q;+# zN~o1fqnDl;SO0>&J{zo?AFv@zQbYq67S^%S2#E((c853^&LDK>4)9@qq_DcxfP=6O zrgg#mrC2CS71PTycBdL(pu68(rfm<^6#4M}dF`ks2psQor2ej+Ink?N}J#O|L#RA66e^?fX61W6yv`jl^CVJ=BtS9IY z?r$byDjV;xw66Hj=9Axj9>e3oD@0nawzUe1I7lQ{KDCO|B*XSn3Fp`(ChzJjR)*o+ zC7L+YRMt9eYgoe9E5uU{%fRim%eKR#5R$jd`a>JKW+vuOKh>PVqik0gxaN0R@;1`Utak~T;UiW+-#WKSAncj@ZNObDDfbkg ztd?x~c6Nx9^^UrAJLLwi5VbCrCLUr%dKRIPOd}csz2GY*pKwUSR2ew=SuXP?!}tz& z?aGJPs7f(AA7|8~FSWAC-MtgrW5OiB%i5w9j*CzQk7n-`gL1&K^Ny8YJL5)&-LQ|+ z-Y;G7`np=*;{z}Lt-!w&b)65ZVfKB-Vf2TEs22UHe-yT<8UHkI?2Gvp%>>HSdD0gp z^k0hu>$JX3|Cb2-6=sI(e}TmN6~y)5X7&cz4O`@FBNnyo$=Nk?7kWNZ1*b1uE{IY2 zql>TBHyZRIVCL2nD}4&=@RM`llKK@Y?2tS|$myXw{~bk1C+I@2#L}jK`T&$g;|dAW z7i-tcE*S>13==mvGHQ`0dN=d8me)Sb%p6^xztG~Krkvi7%y&0)dg^F_Uv>KqC;CRh z)NJf4tD2|ZyEs=s$vl7<#SyX~O`E1BhvQQ_1rGnM^~~}@8bM9#azf(hBXtY+weGA# zgYBcOGx!+5e^daV$@wqVd1fk=D9vgqunm;=RD)b>jtnn>@dYy?cq~E;PTgs(ylV}w z3HufQrg~wdNcD{#(R{lj`=%>v|BCg@lOHne@X_&ol^yv7_(pke=YZg{9%OV;_ zWMS~-kE&~DM_S%CL7;H>6($rseY_uhykFmb2k?hK)Ir8QHqU$NB_LWG_+>&{_58(< zV$AU2?tB1)`u!XG7^Dez|X&Uw8q9^R0ls!Y!v(Dc6j4x{gkqB7BlG0Tjer`KIKfDq}121 zw9-#D-98}rFSd~~zNUBN6Zm?Tw_dbzr8L>tswOb?Q6+(TC64`P2~lPvuslTZ{O{aP zN0!#(dc+%3EFA6wc_b>EjMUR3J7+A+{ADQa4eh+m)!(-DIK&7F4D~Di`2B% zLiqOWVf-U5$FO?OSxq2`<~@lPYWhL*aQL(;gnR$E=yusxzuU3hKB??eurcv*vx)*Fe2Wu36qSGbe<^ zY^=37)J4l)D-%@LBUFufRZmv=3*;)@k@|=NgfDv-mh$HB-KIGTw7DLyEy;M8%vBY!k{NqbUU$)O zx7SEFjxopP5?A{+Gb zB>blizMy6bTRK)c&FPG?dC5L^gC_s;K?tzrXFe-sG zDb63GD#E6%GCMt|=J(68beo?H^l)4KU162mun>cemP0*p&l(z%Z^*_R+ukQp@hfVAVE`Bpv`j04O|D&M%Y*VCDX`oYB2W*R%v`up}(jeVF zGBry44V){q{_5`l!R8#(Hs_#a)iuKA8rE%&NV^Jx6Nj}!ww7(ZI`4bfIV`k5ysqBc z{{FCNdB5oRRvnIM8A?IYn+Y;iKJpw=5IL1tk!fCgyIrpdTBhVpE*aL0_i%p->=}ry zH{)<9qb%m#skfyF#+`SFIH+CM)gqP^5fB1Ae>|VN<8);cemy#LYm9;EZz=U0352No7IoxEmBe9R)LF9!z*|v+V|a?L zsj;Fo`PzlEN6K!VXAC2*x%stfC|smio0-J%5DhT{QoHgm-iy1n5eQo3YeH5(N?TLuY$7yb6 z)q9Ou6aCRNo1Pu-T`}Qe0#jo$rc%hiQkhhVHgArk%F1ucmBd-8fTm9&J&oVqiW;-7 zpc~6DkA#0UAWr{B&nu9TG(UKBcNDdt&&K71cNsg>IkKIvrlsuHP zP8|+5gYqK3FlZ9ioP0Pi?@d1T*?Oo10%T7L`M1NvW^bG_C9umHnwx{u`sK#T@*?_^ zwtkuc*Y7epdF20AkNy*dDTv4eiB-W_c*M$+JSw5kudjF4yFcD41-P}(f42C;_74@{ z2*tDdzIfiSm#!mS`Mc)vmBwp!vq-|^_UOA@1;mWkqu+zb{B(dWPGd;F#_KL4H~Wa> zC~M5s1U>Qumxudy)!r&dt;&iX9ozAzUw$WjZ(gR*k!TWh;ihj#m>%Jq1L1xP>x*K} zGs*0&(V40_zHfj=y~u!M`P9_6XdbwRG%HoUgYYv)8M7}z!a?xf?dT;j{{uUhPzReM z5nTiA;yXhJvld9*Ay0FsZsUSg)Dc0}VNIfq21^T`zGrxS0%i$jLr?#Zmu-551I)g( zO#FO#ICN#w>%4v7E_K|v#Ps^2Lk)g-F6ao%uQZi(&h2_N;aH|wUxK;9 zg}vc-_IdqTlZZ)D-gQ^bL+Xb9K0o=7z~Cf6OJG?U1hHw=B{MPX=)4Lryln_@Y+~@7 z(SP^PU+{!JcR=73j4UJ{CA2Z=!l3!_cth3=3Ak+R`5`or$-UCDvhzXb-jUH!4mLKh zE?be{;^Ja@db*8`%^NNfV&bIaWD*jRce#OW-Qgso+2Ya5t^WRBzXmA?3-I%A<_DfG zEoJC`T}RushXB_)UdDyuP^qcxmJ8Tifa*1E{4}WC8#_TD&~*3?5Z=PEJJIyimYd)G zGL5CPy^!9^W$GXbxR}^(tiLxKP9N`OLowX|1@jr&7%um$Bht)(YObFptbArK#IOV` zGtHRHGJD)rFeovkfE~8((yDx4LZx7^QwZkL0v#P6dCUlmh}XNpHa+1Fg^hdR&55^~ z*pw`&Ne=Zc$9+e6GQd@e*6tmDp`xvHTN3uM**8lK!A$yOXm4cCyUz5sjvp(Chr|CB z9-xsQDj9uyjHzxS_EK*$@IH}d6DYn*_!}Y&(LVFBYqh+yxTIl$W@HR+?=v#jesn~x zr~tGtKFtpP_lw=3xHK(r*y2gU0Xxu#!D(LXYDzs?{PBi9TC~F{-0J`YY1!s0mpEOh z3Tz`#7Cmchy>DFlb<-P7vxBwGxs76IWF+pi{#^r5B<#4_HfRXrKK`m?B)2n?3A|Yg z6;j-ayIT#ybv`~8_gozd3_`^{-m!b{zuiGlptSPT2G`3R-pc1MIrfz{GimwaLkgK$ z1%TLwv~B`z&WiLYY*4$m^@0BPiObvd1Ghanx<|Ix#I=nss1DXgBF~dSe%*_Wfq(>p zBqkgq5}ayw{x#yPbuLJyiftR!LPWUg&I{^W&r>U2(kp>bC}_Y~&}oafG3J1IGNS~% z&9W~1rWfv%I|}iCms|YSYzVcy#&qzM)2A&MK>>4Ey8hpbngwd0wx*H&$@ERK0^?5o zUu~dlO!;^5e>eW?{}{hS2i7GYH%L4@y>Kv(yqFx(v-yF|CpJDHY9sB8-XAYlqUUta zd3WLW;12U-rCIOmFZiv8{qamRz{AS92!+QHkrPqpO|(pTQ6Q}JH!8JffQ6!-+pV`y z(9uC+>YKldG$}mAc0*6gVM{!+Dl?3+vRD$^2QcnQtWhiOLKqF zQRH&B%P>52hvXXjn#yW~;|sASGbq@5Jl!W1(zgkj?e1CZZ(02K*=ZQcVy%)fs$V{x z9%%urpEjh{7sXuS$}Zo?zL#7KN6iGucrtlwU8U=AS7Hn&y~yhew|F3Tw&0Zut+c@$I20W4fUXT;_^zW2rC!JXcuj_ zD&S=qR5H3VdOY7~>eg@*$;2cXF>0z4CdVaSaFLCo7WUa(UH_O)CJ8j(S#$}tw_fxE zZg1(gb;`R{_?eWcjf+!qLJRprf+WmrsuvEPdx5y4VzolEWwqBLzr0;H6ImcFrM|t; zE170)TbwqPi#0?0`Hn_CvRU6!$lO#35)rGOCq?4fn9|bBdLSIKGP$&@Z=eTl+Qr&m zuh=tmtM-0;fYmeYE1(vm_7jUiO(z{q(2~m{y1izDt4wK2lycY;fv^lM1#^>EX$!{aoGY`4XC0>6ECP)tD%P7TB+?L!rj=YEx z)baf~8|oZQ{eIvHFv;dn(iX2%QWAaE_?3#g-&cOEPHg!9m}@;ylRavE&rW>V(W&O~m{*7@uL_HGL~bbv;nd zo+L>>XRPO|n5mMDciOOUNuAjN=(gv4pu@4kr`G^1V`Ek}FR}uFi6k0Y0$miE!{lM! z^y&5x{`<&j6VMku!!kFVsPs@_P<@%h9S$=BV2ERCI}Hq2L(8UYp|oUid2$XRiJBWX z-uAIqW*J zP6Zd4?n)lc;<3Bp#jE~(cs$30)&=k$)!6AS?welP=SX_5WB)ht5QC=AflNNDFkZTw zYRI58dAP&A1&GAM(Hc?pDO2YWNFQjhPPuOCJqvbb$f6}_5 zGoW7`wi=!vN<#b@my2>)pWQIa%wB6{<;vH{&xlMpQ@C5%*3z-gTF5?l?q*O@k?w?B zE@>=;I1e2!Z_>gDhS!FkZZQA2XjNnD3!PhVjc@^)9JjT=K^BdwJ1Fc+`C_vN$lDTT z^|k&c8`IgmO;@%ThZ`H!(H_d;6GBg49gcncH`ev@wJ0vr33_e7nly_;i~6foP+`}_ zih9rE{sI|G0UXLk-xc|zHu=1HR;ivZiy=q-`OCrn-I^x%->G^x^`Mdaj^IjFs*F;P^u)wYS>J>@EjuktdQ{%KB?p6U*_QpmcFdviIj_n1 z(K}*PSrYp-RYa%1hTfXIxll9>>4sAvmZDgDbpkmbrVkz zZxISg#a?sva-3byoxhoQtwtAucKMn_I03my4gP@kEFV!-7Ph`1@L__rct7jQlt%uE z?2V}H#Cn}pp72w@ozTYHDK)LDls0F2J%sJVD#>Q4D!vO!E^r@Nmf)QFDoJWK3_pmT z9cZ9Ulq|4N++08tr}_>7V^Sp#B#*UmBT&6pj@dUZsU3x;OEi#G`lIl0D>IbykKosD zb?l7IZ{>fbsk(|LbUU{Bs~*l{2e+K^vo)fUem3|Z3?kt1Cbt+v{>XaF!A1E7eRtP^ zE`t6YIa;JprmmrkOyvh^BgHRRtGihL<;9K{uTH06T^uxH`G|sN6+NkcHu|H;c)p)Y z7+EZRLB8m?DKwt*$z#h1QP2@#H!|L1SS4pK<}r9@K2B3zh3EJaFWEG(ACq^3(Ne2? z!cRJRxER1pL4*~LbcFXqU(H~dH;70}DR=N=Cv)--=K+I1$M{GclP6#P9A%)0k|TvW z_N_`0AzAvM{gA9Gz5iUetuO+mz?0<9iB5S+EV(-CrFm*Y6+3Om^&2a2$?UN;V=u7g zMdA9MHIO|3RanlS+USk85h@!0C74mA@sr2|wIZclONK)qxgJ@OTBpS@d%O8+M(@_! zj=VO`Rizr*Oz6yfG48yC|3}^Z&TbmKBCyoZ)Zra>RNsY1h)+)djM6DoIWHw@+BKnr zo4haqpqpVhz59b&J-@JG&nI(nb^Or`m^(cWc@FXf-0K-(io}tbifZX9u9dQ3HZTt5 z5oCJ?B63i25N)7e!d}|+tGlgx+$ylRy#ke|(Wl;A$d_u>R<@kmmslsZ#aF+CR}gbS zfrTa1!Ok{gaY0o4(7iZO)#&pGD2jNLJMJr@uN8{L*IwW3-+=Plp>F z<-msp&P)2-|4cDQHs!|~{a906w`P{oFazjkCk|4Z2nbh4{qJ{DD;f;+sIW$Zqv`1{uw4^O>s=LT0*Nx}A zQ`HepjFn=Q8aGj%3$Wl(_(ia66%YIEHG97awS+KQxMEKSU2 zX*SmIf!_FqM7{-8-{iB?G|-(P$UuWf7mv`1Wi8=Li*DJb5q4mm>?Jm7j}`i|ags`> zP)dT7+9!OpD6yYygdgKb84)Sd#$&lQyT>Wc@?4UN#?wsYG^E|h(fShq$E?tv5s7-z zBAvRIdTHI$5%{5ZBb^~@{OmC5=XcMv7=`^|-W#E$?&9m_zh0f$5wUaLcz2T$eM?I7 zL!&X>m`%}dL%^@IjNgSqZ+rPPL;wLMlm;&~HRq9*=8eaAggHF%XGgi$UyO(lk;Joe zAuq#^JuYTXTYaWveF>5K-o8f80+&K*Rnl}_8|AdVCPw-@_#Wti8I8psH^i6sae2rE z5Y`INbGp9lhGSlO&;3EvKd)}td4IBY;JX@#>Z#@?a&N_l zo*MIAZL`DUSC#%lk>zMsckBIeqwkmi?B-nK&b#c;Nu#PgcryrBjMO7Joq24g!0T&0 zC6+t4yc>ZqP-om;qyD&lhuVH&><>#tV8n}z#AD2suvI0QE*vv=88ondw|w;Ii$Nm2 zK*^u`F&eM(r6RjJTzQDxveHYa1)IhU0aW#N=D?{@#%f=p1TCugy}ZItFHO!>WGs?GJzwZOr$5^?mj^O~ND*{?jn znjCSMa|9W|cIcOaPydm@Jl_J=x=GEucxK2%-?KVlneN(v0wa2+ovR*8TJK6wwB;W< zdd{$TDbTHnF=vZ?X13PQX^5iFRxUADty*2Ud0qc*``&cUCINO8`6QA%A&3jyImRn_>=NtbQq*D0;gYZ;yN+>_P;Z!G7F#V*`<$fh7Ck_b zJdxbs)spfuB3y6NulguxUy%^wc&Ieq06^cy( zYN-;Sjb*gL2_jpM8(qyBihU|vdHI3zY76>9t`hiVQO@dHcKhu ze(3E+0iZ`PqMD9sWeign>4pc2pP~vH5z4M6``;#ipohS!_S`tlUtwT)Y4jf$nai{ol% zkeg?a257`oKW*p`PeJI=&^xJl>o;US zn_`Guk>D>v`F+1JXl7--J#Uy2FI3a)lD7<*wzeg7=NY6X9vHDT{VEvI+41&U)^2Pr zlLpc1R2uj;EhsIW<(@gn(03(v`!NsnLbl;~o>gKQ7X4zB!+!d?p&Xqh+UWK~A1$_~ z<6#04Yu&VIzHre5@m7A;pqOTc`#G4|L64VV`h%0@(~GQXP)=|?pjpSRX8W3kn!3ez z*>EkqBrkH=Q?M`!mj{upq#=g4Pmipr^4LlE}(x#Z!>w>iS8N-vv@xHx&Emu75%>C6s9i4}x zbe;M=VzAfKD12!&)vX({Jw_pW#^_?7DB!Mdwz^T&`UjFX`xqx8lvWO%|X; z0YaO3@5Up7{T=kDFDit0h%I_P>?^Q-9B&Bu6!9xTz-n+S^4*pwOep}#j zUpwvYH%PmO4x^2)){{cNVl^4d)_r%!LNgusNjD&N+7#YsO_o~BI&D&4-`;vF@YJJh zke6*NgBv}!fll|15zB-wwa}r>d2*=b_+`;2tn+m%ovXAfc>|AxGC7k=Y#qF(>!-&R zqq}^g^W#wl4HF8fw+|QokE8zUOn$$pcPB2UBzc4rBmA`Ne!Dp<)+o42Kkr)CQLS8g zWnX*EK-o8zBOVo%c~`W2r?sSCW#89l+k|2dBY*t4j>jGH;eRY^IP$zYC5#e~#<&tB z7H#zM^1mzaUkK}=7h$wG^Vt!6fUi8kv+s(st8BtIn5nm7U3u*MqRdlOd%toQGiri& zQm(zi-@Rfrlo-RS{<8zxKabF5`lHR&>MVys1tON-cxwP1)PSMU(YG9H8rbh#txK6p zrUPU?&qos^H?Ru>?rI^=XdzD=KWB6gHoA`(y}^3;)L;k`4WfRDIA4>5xJzqV1y6!B zG1$7_3mVRf$;6H3>~Aw6PAcQ2YMG_YSz;7%5 z$loTW`i}8dO7*{A^AZ`AuBhy0Qr7q}w3igKd0Fytww6-U=0P?!ZkwZ7eJa(gX%EK& zXn(DB;+q<_PS_^Cb-2VDl!nXU%Sq2AZ!=Jt{qP#?iqNVmDMgRQjpBfDrFkHcRKI) z@IQ?W&GK}9-Y1((_9(f!g{^mVQ1-e)i46|6#y68cZvZSCQRfUjE@p!652gn??%vri zY<~piHkWg^PNk%mu(ESd3O2bO49cXFG;}UoU6|%^Q+mwsD~UIa6>A%3Q7cMmwAlg$5R~r3{m|PZ zlQBksJj}I*S-GRyHpP9wdHrY$BvUt%~sV`OeKzkDj+rZXqAYY*S&o|#^e*U4eZWwKMpkCh& zV+r1KfMDpt@-7?X(@|W9I>Wm(KYkgee}DeZb$L-TY-s8AGM6@)H-<3pe++fohNarfge~Hz2eP1HFsa9J8n`O^ z3tkH$ySJ8TXREa>j5N{CXZFCT5|a3irnX6jpfiWJP4cf2D2$aT9GeFsZ@=tjmnrF( ze=kZ~?;v%l)qbCda@YZ5wp)}a4983S;#Aa*ZXpz?q0YTe)U4rKAIhl^NCharH&?9P z2+qasy{ryJJS3DA_@Q9`?pkANt0DdDBO&(}_dg-Sx$@2WRm3Jac(QDc`o(BmeBHQY z^LguNL#u=5AF+VPUPjKx%K<1Yhz-q6exHpGVGF{rE8P(^tx{!s1JW`p=x>71DaM#8Z)7kNR zSlEl{In2Hh$0v93laIAKk-USvwZ>;eA%0np5orBQ!A;EydO{3nS{|+k*c4TA12jX@ z3zV~g&PVM9nk!Gc8`0uU#r)0>Q%md4Yoh$$cy&MfX3G??UIT+7hmX_?vMTJnrxwdr zd@UQNDc|4bys~WJo3-KKBi7Qz90c83m;)zsK^I31N{7?|p?MM>W_7xI?7S>HO3~Z8 zk-m^Q4AJ#9J}J`Yb=m&u@V7qAIPS!~$1AV7{$fwhlK?JZG9N4zu6Wp#5A{_Hl=dC} z0_|JE0lKIT(vN1^jKp%Ic{yam>vZ_l1ktV8x2f5xm^a8))Rai4nLs+vI^=+_IfUw! ze^m+^+)BCX=U0J^;QOC`#o(uP^I#0JreQlm>kiv$-};VT`zT++cIW)|%*8fjI0MJO zZk|nRRHsE+sdV@iJamdbbTSI5bwc%+Go%#_2l#D1d;IDllHK^2N4a~N2T7?&e81My zo-a7iN$s*7eP0({9joNvn&Og@o8Il%o)XvM1UvjPoWrBya@>&j!sR4=`C(f?vQ$c$unHFj=B??sbx1 zFrAAeX@oFN(e9KX&zYuCY1&gJba$sI9Ce9Zt5*+i1-eX&8RgK({WU+%J4zH50Wc69 z@b>xEBA;$(o;GUvga>EHDzi_~vsJbez4H$4qdb~AZzujt+7syMnwe6faGu1A?ZrR1 zP9!Xz#ASK|2@dOL?m~*3D~D48+R#Oy_o`G6yVh!XbYbj#^8stY06|oE1FyzjAol{5 zJi3YLEMVdy#$U`~B3qWLkqOiB<&Vb7Tl;2$pA>CCJD(iAYU>vTT)=QU0k$aObu7kh z%kxT#5R#c$nxj3K$jfGRJ*66AakU>7;bj3u>w@iAa3`|!-I$;4Z&f;C-!~`-G0QPm z8)L-2689c;{n?JCw2Kuy`7rUjk9q(zaGm>`34Etm|GxZt(5%o`VH8;)~ea{#{ESSU|+J71vJ7pW-pRpsY%1 znb8fEaGAPGXMMGI#GR{rL|l^{9|DSRS|4YhieKl1pqIB<`fXdHm-sMLe`Mbv^LA@+WU6+Po6lb$bRp$Y~_X=iKpO zJpKkDGQPVM^fys$_<>~Yt>e%61|KgSNuOAX*>tJ$lUXgP63ih!AYIpKm+-d$xZ+om zuL%yn6+73!v7w`9zxQ|S>uuY0cJDNcey=H(oG0U?JZv9n_j`+L8=Pw%7qWX^eAWBkX^pDL!j8SGNnU||;g z*7s=F_q&Uv+a59K%hcL|_1}TrqqHIqPk6ri1oh?fI0pm&MtAG~HBmipmUKMnHt6De!wE67+IP|9j*Nv*3WV zzBpsWq`N=Iuij^`j1g|SKfWe#bHf&(if>l7%UcIzfXMmO;9BmSb2z*$KyXQTe`7ve zQydFf6&hO5VF-~i7gWfe8ewT+6ShMMRhgw!vn$^}3^ctO7-pFd&(7?ru~!#aU?}Qc zXj;fB%b>}jy6M|N>?NCajiT4O9^qcLzifVZaEGx*2S41SOHC4RGYY%9(w+LpVO)7##7Sh;)-?MUCbZ1wyt9C)3q9%I|hgSR%J zs~NSNE>%Sz6l>D16WcJ zVMycdAC-maFccz$oLo`^P1dOYM0^#fXmIn}swrzTd9@hR>r8pgx=P6rWv#0 zI$Qd3DD;dah}9B5R5;l0{ticL;5c*ieTKJuY*jEV{gYzRklJL*G~`7p6ix!k!8#Q3 zWdGQE3*m-mihB0^!GD!n-g~=ZU)8782ja{^50V&>pCchZ)M{oSqK%? z!fy#}0~q=);j9(n=*n~uZ8g;f=%{t|r%o}Bo>_bb5*FYtl;ot6Bg<8_HO& zv?}Kc;I&zQK?Qvw6Ccyoi4LVw;Qzv0%u@YJym?cW`tdZkZjZ~OBG(T-!(&5ymW=+o zAnwn@?HvE!eW}HREF*nI-fd$NxgtrCZJwb(`gtQ$jGQx#x)%%$H%THjmKHQMdYLkf zQ<9AJmKvV;5<;yf!gAhIz0+0Dayt~SCW0;QG}*8POIY9(+glu2ku^0NXS61QF46NC zA~=NQ=CO1adb)s9F4Em#%8h-De-Z8i&QBTW9u3QY4NVa=98oYSv2j8fIKNKO>}PMn z#|`%0ul?+L1)b*mJlb5(M{nZMxfUh_YBz*s}{^&_eiaa@yyuj3$2MUA;W1;;~BJ^CFQJ zjcp~{NoovxtHUbj-a2yS`M3ZL^y8>YiezWLokXiXp5y2QgGV2|&Dq}`8NXJUjg+|D zwbZt;9-Vkd=rRJl|B$XkU$iNnR)a@_32Kn8)@ntA=4tagK7W#W9r17N@(1e?X20R! z{BCbUjT=S2+$=41v~usCnZMG6(JS8d@N8Z=XeGWM`Jr& z=WpcIm3Gv2eM{538bfDK*9M8Z(nQZQ=bGoZU4j|3CTi@H8gqj{3j|0OPTfQYPE{Dw zW1^}|qhZQ1!2U@@)iA#nlV?u2M(d@ZWn%y|i92VEPEUkTy38P~1J3#5&;~5AUn&w_ z>Czw|2*!#T=C3&h&_&rosi8}l4-h5_Mh>V=`EavHZs*el_mp{RLxCl#dQ>LiZBa}T z-$Lb&k*G}zQ>hC0ObLqgD=kh;%&B%GguQj>vck|TGXmLp5AopoBp@YPC@Iil5C*Bf zgl_%XJyl302)EJFKNcLY8Vi`gYSHMUw3s^j$w(RpaWwEoI5HM;67F>`(fwpp*7_Mv zu{n{!{k*4ayP(Y0CI?WF_jb${DMP~xrZm$D1VtEvUC2YH7|ki_@s=!7zez?2h^o$u zgjN#h5|8jNHzCuygspc_O>p@^M4tGox=2f=H=q$)ggM+I0B^hcijIxb!=bH#CK}^P zR;b>x5JNT~Y9=RbJxe6HVHIT?;Wf!*D1yl%N+%o1N(Q7q2LQWb$#Ze}v4u#UFR{de zjEq8jG+A;y9$z?3hZWM?kDw@0j)4Z`085+9DuxG$v$~#uT0iuiZ}{IXfMm1dUe1-TINJ$5!>D4yK?|NRXM( zqZw3D@9ghKN?z@e?9g%QD5^03zR#zRt?m6A(_w#(HpRRJvfh#_ma9#RBN^&Vl?S+j*M@j1=agWTr%wF6fAS=6g{}lE7Js zKdFp|@O!{K1;o@_4zxrn6o*LG-fr_N&I1Fr+)&YQ zGHIz4wk(L8D8oWhrwA)+qL|vGXlewP09Gh@n4BhFIbF_Me(stQk-{a7Z7xii!dKMW zMw<>53NF5OCT2;R|5oa!Qq?HIQ z^r?en-B~SW?I4!)h(D@mutxJO9IgW{!w7z(rU*7_(aWrmDhw-8Fg>!p zo?9dB_)G1nqvZ$u81<*4XCIBkH7my32EIJD_idW|Y}&^QI(V>CC~g#)hSaO+pi@6@ z(HkRpo6a`+Bh8kxUa|Z9o%RKf1U?j*pK)xvHK?+*6%3#}b{8|i97FEChZ5yE88q<) zeqGolol3_h-cY$%lk5Iq2v5E}Ex{ ziG@zvPcFCsKU#Esy!>_bF|71H{M3CUxE|x8)S~oxuk?fz*ZCbqL-vWz>hG5faT#=` za|o!V|2l#~zcWPPFzKk%j(IDNtd)&iBVks&tsiZ|t>-cDJ3U;Dt?sR`)n~l);zmPx zBa3KboGA*#kh-%TBK(gd^d%uHE4x2mgg%ePC3cT~ZXHFLoE_;eVCMM0{BG~nfMr!l zRH^cnTI#!ra}u8i``h7(Cj0)Mo+KEQ8*rWgYW#8&Q4haI2Z@m%u!U<}j&<~HtTv^1 z>dCD9b`Aj0cOkS!S{Hmi^nGNY3{k~dznXI$Y&|7!dYVoP;_a)3Fi_Z1)$5K|hk)k3 z{=!0?86c>(#ofJ&>_3W278x}WnaB-y&y2dB8U}`xI!{3NDi$TYEFrI*Y%6*-OVBpHnu-5wz=r=FY#*ZIuBm#-AHbepp87r zH!#b#^8{${zpjCy3O0M<&tj{ld@3UQP2azNw?>V<3$0QiGkBorM8snnwyBK+&f5K_ zepLNto>_taToZb;LXKrF_N9bMbN`xoq8M+$RMAl03cnPL%=jx-6~}qRgKk-lew|ti z^~x*I)|Ew|96(?#78s>0e0_h*aE{u%a7v9+#J2>(cOV`8we`SU`EPU8bJHCcqOTh< zmWu=@)$aAHr)O>emVh93u>Rw;UKW~7H{|q_6tqBkabbujhD~Amgp1Ebr&#_V>|Bs9 z2f7$fqSRH8^hDb=dLJ=b4)}BZ)u=KXLXSjKso$8l&pdzdr>4+CUBokS41a=7EBWL4 zKirhZ`YAP^CS!ExIH>RTTi;$;3ZjDis*C4)lF+Txy-Ddj<+!LetKzF5^V>eNM@M7! zNPnFwUs^(kR07*~)-9<%;F#hH0IQ=Co>iFsUolQ$&W8bsakVW&Q6tcnYT-u}4CV^B zIvG-CztFQO>i7LXM%${WpCLp%uThNX=1|ZFKp&KUGbSx?@o%8Q2$DtY6Z#<0@s6oo z*}MNg^^k!DapU&b&1E8-Z5d-09%$oeY^nXdcfXOOQu%f`K_`?^6B7{kq&g_OmNoLe zZQlP+q$*QjSF7h#BmdDR-;zNhK{!OzfWp+}`Qxc?J2Z6G8e)`g6KS|g#AEPTrAVUP zn^+Ocs?!{_r^-s+0>#QnE#qxN?r_M&Q4t^Pf3fO6vj}uBk0%xS$oRZ9iWl`9w(#)d z>VDh2YV&BWh*~oyGy5N%g(w|QDhu8^nQ(++b0%b4`dF?Y-?ZI+#3(6cyYA0 zI*tkSDHc99xv_KYhy_PPOQ3|_KxY15%FFIQ%FJdh&@;cMb_UzsS~XjH0~>5gUKqmI zPT8p&CM{&tJ8R;*fy|YyR3-DJ-T%itYQ(e%t9*u=WMZvVYV9owvU#|ug+qLxULPNy z<#PIa|Lq$DBC=nyPjQBCD7d2JH8Q~&dIrrTFxLNEfdy* zBm?IHu_<@&j+_L{@U%gGbN$Pm)MOFqxJA!vrwt2$*Cx-!FCEtftP~T9*jN;VpY5Tz zk>O^dGu2n|*x7}vk$bcyyM?Y^t3dw~#UDpr{9-1I-K7XrlpTT_{mv|% zHppvR&vaq(6{gx7U6^t#5{~SM>XXn@POqCU_s!dQSnh@SFu8gDt;TKKc3ZZEfFYRE z@kH8Ku1mb1-8;`aU)ss-#2&l<@@R+rd4C~@vHJ+ylN;x-&37~If@yYVph2xNzQg83 z9e2z2?{9v+O|^2xfa?#`-V4typS^*IRc8HvJr6PlU>XEpAMrQuVI~oGzto7G zG)I4KZ`W$st$Z|GAK>I4)c!5iBXpKqWeDLTt*>h2tr}ep#=~wT?qRmrJ30VKBvcjq zp{GsHlWGr~XXXsw`LDJ-|mK(PkXeJWc@RunK>&WsoL190Ws*5)zv_`ZepTYry3kG|GSQ zjH4+*^4|ERinYhH!S;iJp~qc~S=PHs*nH~rb7lT3Jrm*Me8P!fC4JV!SDk8SF~(X! z97nYNf(Ow4d6)FV1ja19)hD>KqdRB#%LvR@vT8vIIoMlJN6%?UM5alyzmr}mz@g<~ zNB=|mGrP5)gFVW2P{e=(+$pidD^+eV13CyE)_cZ7mgabaGl<t@Ne^js=9tqU{;cMWhJK5e~6r7mH=#L!sr5h|g=%jh{&pg(~S=MVm~? zx#=(KtG!&&l#HB6OOfliwC4b`g)*PuKqms4C@OAtQ6p9Boqhuc71;!zuvWmC>)U6P z&<(BE=06O$aSE19W$gD&B3RjJlvH4?7EmL6Ms-D^I29Y!TRZa&-2)z>J-M@*hEg8e zcm}m~q1^wp+~s;_ss94~j5*n_{We@t9M0NEZZ8mBs$hdbka0nhsO7!`fbIMVdKxpH zjllX~@F_7RSb0hRH@HxDg79i_Vxl4mp5U%h(cJp^BK648S@K zl7Id0_VR%&3SdjNY*eoon5XJVRYA8K2~@`@%V+H9v)>Gv{u0AJqhw5CyN!c(xqmKk z>nOd}M(L?a!dSzhMfG?ufCj<*LT6c4J^jW!y^M_EM$iZGzL~Vg%Hiu&yng9Uf$v{y z#;0W!>KZ{cK4V1_4~ICB88P28hj=H?F9H3T*_{|@WQ1z6;Zry($Qb6@PixO_p8;Pk zpvVsXOo@UwC}w)))By8|kO&U1GS=TPo%{z!&f$_n3kp)e)sz2Rx9C-LEJ8G+?m#{1 z23(UdgNo!+YHT=GJ!?(g^#cbH87*>Y9>8v_0v@>GPrwguH!?OeRWLcF#S`3~#?honC_fzc^_BrAWV+ z0LfY2Nici}d}pPuz)Zk3Y{2z@sx9eV5b&WM;$kPR&o3(&b$+ozktcyv=qVdH6 zQDHCT%(M*a#mm5FZKi*JY8bzo~oomNQ^PP3wMEyZ;@HoIz=pUiZUB72eGC6azR;5MNbgnGVuP=2WtPcM1LLmnqSVN zs7%xIH7SPG(Z)}qsok_4A#4CHd&BTf=rckG7)6uARP+*ti1I?9JJY|9GO^?0Q5xr(>Ri{C^cF z%bJ_m(k+ZZu{?ECL>iDHti)GI#NvN#jt&h^SE4V!1&(R)_C9->?4b~Xj1LI*CJ$0=?gT2oTV z#7m6{Ye1u5hmt9so1UNEdFcjzEXRHfo0GcHMW9E^PSG#odq(FI{!yNN8tenp3**Xv zqI!u|?O8uJz7)uUHh*J)5b`poJ*W2HX`-q<1E+6q)Guw}oFM9EB%uaUOyN8RtKEBQyIfR^9 zE4tOH4(bH8>+iQ4w?V&eKh}v*lpMj`O&&}6V{pCMDCA9L{>ud!m!bb~HCpiBFWi-2 z7=&6_9_zRGhmHgs<>)Nw8s>g}{}u3bbxk>e7D`_%$Ch`${phfbQyMDTqV9bXg|AeR zMTh6?54^-<2UJqTav@RZOD^2XBJ*)v-D+bm&+f~^H9Pkn==F{u8oA%NA(|UAgN;$Y zj7xlnO9ro;q<}z64yHlFL^^(W&MttK7xeI1M#9P!N~egq_G)I zNOpj>jI)@DQAdGYT9-XhMZAK-(qsrm##H|98-&GJb0k79)u8FhzasWRTAXyURMFVL z%9u`AK*qDF2tzzYED6`+t`xC*8rcshA&7@+4L*2SQete8^N#|A8f=cclA5S2~hX)+<3b=Z2ibhbLU zh%m;GXvptjwbhBR(d+g-DDJ^`pmVC|BqW^cr-vLZAP&n#H%H2AmWD0`hc*_$qRBv_U?0BNjgM zU>01E^$<&9X%RkSXox?}rC9ku6-VZF50MI>6EokdlK2NihuEtP zmst>ZCj?p&Vse5ro0Ma7>-tyz{-58iGjA^1L}%ZWt$(%nRMKT0+YJFwvhCecjp}A4 z4!He3Z>bp6+q%WMBYkp-lt_a_vrNQ=%SNz;ZlMx# zrN06-sn7&x=w^1Wm0{A9Fa(oKea`Ech9z38w|&XU|C?ys8IkB$)4L{0IBFG)Klb{) zuMP2k3<@72h3Pv3;n0~fw7he4sbsk&+WKX)WHJ;Jw!#5(796bPB60i^+JX(ZXVryJ zZ|Zr&K(Abk+a!Q$+QA|}+9Mg1_#ZV47kW{QUgO<&!R;e>*3Fw}bjLL> z3^%0>EgMU_f~mFfE1Y*?e1nl;Lc*7*&fkZ^ja@E+(MoQKF$DD>K!2Z4mQv*yp8I^CbA3wXi-K0@PhA|;Tu@&KA+=aw;j|8I(hjDl844W7U;}Pd z3@g19{}o>uBT-v|;P=_6Mg_*zI_ z9Ghg#-cCFlNxo{1Wh>7v4i*$7`U}N`BsKy?nMA`btSW58qi3xtLrY|9s_*mTDUbix z`ib}Sa{5zkMQRGdW$zP-1dDq7>$B6_@4v8i)gcdvArE&wad$lrh@h2nNxid>+J~?3 zx+jlo>v#LNxn%^)%iM-_PRslVDE z&&iE*HcVOy8Oz#6=`M*f0K=dYAy9dbn`>+kqkpaGp!pB6I|MJMOD)S`eS%^N#P#dT zWZ<^PO&q!X;{gN*(rCr8d-n|kjD}gq1T{4PhA(!6IkO>7Xee9RYCpBu(R|zF<+9tn zS$1HUH=QQNSR^EvHg+lAchp}tJq2L$z$09dn?LA`S`5-%q6qy$fr#Cq{yhwr(4GMslVWea;{Z|S6E2_~$B zsU+dM%5ou03eRwUeZojmsS(9S1h#5^%{`d3ibO*GIo9)f!& zC-0cvQ08wKD{HOJfoMCddeA+C9+W^iYvqP#otwUA+|NPY_StuEM*L=i(1Q3uqtCMd z0}m9Yaygli&q0=VQNOBTaGu`yU9JV~0k?ngB%(00?|Yf3fQINMiCq=tCFflf*55Yn+A6RQ<1zCC^hVV;QWH^F;zy~T)dj6rZ*@~p-OO) zorwMOP<`|lY55X_P zP~&)=d(RZQ3+L_*(glqnLPPP}C{xVv)q+~1&H7d%?)! z$-*SH^Bp-0f0`u5G}o-@L!U zKzISpD{xLk^U8CP;X)Uy|C~joU@&`Y%;pFk_V#E z7^bi}lN!Urh6mXaK8cY!_x~wNJ;#8Ow>Oxr56$CnDJ&*C$4_O{TMVt z%ck9l3X@BUrO%aDO#w8|2ANX-;Sh11{=8tw06(O?ft#%(bl4 z(ZQMF>8WVeSb$-<-^-WJn+nlRMuCf;!kx|!KguxCMfqDHiNH0Rb%JVM7d1b(rgTgW z?<#m)Kuiy+Q)lSrL zU8bDT^a_9P!9f2NECrwK07s!w015Oc_PN>j<|!8NBX8itPF9tHmKQ4QrXZST8oO=sJyYx&OMqi;~Vg$!MoHdlBGB-9M^7yO9 zO1vf}abY)B%~4usFOfSJUhT&ap4BGmYYK+NFoz#v%Ke@vPWZkyH-=g$k0WG@`hcK4 za7XSjIK=-d7AcZrX9-^-vbbClhfj`ig??J>}n;4%g zr{bK(+NEVU?PFl#$aAHgU zk&V1oG|~NlcGl(Rq%6P;m?^}tgPDCsBa#1&41&>j)i)6clR*5cz;tm{^g^cdwF75BgHo``i(KEmTqj*LxhLD$-W7OL zYiRx8GZa{*u>3_!aKd|K7WZLNE(s7V^3^9i+|qP?&|aIl5PxV$g>VOa{@7qsYOt;edPAtdG>ys_2tR9 zW#Ruyo!Vo_N+78}#Gm6jn8fZgOCw+z6PsgBfE0E&@wgYm{yeS!uoa?o$F0a>sQ!}u z>i0W{mjQ?5r*s+1$BRL#Em%fbjLG^wH8C@!j1@%|3W~jlcD<7HP{*D!wQX%r8DQ)+ zM0owI5DSKrqI9dWqq|ygGA_Fj7?+s2OqzuY8Wpd}EHB9c?+Kp(aFKlY?`HD;&&|bq zhQ<74oLx}aaWJ1>D2^DdRae+V3Oaif3_dy^7mdu#m$~!F2sO;cHq9(}8|ux*=p~;b zU!)yFKV=X7VA09&=KL+|#*~}^&hWd-6fMz18bT=9kKQjLO_&t!$3_&|A#~IzlN0GA zG75}*r~EsFdE^OemKb5Cs2qhQc9k-5U^s@@d8&k=$Y(0m zF%k*vGn+2V9n6}H4gpe8{d07EA|SSXp7XpQ&8}k~H{;ta10pi%$jM(*zX7soi7!VQ z*rm4@uJqOFv-cC(!7;HOv|cYUPEoHV^tIrPWIf`m*; zidP7*lgAjTStG~Dh|`^SJOfV86z>Q?ope7=L!&|Ys)43{gB+gB+QZ|z&gCWcr(XgI z$jp0Hs~@?m{RSj?PYJkH5?w5F$XE{pDy)PPAXimCGo|936t#K)x%2wkTAn ziXASB=!j)ph?HflE{<|%>ii-qF3OgAd0fIsT8=0M;es5xFoVIo6O3=SpX0Wz>D0Qk z-RB9cvI1Os-Rr%%IL3^uWVdo;S2QWy*`VE zRtwd-^ud!ta=H9J2cCo4H`8WFQl!?qU{UcX{BfeGM?Yq0T9@3S@wwsU1lcn-j;TJkZ$6y)FQoTa)-EybpA_GBD;T+`hpEU#t;3cK`W5euq3#s zzyZB(&d!*h#>RJN5V3-XH)X!7iiTP3?~kuuW3ras-y&RQ-te~>f>W#6BCx@pVPRo4 zA8^Pulbhi9+?rw<+taGu?3lhG6DH0Y8bf(%)#25?oRo|~i%>^9r+(9pE6}mqzNnEm zpqfxcv|LFFI}X-TblCp4C$_%)rZ))JDKoJSFAMJRGcd3Ob8#;dF1|Ls7Lyvt01>q# zCOuk$FvifMh1X+!+%m>R6b=LYP8C+TmXvhxO%zAt@n<~T{JnM-L!B6vv9HpsO%Q?c zn7w=h@nNw{vdggfg_X0ljn+^2b(78kchAW11;2DZTX0sCP^-3|?f$_mt8uXzEIr+S zS)!BN9RiMWJuUS(p@*d2nl|EDa(TA8Tf}Z$HTzu*2%lh0=k>wKN~EU4cT)+aaKb!p zz88wP#%w-${#^B3yZy&4|1F?zP98x4e(>*Nm+6(fS-Qo*`}|{sy>pE_K*tnw`KADJ zWfpWVNt{R0xn~3CHDiDdbQ$pg!3jz!d3YUPBcz}PU4*!ud^Ul;`IZO-)|FlyE|O`o zNP9Kq$^O>V|9QUUdj+jrn#QEh74my{kh#(Yr<(Y)uIcqh{iaB5j@`hWYa(xfWvl6Qv*qq2x(fcuM z+;BY^@7hFa}Ho~M{~0k_Ec*Z<15u2Yq|Zz{9l z_shzK4#LYn`cQMV0lM1x)1yOK>U8RbDfg+8nycRU?tU~PEvY(QcZ*p9l7pe>lq z!@)Q6eEx`pioSB=PJT9ip7*t3f4`kn)L6Il(M?*D$AZ%HMjrt5_0t6p3_JByfVa{~@fPM2JBt*x5msHgrMKN)tzS8;y2Qoddlo zkEZ@%=Zw}b`0Ag8YI*_IPxutL)*3cdouH+tA%oNcXe$b^Yin4dsv$2TfaNCYKv4QI z2WCVGukJbO*rkYs6~p}IZAd1{|vs9v`1kUMb$3qa@&JU-C$cwRCeP(P+=Erq|KwYY$O14L9G z6V|a@&kpbEP*`Ra8>WY%%8(Tjn)aa3rm7mc~S7DQId8`C)b z_dg;WIj)gf?dz0voEc`pO+#3nS-rrhWuxcv1hc8l@e>`a34#lw&;-zfhBTtL)hFNZ zw^#sdJ3qp7$2qnB>l(X3ahs)Agc7zu7LSu|qQNzK7%)DKV0F28N-5%%H(ZZ)T=zk) zhW+oHX#dai^5NC*=Mc~3$v!dcV2S(2w`Ae4+GXhtCP&t2r(19Tnukr?q^&9~Ao$4B z1SLbjh;gn^gRvm1o&!>(0Q1M7)^7$zAXM!lDzU@oV!4uS*e;{IaSZ9svcZa3WnCQqmso)0@>LQrC6QKn_YHn6$y+e zd?v)*L~_AnfslIii?-)AYF*zSI>yil?)I!Q2t7@eZXC#wy0F_9dcTWqcD?mTDN#01 zh?z19Hl#=+uL2WX(gP=mO*7vB1#^8SHvl>trRebmQ)I?t$*l(Qr9;B<{= z>z@Q72^a{eW3|ZH4F$aLSI5aROYULES!qyB-Qhg!pI;bxGQX^wc&?Xy9)@!14#HPE zf0d45%0ZDDl3L_#!Cw)LwJn*1s!5kE)>{uz&GSfdOONSdQknaPB z-YU#f10#a1RLltU73W-vQ|b(AQ|dXPxy!T_aGB{0xN{OZ!_R*_sq_7O`h4w&{kRD^ zzt_*Re-*y#*-Eim%!ee60T z^~|nbxL(u6sK83dnr7I^!~&5Y2_#+4y3sGZG-Tl5d{!Jc$epaB*UI*MuTA{| zWv<#pCskH5$mH|phWeAAb0`nNZKK8|ykwRbEp64HP;_+4sP2+VylW3BE$B1JP;zS#0Xqc;p9l3{9jDJ_1v%r>MDZ+*}+Du zqEZW1y8HD~WY|J(j1j8n*;zyadza^UJ%C-LkW}1VZJX0YXM=EUDckBc~^{>2A=j0f6(7s#%-OVL2+Bxeb0s^Ccw+%3~#rA#pz4@z0T;cSw zIIb2}t`cwza+pMDq@`e;;sW=y^#xW#TnMi3%X?raNzj#6d_2~|^Gz+_xFMz6g3e}$ zI_~ry4#mEZXIfD^h{x{*m;I@96&N>q0{et`wJTD!jI4YQhQqPaEt%baQu1>*Suxm$ zj=_IB@n*sxdOQX99x6~)mNqO!73yUEjso0Ae{M@V92<`4)SV%ApB`2 zz8?<q8c-g>O{Js%JCT6XX4eV~gna-Cri(ND|>s)(JX>1w}&kaGe>&AKIR z=0N(OUG1P6%j9gq_Dv7MB7j48t@!x;)K~MB>BveX06!#ZRH~mg830U;REr6WMX_v= zPhi~6T}n(|ZW38+KKT{&TkbL3MowsL@M}ZpOj48y#S<3VKJ^`D_xgfC^xV8fEli^E z%OnxjwYx8LP2WMNoTYzkrQ-a!cUorahr9{NoLrN7x~~knRh*}o+ACBHSI(+_c}F`9 zsFv0SDFE4$@9t>M0!oYcbX=%=l&D>UU9)UVNG6hHkLSS-@ZItFdo` zD%fGoSS>`YS77m!j-0_K4@)<+!a#-*X+WevpIL0akV%qJzz6A<-XX_~2rJSJlZLfOHcEt|y)D`> z>}YV58m6hHBZFSZkY#ThxiDK4545^;d+Mh-Zlm1R8ItednJ|tCm zWJY%D?6aviw(=e>s+5)9MtWCr(!|-vI1|1kQTDM5HIF*H20FEUK*BSaeBQI4%HGfO zz{_%{Hw*gs$1}ZuBnT10kJ778nM(c_bC2F#=A5o=U#<#kQ>UC8B*+k4ccQ{kSj)MN1r^nxY;b8#`js9%_B+R~AkFAFA3>S8X*5g3R!a;(qz-^h* z&#r;614aA4xm`%CQ0#{R-`~HG=sD6qhR1n#&b8?M`Oly+RR0~)kKI1>`|h8|yED5G z4M9UZ9}-5_&{YBZv|HELFg^I&?Lxyg*MBxPd`1n{zx%~0cAP{U)%ENqoe8}p&Db$# zs~oP+EdMy|I%9ED5fcl@@O+cbU-vWcBY%Fe!JO!%b@)ubf4zJTSVS zBatAEIx55Nn0=J5vYco;pZrFy(6G`~3Jxhm#;UYGr=Xy>Dt}-!K!a8DO-EfVJinEL z{vRMzNfcCI*SkpP%~F0qP2>XxrE0yDV-$O>CI6a^h19Gq!aD5iLM*?YHL^&fCEZs! z=69qC{76+#Gzc;$k}M0oJ>lgr)$>J9kYS@kJ{%)33tH4JKi}CA zK9b==XT8Xy9m|KKXj2^(W9=a;w z$=8(9-eZR*$J2I`8*cH-uVP|ge`Jy$swsEJW@-OF^nr!ADsVr|FFXo~e4 zx3OQT_Ec;BI>3H8-tuoz^pxm(@`=nw7B7vnSNgXi4bneNk;DXs*Zf0PvpM+tQP0j; z0q7HP_1hj0#u@3>BCdJNw(Pm0)cMTjQDBD?gI6;8)-@e-3Qa;4;ct0` z<(*lFvD&AJwWnFdlvGc2H<*fZ%c@#$GZ5*Q%@?B0x*6pgaXr*jMRL%l+qiqFay9px z;1=~a4f;`yWxOs5_JAgLc3Kch@YBQCfShPW_^hxC#Cgq!fyK<3;gC|axF1IcxZTe| z9aMHV?3~)KONrxntP=aUk&N)F$!&zDuhjTB!UB$!%1TS!#X=I4MY|5zBgyLMjn=ql zW_y5ojPl6h;Xm?9>+z;^D#L7A$*m;JH~$V{S;eQ_NHn}gcyTWm>pJ#+YK&iX>+e|b zn_oSiqP6*D8C?hBw}G{Uj)9e5a5RV!?V*Gb6HbKPoBPCkp072dAm;q2L3Uo6Doe0%!hu7A<_++<2ihag{-1Z*O z&t%akR%Y0*B+Yp-B#L?5UFkZz?x`ewQ459h$a%tZ=QHbWt`$k?1E0z=E909VxGT zm;gqqK(-?72&KjxVMLSitE<>ZOEb#&_e=#wYN=MJmh2R}?Joi<(OozQbB_azF?6}E zmf2fA26Vmy(^`}3=(!2L6tS7hw>v)$E86?k(7@#)YR0R2)a?l zP^58dX9Q}>&p{gEH;S4Xh@C0!>V6Xws3%w~4@it*ro}mJfj6<&sPW1QH`PhHUkH8T zp;=ml3{@D%yiUw8X>JL9T8q*Gsbh4*@;q%zKCWg2OesgD3nkGwG=z9za^D46$Q;27!>7jWgI)_ zCOMrG&m=}?iS48}=S->;VIV7)xNw%91NoO{q0z!9Q}u6MU6Z10<37e|&HvBoH42TC zcjM41A&hHG{>y@k4+Gtr`h{ywE0gFS8^y--`N-gW`6o<>kM}m*5-{}{=mxb-aiWSZ zMDyb5AX(x_$#E_}a(4>jJPW{^IJR@f@cM%_2gt+Ai<~?o%a3rgk7hLVKf?L{NF6^$ z1dT7bDBA-77ZmG(z+(&gSttkZ6G}NJb$+pNu;ykt@B!O_>F?DL6xMH=)9CI0A6IYT z5Y-!g`wrdRFmw(j&CuN`J(P4wmk3A<-8j_H-90pdbVzrrq!J<^DGJ{C{?0x3-gEzj znZ5V>uIE|nv&=fW%L<&%h-ouj(joxktx|z5Uo|6`j`r7ZTp2w4%i3)BJ+Vk98g>K| zI{eKAK>8;@Z)HTEB+=s#a{AhcY?DK)6P0wr zOgrK(ft}Rp&nEdJDt>!Tk|qNB1k64xyVUF?>~HPG;j&cu4!&KMsM?G zy=%^oVn>WVBqVhu>Ol&=qXYT)6JBMsqAyo+05}L>Xg{GG!q~q!j}js%s=9xGG%75e zMUAr?v=&Zb#&(d);a#5sRtO&_g>}=Q*eAR(hJWF!dVYkZp(Wd8bN9BfFE>o))?(kb z`4GNZyv{`tNE3|Onjyx6!LbDuRFw66@wr|nLr9g)%CLsbe%DKdaC_FO@jv&4o&T!m zjwdT6DR8lH%OfZd-{d(OLd-}BU>4K)}iMX#p{8&&{^#sGY(N3;mi0Q};r=3j0 zBWmr-v#QO7%f|Kg<%jqVl+O3rD%H%}87Ga5Z~DiBIrTk;tM#~y5_E0;DD5-QDY5xf zPA-*Z@h>idxoZEO5q2chA)ipe{KX^sdvU{;lZu2e0MOYP z)q}gnr8JM!NR89~k*nipu&Wa{XnfhV&d?F+GpdXZvFl_uDi$uh64b-+D4O2d?X<1n z7nu8}2UJl)@qlk0z6;{os95_S%|tecHjbj#yAkCT9xhfCgl&%;YUh-HtCGfy)mLa8q%cY_W+a<2 zE|VZBRO*X7K4)CEGlVgc#d50qRnNrlWYLisX=>Yq!olu_l)6p}Xqd|tz@UaaN`)B* zwsk!USN(w^2dC;dF~D}4c>jscGk4laT9M}0YjD>(dflMAk0|Yn)h?Z>XT{P1KJ*fU zHEK&qc<3Zi4-^}fOn$euAgwvR2&^qa2-V$Mz<$W?d9XOqQO@dQXdeG$eqmsC3Lxv6 z_hI^JEcRjs=ZnPUvKeBWRK+54#S2v1?T(hupO~)luT$}R*BV#=Zl|M-+uDu}Fmxvy zgCnv9h418y9DPJ@6`7lbEFAez#}G!Eo}QWPpxuiVPk07JBvCE1w14R8>-F$k)9|C- z9JIpV6|%maMsoV7%n04mZhfC)P?s*^kT`RN%l5Qg6d2>XTJ}xs_25c;`CB9A@rX|< z3;Mq{%siQ8>^|+6KWJSOSoXwR`R!+HxlKr;Ke>@J_8V&cwdF@~)Zx`CG3+T&_0v;x zJU^z12~n>Zd->W2pZ0*u=>Q*A?6X3SvMd(2O~*C(w+VKhz(%XOgM)|vBDYSJLI%DA z=dLUfEjFVh*s*8=gN~4dVre}fq8jir;Y8bEiUk&9<1j{+X|Q%vcB(~LY=d6qz+P6L zXK?(oxxLnv(j?k!38E?xObkbrCh!b>+R9+$YsVZ!A^u>GWKKLEc|F8>wNT9dc%bn4 zeU-HuxKiXc@e-Tiz&Kj-g!^vJL{{!ko7~wOl7=hPpxVl>%Ti$Qr8rZ!N~6T6#bO6u zlwXL4d4j`GXr7@0q5vlzD*=;B)hOF?t0qfZ-yl6XJqB!En+8?-uEyMv`2S3aPa`=y zv1Z!%ie`Jh+4=c@IvV{9eB7rR*hnk8HFbX&MO+VIG4`D`eOMU05gE`jmu`28c<=70 zu3-uQ`-X_q?(tTB__Bs$1S!~HmD2Jdxtx3RG23I*Qd0OO2u0mGPGgMvDV)R_CqXc;}?$N$X&(5lBTHyL|72q74zJn--eq2t&X zrD%-jR@e8Wh;Ej&<-m++j<*Yn)S!vgC4w7bYaqDibfdMJjP{ma7_kV*Dkv-XPOQKZ zE=KQk#IVs%u*&0{eCUM~`G*zIBP@wx4l{>?A|xFqK~JPdKjA(SpVB@whxBBXPxP>h zI(TWg%=M;L*N81E{;i+&_ld3wptRwNUOy#F!nm)MI#0qNm{+_S=7XKN(SGh2sdY=MoMm>Eng}U8i*2%TGy*(KV0qkvH;HYK57jCdB=&`tf|-DoQsgr$ZOk~uxOI-TIqb-q1y3B*C#$#PJ{!|1-OsacP2 z9u7uCnB*VI^myMf!P#p5tptCs=rmoAyhSI${cXGX7wn?F0i9IpS9=#9|9A$wb#Q0F z5+}_?@=w_670%^rR!is|4)STjzrP7ni>g9#o=2On$;nPQk9YG|qVo31^v8$(r0GJ(L@?a(rXTZ)b zY|qTG4Lk`gM_7f3nCmADfkmXR8CFZbj=-JvQ32G7J(9LaSAWHVx?M+=i`c(GqQ8JA zMFHhU5)m7NrF0?S=rh*%wT7|2A;n_gjOrnezi6Y+h|vo;JF`T=!!*34qG^BZ?!ST* z-H%5ig>JF0sZ?A9?u=?o?8gn}2??^FlpO>CdoCkh?|5D|iBmO(EK^p`0UG2ST6}EO zJU%%FWc&ZD>rO~>Q+GgTMfeTEr3w(#?n7QOM1$AXebn!6C7_oS? z0*&)#l#C{+Ysr4^$P)T~uKn6uo}vh%v^2tNEgURr!iIq7e@ z>GiymMhVNY_ASP}4gqZORS@lbww8^Z#HgQ+RiW&JY3ar%f=MTwjtt?O)7DjMve!>Z z0}n}>SP#gL(7W)d&raF9L9pQ`qE&vWGfWqcXs@MK!khNIWF%lLOto=!&ddHLvbj;b z*aa`;)?(n+!YgQm?m=yGZ*kvJnR7T1s_x-~>;Wo!?b7n7x1XphUB5sB`H-oc^I`n| zSi`Me%V!|EbGQB`#Y0MG=Um8c#E~HM$ND6W;2~bQE}To7eH%*ZOf!K&UguDr6qAGp z{`vR~{}7%+qnKW_=Z_bEWTz<}UH>36-%_IuWQb~mL>|IMUUl4qXzg^Tv7Wh>;OO^9 z9K$Ki%>j*Pic0`sKiP8bEQM{#&3}H^u&23F&4JDA+SE651rqjGh9sE_8e$zZVvc-C zE|68?Xr?ZAp|} zT0EB7cyY+w*hy(McZr0VK1lVuwziv(D)V5dG4z#wy3YOhl*hjxD`Q)~TmpvUPn@g4 z|Cn|Ra_V`vLaWS9v}4|@(CJDR)(4>>c*T_^6Gr+!-C7?kmK|_q&L$EY@+8PTrez`e ze^H3`(Qa<}iL#Ic3mNe8_W1g|E(q9-&Bg1owRPY|-I(@``tp?t(99(%n*@QNHDn$DIZV^#Kd6;g$P0CZ_%1;{3QT*-R*y_nzH`2Db#rPSD(t*QsNa3XTB+8xyIYZj!_NJ8Is*9L>bP-cAaJ z6#(X`9}@MaGUKDFb{xhzJ16)j_i`eJCM?HCms=c~ZvIm>dg??X#XFwP5COHq+esGWr4kyM9?R|NScxvIJ%RTLv)|NcVQswz-4eLz%(S-Mcn z-vY5ZbqlC6!n||=PqC%z)K|4-6VsB|5j4VGihmdc!GGSC4 z3kdCrHBPvT@foU(C2sCLFWuqrLT%RDw46b79!6?f2V)oh;j1q^6>M+|2Z7Ng1{QmK zR0H>GeTd!H#8rdUu}4jdlD5E^&NC&YyYf7j;L$`wF3w}_3iQyKn{t^x_w1EfeAK~S zg*rq*{7!46d!Fs3p7JEo%$%>7X)Tzb!f{XBA1mdS5TwRGZs865d%AG0)4#P{NQ_lZzvYWZ18LCyO^|wLleiu4 z@l-5GqfO*={$chj*ZVQjR!bY^OWNlDZAAF*!8RR1(>We%B&Z_SSZ-x21FYSf!oIn* z(f|Pi=?&}|ydwBvyZ>EpwJfyo;_eVXyw$QQV885Um=86_bA#}e z1V+|27{b@&Je|Xlgonnm1k2!l6dNCyaXR_q%Iy8n*BhO$%1WA}l&dB}uG7hNa@oA+ zjI(|zhx^pFWG=Usth&Z4Jb!M3 zgsIaOXh#AQ(-n&M2Gh@Sh5hM$Hu|7pVT*t7@eArQyA{$jnBnq#6^wFdT&@sZbVJ|T zaE%upX!U!iMuG97x^HHbY?8(~M zaaqIP9nK#YL7<$skV^eJbW>BnC3FW>VCWdTBSX2`xt%XV&DFs3275~JzTE&u04FK9 zEV}Q*nq^q^wYehAEMGk742I#na8O@F_I^tZ!*# z`MHl{(U3k(a_7yyD~}72`x`#c&5VA{JyN6>iV2*&-C}6thru+PEemilRMFAW>1OsC z78M{eF9A2(+dO64E(YatpwC~Uo4TuLb@es#s!&-w- zhdQH$%h52-me5KtMEam_6(=d;xWdNO*ztT65VLo|7B@ip-yvDxPZZRf!mzJOYM)e! znL_+;_Um7r_C4%Ugm2_KO}iF1{a<^vkyLFvK*j1}%W=J(duYb?1{dWsl8itd!7Dk7 zU0k<|jp!>g>rqvNz_Zw$N&|}4sUfI_o)PX~lGfnXe@HaRul`i?n#@G()p$L0rLIlR zIO^{tS0o9X8zONDCCgcFLj_;Ve$cPddPF9J{^f-a8nCEJnHKtn-@gf`YGevk7fF7@ zt-~XUmZ2CcsAekJ_sYp6I*}NefMJ2ziIiC@ID2r5pN1UVxyB^H@D$-RBk68?yI@ zdgCQDu^OB*VDy+D?~{H8SZN-b2d#M90i}936&B?+6O)7`Q$!qQnyz+TPUG`#vTk9Jx~3ARyuHu{icO#IEI zQE(5mQiWYFJN+%C_q_YugXpp28*)oM5)azDhS{hE5IWa8p>1sVms*%x8>sVrJR|d9 zOH#B^wvl5`Deit{mLlgyV)%2h0gPs*slrGx^e^5(OJ?y%(IvGm~c= zQ&Xh4w$I0D&s-!1YG}2C{qJKjIj~PcmeI575yvl+n#dX9s=Gnq#pD9!JtE~k3wECp zNT9fO-%wf)|0DFjXJ_9ae1nTt9%N{zmkmB9xz`nicB=$yo`xA6fYN^{d2LmI{cr> z_y3+%`pVdJdX%MkpC3}?)9cled_37ePa!6z3@G$%FZ13cdzHO_8yRoqao^peLdh{3 zuQ}9Jli}Z!&Ygq~c~&grRXz&t$qsWHtA$Yr`b;JgFB{AhnY`0{bo<{=YMF+@{tsf`i|(o!*+ z`m(^X@JdEYQ`>v+j;by>VFX_t!|l^{dmN^2ko|MWZHHPd2mu+$6M6`5qk zrNLsD}~ z8khCWmduhIwA!H!@6|lIL+5_m?*F1F5`Y3<|-Re&85^)=gBVi^h@6MV)~T!>-~xN9ukPN>MqNy9fy zga4`~@DkN=D)$JYOx}^x*ikp*J_0KMwQ0TGjRUE+4o5s4qc4iSXdZU<#vN6& z?anKgl4(QnPhgQBa7k7Eghq<*<)IxjRww^GCPmZ?0<kr+=oV8wqx;JcvMxU^}uj0=%!?jL|T z&&!}rO1Z{VvRhpW&ub0BLn89DVza0w=3szGe3lvMI~M3x>*kp?D!FbY!J zUW4ibce$HD34geoFo)mR>Jh!~<6PtSLUO+v`~^`$QDQZ+1~W(=e1+|4Le_%4;fuy1 zY(A!Q&+7PWAQ==x^?&O~B9suQ!@6>%#0FMDoVwm$4$$D^V3GC7S?eFh|5k7J*&X4% z(D$7P=g;+c#nOD3qXue0{GfZ_U8tZ1HK)7P!`^emyozA`)r(e|OXK&iH zMipWMDMRu&E2*FGRe5y`r))G{OjBNA)yPbrrUB(*7j z!#+S&^*nqUU{B7ve8fOAdV*cvG?ODY=Kpwm^!Skn&Jy6wEDO&GL3Q6Gwzxr&tP&l{ z*>^aUO0 z2k&bo^7Ng0Xm)DY--V(aKY{BaKS94Tsx(bfHoLn;z5NB%g5@Bh&&oI)fcRxSPIKhU z$=GxB|989mgr;-G@mjcpudY9q=3k0es6CZEuE)}wic20O9F603n*`hNR|~d$u-O?_ z^cbu%#?cls+W6a}cp2@o5=ql;yf5e-+;wj+kaF%yqU7qDY11Cd1P*z8^IUmkt}yex zG@-xBJrwxL*lJg?W5S^u=N`9c8+dE2K7E_|od5N&6J20E(+|zPXkB}@+JM_6czXLG zhA7n&&8X6|Iz_q z`nHo>iu(x3CK=RrGLpdacKmrjbdlyTV)9k+CFW!ei z3KEQriDPg{$wni69FMl~&@x|6Ykn4J!gl;*j1G8mVeL!UZS;Cqf13#9eS_~A1{piI zk!%DEk~XM5aIzpvQC>56@BGlBqBv1hr2q4SG=gVF^XuQ)ue@(~S|yf6D%M&0PTSi} z@lLMjWSiawm0K3?_oVpEswU);M_|s@S)qCxse~feD;?`j*ZHQuft`}Toc7@}CS!#v zy5LD6d7Zm2v2I06mZ63n7{Luw=~BmY6E`Tz=@8*B`W<7+WmI-7rgv&#Kq z6#)J^M?xJ5J5JQr8wzIPq<7xQ!F_vHrhOCZ8yfk(=WwChj#n;z`Ep#SuTF>gIqjd7 zd3WlVEoVLPAgVbXr1b6VW@BeX;rO~d(W^>`+iv@SdTdQm0C^D2pf!sHLT$oJ|nLUQSQ4x7$f&^a2MyGOSf`9b*EJ? ze;w=;A-K!_;%$Q3N6nW|>bxzj971BVP@HQA$Gu9yY0H2`E>B>L^g)1Ce;^T>#c`49TAXOgV zFJ{fc#pLo4S>V~9;&4&!!kR_`+nxn!MqVh-*OVnBw3O-vE#mu$?B+xg=zQ@mi|jlJ zP|_}^9uFX;mb8)(o+%rvIC;$OcmV$P(NB3hbmuq>q9H%NikIcpL8tOCfCu|<^c3zj zG$e)EEtAZ%=ZI2|T{cv9(YfFj8{CU^wch_Tpl?XUa^kxsE{!!wZ}w3QKT1@wc{Gdf z$D)xgNooMWgB>)z>ucDD0Raqr4ci{jrg(GH-T!x&6^qB1E{zt=;jCs!5>8j+Y*iKS{2~+tc*HDm}2s9BK z=Cd`ZD4?~Ij8(EQUu~?TGVQ26%C)00K5vyAIdPQ}wIih4_>T7sn6`Oi5oQ4}MMv$H z|v*iWu4~9fXCNe`8)fV4QjX?2sX0x?(;DhZbTtcca&HNZQ%4;M;%kCKkifHR6rh zI|`Or2}w5@#zpXT$PC?I;U5iNc97VYf_-*uj;cBF19i@4Xk%m{BSr9-J`C<8G zlC4OkSfWw`6tv&1WAFvnC4Vt*{SAtWHG;x#mYtLZ7O3(5&eeVX<$`&aNxKD_H-rINow{8!xIhHOFkiXzmVn@MPH=kxXr| zD{#7Uc0uK?4Qle%5aTq~2xtDU3Pq@+%d|2C=f zRa~yx8CDurgLg7lgkwHwSzCHEqjH!9OF9+_pDkDG+5Z|aEB8iR>kE?@TfgPup(+&= zEkoL2$5}v=LOeZJ0hQHS>UofF>us;sMnAFa?5&pYl81h(NYR#&R7Jxs)cE8m4T{Ee z%);!5a*IV3yO)sE^H3>>H>>=$J^pUHi9szzq*6zPfLkkIWvj6AYn_E73g*&Sz(E2%^)P4|4Q~5t|o9Lr)95m^XmdQZl$viG?Ez z?7Y~sJCvCc(ynSHL4pSdHS0#Z>w*k4w1-=$VqPp?voTsZ3yj~tsF7dqE;?F)Z=V_ zNi1Zt!Z5ol0TA-eBi;Pf1xDw4wugf6(s5V5^?0D@ZpHpwX1oa9lu0LhtC7Og!^ICs z>%foC288wkB5ZlMvlSx5(LSAurGSWxFdvn8$3=Zl1G{XVN$`odn0Af`7TH&y33|=% ztqupyBSjN5yiS~L#xx!@yKHk;l+~-Fi1z9G(D{2fi0OA-Ao^tnv-sCg;aXY@7>{;` z-xQ7>-}mn0j+z$9rTxpELZeExWKtHKL=<$lTcC!Dx{y3?Pwf9YjX}C3x!(>J@W7!u zAI5YR8uO509i8L++Q(tsspm7PforMRlLi)k)u_#-VexgtV+gd5LB zj%1py$E5mB2sKP{$epOM*N0nBK`;UwLzCT!V&mkz+Jii?|GXVQv7m)?LKI&Yx|b`6 zjwvq*H>Hzv)FTdYKdHvyP&(PY?n#VQGmG1%2WztIlUAHo=x!5 z9-+ibb?8eeKg1`{cfTx474vB_>V6|4wH$=>pcB@rRPEb4j3?`ADpvKFmlU1{`H*Q3 zwucy`+Z#*8U&0aoHeGP+V04O?cps=!_%|_}Wu%k4p?#wl)|NstYoDOc1IB(~Nrgzk z2Rn6nbBMeNbhWMtBK#W0Y>TH`bnHORhFe;v##Or^n?Efg%pTXQS>-dwOf0O$G z8|z)G^IGWid-iN4`By4$_eZ_af9=o$24qMFh-2Ep76tmpQ!ssl8O2qTYi5W@h)9O`1ywdCstmbtN%|w{(Z?yiM`5s@$kUpckFM_%tOfQ6OK^32M~)<}eEb zv_J+Zj0vV{u5*{QWd^|6eQsmF67Y~H<&@Tqd)3^F?ykyq^Y1$i&pdsNSDkEvC zSmk|Q+b{OFBL^|g17*LbbQ;x^`X6xB#vE%2X(uX6$lLTlfN# zFwfi{k%AiifL|~M^^L3Hi;kg8jAkYc*?Z*6(AbE9#zkH|d#2ne|8W9iL|mCE8;W@y z^0Q|GYoGG#RXr3wSHaaElZvy|jSglXVd<%N(C{4M()a>w|RJ5^sa z1oOg*RzR4+&keMQUts{N^FiAMYu|q>|Map)z}FFC7D8>zhL;=pBJ~pc-vZH7`g;8= zVZ|qv5%LA$Xk`lv=O; z-M(Uk{~%YU%b#Zu%W-EJ|{SWYJM6~&fZ)Mt{wg#{%1!Sfwne@ ztyiLmr8XAD{P_GKiZ}s~eO_(@F}xYfH$$d^rInBW5#FEx+1F~@hyT9MP1H`5}^q) zX2g0$%7I4$Yx?~TUK;X7aqi`1vcd)i7gyn24snz4X@D=N_2SOunC*cfa;nmhbKa)a zl;Nq8Z`#HU#U-|3ipGRgt%*f|!uEqhu1@2Dvg4<$ED!Es&Y8dunVF;>|3tBp<~pnwcigVwp{r}-J9#v)z85D70ihR@OY~W52%lBgo4OOMZKthJ*PoI@ap#y)an&#p zu&yYV+!5t%(SdRTvZB^@_M+TZ2(ZD@;e_2e$IRd_Iwf?rZH@mI4g7ND~#Nw1y zj^MgrN(T0aH<;q$3rPF%n0u88NqHNxqlNQhdFYGkWgF_#__YA0gFfVlJ+>faGl#q|NyH^SO8}2Y+;LS!n z%J>R1_m!`mtZf1q`fYvS`Bnm5N7y`_=Y$F%-D~lq$_qO;2L*;04ZCdg0Wy|w7IYVH z2xSEPWW|G95F?z(ndbg&kwoHTytOf3=pYLVvc?!3Fas=`InmO>7ayQFdir#(n>qbI z9Lc#=8V#`%4lI?v+gkT1V@pc^az?{fg;$Z$6QQITs<5TBg?%FEPLvw({u?>|-r@xN zn%G!I;fgFlj0Qa8E0>t-Y_#{E=}J2UJX}+jdo>esTmS038-0)RJ5nZzOb3tgSnoMv zKj5eHl=QpkC6dxFu`AH-1EgaAAbVMrzLcPIfc`$VeDl+Gm(5^f@ut6K@^Q1YDFEsM zIoLUZ*3lMU|22~t9Fc}3#p@J*AzXxF;&x$(>O0@NC@AruGK;k^sY{65bH}cO+Dn|@ za)hOg#1ATwp8QtKseIRuB5yL^Zo{|!mMbFNqX;8ZslVA`hBo*;!{nr-F#J7P$;TTr znal2j)l{C1`T2LHuT6fn)7D^MZHCSf6_T`3B24ss(yDp=YBGE@@$m@5-zrgdpn{0N z5pk2zj;9SDC5jsqeRjuSs2~x#zB;>c!Pf#&UV=~B>2EdL+#GqNHDT;WLpz#YNWVCP zD+u8vxtz;r>d>|ziG|-t{&~D+zfw9BtsGD=CkZe{jv{l>FpTX56hGL|Hv7AaqZfc` zU4N-kr_^wI9f?6ZYU2p8tfM670yX!BI8Fhz$=fh&0NpafsQXlMWG$E;mEKHOZlO0UZI(nCVyuPKIGQbW=>^sSx#zz zN_z%_yhJQln9qDg7Lh+%9(~4g1hXjk~&Gg>1cHWzVnM|{yz4iwz7a4inur+)>0brwr`o`o3 z$Cf*RcU=qsj`%8|Fo7c|7~IInHd1W~6eQr&uz+TBWrw@W05UICvvAwqe{HI2C}{{E zrIhA&glt#KuA28KY7!&kqhSJq-;TBYde+MRMl|J7I#F(i+EJY9&nd3!rd!kn8M_Rn zjr>?)SyAsd?{LDASD(t_`e??LSk;As3rLX5N&%`1$!IlV3M4j z*Q9$xqusN=I0EpZ2(l2@+95k*mc9rbcSOIJ#5zlLs>n`Jtb1e`FG4Z0JO}qHE@q7h zOLkht@*1~oHMbq{P#fdfCk=TK-k;vp>3ha-$He}PFkA&UzKRRnce_oR#~0`vO4`fN zWOG6=(1h7jhqQN`3gh6*vfw4K@lXoH2;3~BK^(_uKuv}Uo-u*t_&nKh5&oLv46_vy z4M<$HX01Op?rpmyUvQ#sAY}IQaFcoPvb5f0i|bzaXWu}bKCfHE3K?U?~L%)VBaRBQ z;dtdiwteUt9a_awEU|-zTX1=gTsD)2u(Q+1m|dOaJR08Ms(<1#!UaVRK!-ktGJq_s zxJ=ci?irezqU!+3a)88`(0NFGa1-*oEV(BlhmfB0C<89g+{+<=zv3&y+*V+u2I)-# zya-=kVN*MNYjj!{*0D)hQnr^X$){p*t74OVWU#qX`5827q>r8#B+&f*&*9n4$udlF zYOG?H1nJ$Kh;>^$+7}B?VaUg7Uwp_GPQm{f1bnw-Kt}f==!4iWeln}6K#L1~irqrk zPrqV|O-EOS5hE7Op#qbU%p^1$rm`K1mk2w0Re*`fw^D*Pb_Xr@=89GN_^Nt^jnuyF z=MqC075^?70IZ-B;W-!$@!03GMmpSJ4dT|SrnO}Qwvsv&HR7pS*ovNvq930+D z#yS$w9gasMXIM&Wsn^C8xOMjTIn&%Vn!L@PU>5}@g32G8NWP1 z#}_oE*}oF$YWH>xv3vX*EY&4tl7&%?JPA-58f?HzR$oMN&|%Dh)B|P&!yYcF}$)Ohobx zq#0%L1;QAtHTQ^;ebX#@C`5GRuV)q!mpCSx9%{IQFNgceqm=w`>%D;c=~*mu-`vPE z3IRS}0zyssviHxp@6ne0A(!E@> z(K9NsdjfexwqtN8jQGvep)|388jB@+;qhdQR(<8zT)?MX!e|FJA13>$61 z5QoJJ3NDm5G1ktevOqha#xqJA*QuK4y_P%2=N254F5bW55ut!1HC0Gs|! z8JErI`4z)!wrsvYLP>5u$n_S<<=&SGG6+pgXJ5$fN)z>hr{l77rDm%n9NL)J{clR* ze+H^<1VamE8^C41>g^gJ+iZ@QmwIOGL1$udL?PO1(F~!n!=kYCLU~C$)G1J-9@Rj( zfx4VM@-v>K){yeZv?4jQgGFbAx?nc)1b^OKQkaf2d6kgB(U_%I7-eio8{+b4@u1*@ z58>gWBVeFwJ839s6;or+Dms;AM>Qf2YZ1gz%Z8fgn4d{z`Xy)RZlPl&3=#9%MpR%F z^|CUx@g3PDs>>C+z-e;#hF$^{xIXo7KbtS zk(apI5LWg#oRi!YiMkj)62CyK83KDCa^eOUo}NUiWl*#klHUNvUG6d5UsVr#5c=ij z5)KUbWy4pP1lj+x$RqK^@Xj~V6rb(0A`K$ycdw~B?gmP$I2Cl0Gb9m8V~6iH9;~5M z%Aste6YclC&3QhU)W)5)u&rbQ2XJ%t!8$V#?bKr9qgH*=0{#W|>h%vR3N&;GN9#fp zOhyt*tMB3miZ|@__ZMre2#F&B4o<2jUgn>LM}ZZ$0!?ysf}?452#8rdxTH` z8CQ(qp2lRE<{lANiWdk&xy)Dm)cb1vWN~wbGd-4j>WGwt$5HcDw%H>1;0u^hu*zhz z64x#Q?bX$J*Vj3#-+M@Z2aTB?fu3*Q9Tj*S z=X-t3t>+P}@^w|k3;)F#&Qg`(R6L$nATsrz;X~u`^MhvZ7yaG}?*!b(DH9c&<2T|5 zoIC*(K|0q>Z~CXdqsG2;;^L25#IHZAi!WBbGxVJrYsMKD2(qfO!xqZ*Z6{ zLQQE9-rAMue0YpHJt2m2KT4V(8`ko5deH&wVXez)-%4*oy;O`OL2_58H%t);6AF;8aE!Z7O?T7fJ>dx(8W{mMm;Z{PU}a!6`UiX-H(5M)|Yc{#m&t<1=v77 z{;Wh-h;H~Yj6<>K@vbrrrLM|klH~;N{v17kcQ<6?(jr(ExD6&5{4cz0B(r_Giu#&A8cNjIE|aYD-OwO zBJz`cRa?%z8}Db-vE98p+|8y%cF04G5?6DAYN_4%&#v=)Si53pK9 z>~ldiU_*U^N|bzo9}1S)NYcM)whktrVu)4kM`Wo;bZ{WxcB)geY|4}@x|)-Gr%n7o zmYUkrny>&6zgB~F`rj6GZ$(1pBwbE{4i^YXg@{iI^p)Rqojx!!3|U=Ye$xbMQY2$m zQOdJb&yo_GL|riJPyzIKd|?ZybCWSqFgUxHsI6n6G^%w><5e)AKrgs7_U2}=7P=hG zZT;>u?$=#n?KZUASHbc-y(E(rhivbUe)B+7+k4@k|dcQ+(Gm+xJKxq)18W!u}M2 zMjFy?$b`a&nPjpf@`@H47i5GN2#XnFtH(w^Ou%+~-@&_Xwo^3WyRyhV+FUh=4P@Bm zsUuddQ50_htdNm5$lcV@kT_n+L2b|>ET}AbiV?o$M=ZbO@bi4Bb+boT+*H4&K3Re&ig4dAbibrXZhB z4vW(WffsTPcSmPE@YuSUxZjor^b$HfsPjq2hW!pl^N%=^^GsZv)?TeX96w8ukL!nk z#(QL$t<6nH&PtD&oVVkd?bp(hRC<^6nMz#AHKS1GT%(zyIT4U||qA5kJqH zwE)0VyIx0MPlvTb4-*%4*4WEu*G;lqlx-$W4g~q=W-jQ3(PV~IWzKw*sM5r@ zb^sN&)Fw%6;5ZvzYz@L?6$jIhvLmxyU8~WdsRS1#S8frMHVN2|hgvFlRxXo$w-W^T z^dvHk=>qE(>@<2DhcY|t1OKKg@A_GZJh!SWznE-v!^V6uZA=+A=#bBb8o{V4SBuUp zN@)yBM}@MF_DIgca?vSecI)(g1Qf-{81qP>;C*rq26;;#8pz4apdPmZ^|%+o&J9g4 zT)ff`=%LZJIIBV zF|)#J+VM%o{D$IC+O|35H&UVD(M+gndz84KcS;jUQds#9}WH+|#V=EfTPL_zsK9*r*&yr+BB4po3 zc9MO`nk^ypd-pl#Jg4(H=lOolzrX8x{^^?0`#tx4zwXz*vR_hu*UL{fZD~wY(NTic zSKp@Lw&pK!TzJo4`|8^41?X6yOf3oh>+s-dpHLxzFmK^eleDU78(nIiwc=< z)`v`@SblKNM?KFA-zwx<7W3O?w)K8Q#Yv{2o&onL$hmJm;(Qs=P&bP+mcB7qDpKc7 z5&P^65@O2eVQRQr(y)J=$}#l#h09E1eabhLc2$Py?K8+=E$#~r#hQl#GGL+8^{<$0 z=CiT4GsBs!r25X#%g2=cu1+3JB#T(ltS+umCr&c1IhV`?KQEEbmX zr@Z=Q3Fl@s2lNckGVf>pEGxG%!_o;_6H_z_V|ZUPALGrjmqppjWmd}6kuCHg-j4Fd zM^morTtLUC_AyEN=`80>$Tvyi;CBed zQbQ?7{jkdC+6iq04^3j!@V8&?SlNw`IagrtV&d;{5JYhL^z!h@)!=#Ki^JtO+ zcdCzoS3ZrE`K?gOVskvXfL>YY!K^*6BguI*8y(qA4Hn+&3zv4u#Au(4IG?&dV#Omx zr9}xs7lt~fZ$Y~aZZ5B5xwZ_bNxInzZb%}GIv?1;CSG)tz)bh$pvm9K9WdwSFGx!X z7(ek~x)GpN47iy~3P|ABHw9Y|c~zmjcuKC84s9dyKoT}Jld>m_cG;!J$iuhTOG5l} zB#qt7?#|C-lBs~AIP>C)Fv{fNH1UW;N>uL6i`dE4)2%?Db(N8 zAtIitaH$cR3zluRO5$TtkL3eP0!^2O(|c28Y#uT3&81Kv%~?KS`4o)en8~pWoeK2Q zVW}U!q1(>Yti9Mih$|Mm8V&aU7IPaee$A$F4eHHagC5w4 z4oM9vXZbMtg_+~hIm2QYf5;Q88uNH%HbsZf1*JC#!#Fm563c;vp}8kF3!@fv4l3N) zD3u`sJypd?&Ni$s5)9)e>0zAbn;(9^11tu#z&&v zVAu4kO&v1_g`m z8ah?88Z-&EP};>gGqt2z(z_`SrHC}=L5~=aVc>Kg5-#5>7lPR%5J1N*2_#6s%IIE* zBiGD!^8%1jyvDUBfl%=j^U`ctka~SylEiL92ahm2(F z?Rb|{+Qr{=(8Zt1bKhAKRB>L?9VH|6G?71bkxDIzUx2|1BdPK+&eh5!fKnMHV-tdt zb!DA$wC;JaCnI%9{!XU@SQ~a85y~dQ%B%Ir+5{11&aow_o^eWJ(I(V_Pzex2xt#V% zQ6?58G}c@N-|x{BI(Ks=u~vB7xnE~&&GLN&+Qmw|@Kaw6jc4;}qGtO=`gjR>-_itLNpn zS)Bpz^0ZiH71~Z7qb)ntg!ckI+N5_E9(-;MH>hFH|IjMgTme%=1<_#{Aox6gXQa{@ zgD$W47z=B?x#haCf)O&gPQ4J7yXfZ!zQv_UYaX(%g#*6-BzXP2{&4u)NjroMN+o!n z!s+}q-3Pa+G)$j^UOWl8;iOMfNxFEmh$X4-OQp;k&~S(28kB1IiYgP$%f$#2p#5z$ zNJN?f^jsx4haAew;Sxznz4iD~U{>v`ttIcTpY1)a;cC6U`<%)EW4e8Urm~1WlYZYf zH|3V^;h&OAM#Zf0!Oifsk;}p=P+s1<;pJ%YA;m~E`|(Kfab>5nefm3&Go*Gjs|Dts zfV<#z21mxqVGMp5$GPt|()M^EAuZ%=if09b_I(Y`jj;H1jfqluv?R(xF=!%@NBy;_ zqAM0m-H2_E*Ro~NY%rH-9ONC+jp4P#om!HF^F*vP zItm+b>P42B*JUp6^MwAgZOwV#4)!h3vTMl70#lGs=1)I730-8-)%*_=1D)r!FJRvZ zXZrX(kDw%OgwiXfvN@o|Gwi#Y5@PgyEMJ51^h`!RHy9fE4)4$%KU2Kcy<`(V?<2L1Rn-M1DZpBADJ4zeSv`mlU-=gcLEJ-6;d_xu)&Zg)Q`fG0%$`N}MBM`(BG z4J9+Yx0JWCheqhxr>FAnJ}isd{Et9sDJxVR&8FL6qOy&Q77`BuEyRg1=3mD_+ZfbM z1#IFo=E}vY>MY4R>>akL+$~mm4Bget9lm*7xf|Y3teb5GBFZvlZoTlHL8*%c|Zea#5RN~dMtTT^y>!Dfv?#_lp+MAeA^Ig=6%ZiX|f)4^M}XtYTb z7P2Kh7T+KsxnqvdVwN}_x)nR72q$PAod-5}VcA8(uSvq###u((RXBMXKkc)eIcnj> zeNndMyT&mPlVIikDp6;>tqI0MvF$%Y<^~)gy!wPi8x+aL(vD1z>Vu`#3JuwD2Dr>g z%ajSEWN~7*^wEVfdA*t_a|hA0mawCmg6O#fkHc|BB*_m~b~SK>ipp7?k5trm$r57l z&Vv^bzTQ;}+r5G#SVBgZDkx7EA%m2U=~26)e&P161eUM|c?9Dm^%3PyRq$a;m_cf? ztyyQOB(X7Hm1V!-4^tmr-*fL6EM<=)>?tYPoOnr;Mun{-QiI0_6RjvQSMJK9B-7y0 z%Gefbp56xzAI2raw4P_hKw-XYt!>{bbd+c%%FzbCCX^eI4X?E`P>>jNIvc5QGo=(A z60>f5j0Vp>2pk^v_CgzPrj$h7Lu6bYx44Z=X=-#=f=qI2zlKtLhNiZ!BUsW}o_x<8 zCN8zZCSDcpUMw1{&X<@0LTEupo5r~+mdg(%5K0~&)AXZXwnF7ymJAB+m8HFYz@W?) zBhNcd@TMd1d%TR%5SVLkd#CIxaEdLNiF#`S9rxG}#+_Eed!#X*D+^@Q9jq)I>}=vt zxR(4^)nkb3=>s~Y?jRt=oKGP32G6Yt5Sogby5NKIe)Ihc>Lh3s#E;Gi3XN!vsklQ* z!E{@hl%kEV@rUdC>$`N?#r|4xUe zMmujJn&S5CSm+2Ebe(bSvT%ZfZ$jehmVNVL>Rj$MB|wys@_oC&*8%ywcrcrHGMnhply*18f0t!Rlg87c?;D z`W&JrnC<~yr&4^eP`FB-t;Q~C{Q0tOcAU!n6dN%88BbK8L+%4j3PBxi_4{`&T*S&j zp^a`iyA<)o(B@q`zDF6AY4NkJOfp5HiE$XQT}+e&gHs}$pFg}DO4fl;i~70W{Z+Im ze2H3>JvmVE5*{PwX;Wa!+X$jV+E#O7V zk##??+p}k7BoaT_jwdOWLB4 zqS!zqt>7%^M5jv4cc8>#x}%LZR^o${gl8GwVUZ7nUiDEqn_fH%|M}Z|Dh0MObQv#& zBy=~5*3iN#-ZO96?@iVD^rS|_f?dhWJA@x86v-@*<|uU>vjzioXt;NtBGPL;`5!wU zfO_%{hkY=5#3e*WvV;ZRvozv$27l@-dy~n;g0*;QA_CqMls#k6#?a}uEQB4rv{Uuz zUha8qoB_(+k^=<63I0j$0~nH!p&Q-HIDW)(iLH zU!I5Fd$%GG?h%#FnB|qpK8|*69n^0oq%LVXC&hfE+H!xCc2;ovwLQI@0{k(Q7$#3H zP7J-Fyh}~wJt2n{&q-df&gjK9w=g0>8?L=YgLjDhZ48sq=rRs9ZA*I)Z#0;=jO+hd zF;v&`Rvji9oIGen%wN31&@3dH!DHEXsF!ta-Zv5wt}@ZGKw8`_8mC&JGeHiGb6v^n zRfo}C^!>uCS%+ePQ#%+m?SFD>2o5Hl3YXDRQb~v-I1vgnBO(Ln60sdzODorLB8Mp0 z-6l-1T^~O>lhL2}ZD;bSym5THDC?(s%+WX-Ls%tsx3xiH6dN5P6&zXGeic24sC_YL z#Z<^@@42U@3Kw9L5Rd~!;}#y$GH&&etfyVy2AcWvA|7(|Ud_+cIDbXL z_?G+bj*B;7M{o&v?6W6Vz5yh-vLv{Ws^EiBNzN@uyid-+jDcwRhzbZX=4fY)LORSA zCA3S4U+*|C(T~vKgXig{V2fOsViv3)P5=mjkM3<;Ps=m_^Z_&bf{zs!kR(1kp5B7~ zCyRF0b$jV4%}kwf_ICVx{V&cHX>};mF)Cx4Oj)4T^k;BuUG~rF4t$fGRFl_kF}T+B z$dJbwxL!DC+y^_M?s%bURMldP2Mc}5Sd5ciSd(mq=VjTK!!QMqLwKI#IF%OI7T#?% zdgF4lhX^wY?vbCC87zOd137j({R4%2DqeM9$zVV2!PZ{C)}tDk5RYkNI?*7a94OzLr6TO6rh6*|)pSROhv z4gk*sVBt5zn-_>Eb$hFN_RG=DcHOdYq@AY#Zzk%@ct?*l7_NnrKU7zUSU+3n^r=JH z4i2#{u+@?AlT8HYN`?iLICdAYyqqK*U^~aAeBWoGhR5^b~b}I!U1jl#Z;{+wqR`1(yv2c?v)9iu)wEYk(Bi z$BUKH;7EDNC5{!|Xm&V~hvN;;5gfNNQ0Bd3m+2E}&*btv@!*-+NOMv2ttLc!?xP&UqvqPP1G;yL-?@Tz=^ZzFZ}lb@8}i6&Z+i&m z3$G$4HMQi4YQ=f$a5c~?QXUbSz@g5!q$xYj&RI!N>W*Bk&~o`<$6a>}R>#6Yv~B4e zBY_YPukK`MCIs8C@mm7j*e}X!P{%l5{a|Oso{?|bgF3}QuP)fWV8<2GF~)W@lJP? zFAv2HQS2Na0nCgSmdQ7Y0z?SjJ`OHsRTlC%9(!7v#WTTKxyX^0u3L(g_qpv@7!gak z`1M;sFTcKvopoA>=aTO{QJ-&IraNntUq?MYF>s%2;L){Ck1C#X7564S9GFs6_@k)bjAt9QHH!Asvw4FO{BwP|EuX`TZ(VU}wN)6wXj zu$$h+MBW1owNbb@-o+S7883fyeBE485SIe6jkLSliNw`Zd{lw$pf5|DfU^N6IVTyNyrzFTmvI zkwFq$=5ub9yLepn(gOj6$DM9%bW=JUp@Ae6!u^}72(DOT)+yIYxn)vJw1Rgo0$cWHxf@L^3E z-(AWibql9HEW*PYVHgiz_=(=#Y}VPkc-V3C%WT|Fmu{V|rWW?jlF-g{@oz{IFSk)2_(0^gEHm@E0Xj^=~;j%jFsXnn7|^>KP2 zAY&bAZBX{E4;BHbkbnrBA)w!_x~8R{c4a)h<-o&XS5Xdf6DWm!XtA^Mh;pc{8Hz%Z zq!C={Bw0VCiJo*4$GIGem=g)><4-ypE3fj#tY&96?_cFB81LluP`lK8mD<{=FkW}1 zsKkX@Kw!=7Sueok%#D4o+~qHl9vs9M!(l2FI^Z85ogL>LPu2(+~;`oa-nJ&S*C2`K?_@v6B z_qyl1=qmwTNq0zq{h;kyeSEE3&CNtI5q7GAV?=(O1`j+Caf|J46!JkrTnwetnh#bZ zWL&T`AwndrsBxPb%amV1LPoV@Jms2{R&ypfKjAR`CeJMP_JnY>%7jz`7B$c(xKlpc zL(C|ZOzGrB-=AsMz9+xPpm75-D2|noEa_^@>uPQ>X?lAeR`$f|i24$<%3Jq+t`U1o zhud;6AO7gx^$W_xW_qO_vV&HI<{E#AP}DV7`^Pnh(j{FS>J4b7r#(k$@i9D6nUiMG zob;O5E_;b{K{p~vM&fE57ld6dCuDVw7xto!=6fU(vb4iJ;8N!c2u2-ocDdJgqMO1f zKh1=MN7edNLAf~@^D~gJzJbJmMKG^R?#%rupXKf6gF2v#Qn#r0ADy9_35xf?UgZ-e zpcU5B;(1PE4;vdvvZ6KVOWyBvT|Gn&WiNshJ$L1^KjfvwyRz+6yPatoOt@pVrJ6s9 zd%UeguhH#H{XF6|pg7kkbL=h0=>#spEd0vmR6~jK6bi9 z1q=v;_c&F4^|(T)&UypT1x#ax`>XfapbHtVocv#q+?ji;6fRbNJS}{>+z({H9!6b< zS(YEpHP+YDx%LZFey=5om5V*4hGaC3uI6i{?0Dk8Ui5C-3hcreCX(Ak&q{X8^+BipxID4pVav}?4z`PXgPpPTuEz(BpV(t~>UG5Z?s&h`_?u~b zr2B{%xbpk>2aWT^}y9x7Ny_3G(qmG)blM~%q@v}uXBVRKSczq_Uj2gy9qqU7}O!6GH zhsNipnvjO{yrbkBVhu$L-;;%-Q(H#y8kQuficPmUy?wfA%sw>rQWU*tW8}+PH>Rz_V6DpJ^kdbVcgS!a2SyIu6o-r34hH z+Ed~J-Gd72!4q6OO=}8N}SYJ+vnKhU5J3`=E6Ymlgz9SXzdT>8)c)jVGvL^u^Hdg&lgib!ebaP_OhTz^(gFfkp`(`XMf}Bg^LJ z;$5TdV@<{odJ%6(65^utv<@2@T9` zupM)Y-Ub{jK)Xp}Ny}Sxa)iYazR)kThMjo4KXunt(K}ae#t+AJ3N)rTPF)|{<(lEn z=+>f-@mE+bx`C@UnBrLdY+WJb{Ws;D(eP0<&s6L|=hJFPU}bhMk#KywIhNDMc<$y) ze`2%ZS98X1dgdtrj{nl+9|*aB0V*Xi9lSod%PDOB^o`ff91c6Cuyc>llgW2Y@^^5b zlixe^WTBVMY<`)kf1HDJ;fOD{8Dqf;wBg+2aw zKP`rm@uP~+Gm^QY$fbvglRvab(1b?|3!_N}Dni%eu`x$1o?@0`E#T+?1p^RT+@OeE zi>K)=x#m4&{ITlJ8L*1?&r9Lo^snS@_4K|G4%f0hDRQK9$b!X0aZB4`Fubam4qspf zUrXUCGQkvJRqnyUp133?se6KECqBVB$_bP0bl4dJ@wPlyU;gG)0;|GHN z?(NYTO%V!rN7~NMIiX4SJTy${oXXKmnQ$#RNry#*1`fREF$~Aea#3b2pAi=hKN5_u z9GAd`>?!_O4+S6kNQwXa)#3H$Vb;%@t-gVSAJs;TOGe2>zIwNM#XU8iz6F^4 z`ks-92crd0z@+T9q5^O zyuGr}vk0Xa)xs1ABxp-4Sa&s+f;*{m6E&JNZiFgit`A<45nouSX&2-}UCI5j94F#5 z_z+;K^H&l8=&@Rn)*QiZJaXclgQre1nW&XJ7+gM~TjE-qR`^1!AVDn7!6xa$P} z#*ParNWSjWeZCwAw4$qh^_8|jqC2nQA&*z!&mQ}Mnd`(?1aAS*cR%W;9bAmOp?AW9 zoC@GwA!fa$O|cs&@eTe=jhw)^)mK&3-SFPd6V>|mbXC}1L-92>M@)KR=-#0SZ-YV} z527_`RbWx4OFPLy$Wm`~fr;AZ8=aBw&*`7@Ue79hq<4zvg}=|xfk3w#FLPeB>Q;C= zX_Js6OlpvvnhurFpXP@QH%ooS-E-yPF7ai6Hu%Hl9^%3>n{fBeS-EX)J=p??E%5vf~7U z{XO?IK*dJNI?X4tqo_alq*7^sh!>AgJlpbmmB1`IH;+`n4rbHQ9NHWpa^9I_u$n8I3r54R@Z?hJ?b~~?rS_wwNbhYBkO2XX6ZSv z$eUQn-fd`pzm++|WAF|z%VQ7?Xy{Zp;W%nZuf8~ zMIkW3KxPL@@tq6(P+P>b@`*DLJoRNucF8{Z7#o;WNSl@@zf0z~l!m6WZ0mn~9tkEQ zjrvSvecqq%jLUGYVpO5*V@MnLYW46M1=CmTSnA3txQmdu=g^|HRmm6^xzFs%KHx((tw^yeqaWah~jb)HAZf_vD2;pD*c<@Eoz8 zoEX;o^~B}fk~?9alK*tyS&rH9LTcsW6xP^)Zk(WdD-YR<2MqAOBtQM>IpZFG5$&23 zjBu;2&5LF6D-C)JB@e46?=)#$d(IR^wS+faP0Gtjedqhld9{909U1xbbzPs(9mAy| z86mIuw9&iuA1o|4 zp?D+AS&cQbS^^GD}Qo@275Uc$(z$s?#75RpBP7Z}*^-)bWf*k8kyZ(ONW>YsPQn~~m4+tkT*mq1fEHEK7y7ArxwnSd4uYS`% z6nE2zTbCG!S)E`wni1Q29x)H3v=m6fQEZPFgu@9>?nkM>6jhJD2Y&tjRw}{W#P1&I z^5aHAykG6%G<7EuZ}1i&A$2rb8X@xYX7DGUny7PPb>&-bZ)d}mj+>RhGIi{&ikwt= zFSRc7MxacfhYhS;42^B)GAgCmaOJ{6cp*y3bK7+b2sM)pN#~PNm7o2IgkJ4WG=fl; zU|;J5u$Xwg%vNy^ub3gshGFEin5%o}*@jr~QZAA8Ah^*bD0XsZNVWLnYpx(syF&2B@A_s4Yf|C>4rdiKV?$0Wi+moOrRxIsrzV z$g9k^X`Jkq7B!bZ;DpMB9(DhL0UhOuLxwnfPx9+^k0?rMF~s?81KFc$^@`T z_oeWe9u1^IiG}us(*Kt(N6-7G;cj{(7%@Bw9u#^_hrEE`)a9RTZ%I@)?>1 zWf_|rqxnThlY@C(d$iCyl3^XCc1eD7?22zgvWZC!ObR`8tO}KBWGc=tazp^hDUwucb>%WN% zRNN1OiXRZr^ETszROYKjdAC5hm?+p`BIM-w&{8ogP1-#*hJ@oddo7esR`3jg=ph|% zH*>!*LS?Dz=@UHQvsb*APXAQkOs>h9A3)W!IaP$bUEQ7p>s33M6F4}WWMXAuC7ncK zt3mW2$@QL;4vB&*;KgI&P+fYbhtg5DDYl?3cu_?QuscDrI@jHh#cyh%qXK>s%TO^Q z*S(|_TTs)CYg#n|fU8>3convOg@RShBjSi-im<#7~Srj$LnD zR{FS*`Ba5(yjJM%><-M>@%7qH?Y*hZ@dNCm6L2k3tWfPzECwivI@nMRxH!hp!#mAh z0hm&Uy_6B(31wgbEuXulyz&2xF0+vxxX(3jVK5-)l)m&T3K_P=G8;8K$j!&6hQ=K9&Gn#n6UWMZ# zUyu@D6B%>ZO-5}Q-b4Y;4UaxJgtVvL|8NKHv1GTpPM!N9y2UOYWL>s5sGO`{8U>Y& z?0hrW(9Yv`d(&P#>_Vas!F$3}Lq3yT8_em$mTW+6t22(fB1Wrs|CM|3Or$kYaHR33 z-mlTv(e{@+FWlsccO93+hU3U+qN|nbnYJYC_$-ftYddB{J%Ui{ zgw|b`n!f36vt=;rs$jk28OV3%sf#|qotRSX$m&$Kr;@6{`#!|3>T6sWtVm2ox?RfG z#lHm}o|BLrzLuh${$ALppA&dlclyWn({q8Lw|=@)$t$dm4n^l!O!Hx`Y&=+slrrb} z0f{mxh9^$o#E%WzwUA=Ld3^>ANy=@8^R|2V1E;$q-|HKu~L7bbH-T9>9QEvfyfpJn873AB)^ zka9udj9cI?Zr!K9^BBkz7NZyZ@MEL;j?EZjT;M+F7jzHPvU(PN0-h3jGRJb_ z$(xq*KY>Qs5o`l`o4pqE>*IpGJ6yI|h3rAahXYz-C#%K{UG18Z(5GNdoz|@51xQQ3 zgZe3-CyNSh0SI6LfpWHpprgW|E?!QwWHFiJuhCukN)75kYi843cPox-%oTa%IU`b` z775dU-zJZrT0uB6I4>XalQD!+0b%L!5wV@Dl+{y+V9bb=>0X@nU3(dc*2$OGkIR7W zcdCSfoFI<~uALXp9yxLOt-qGW`V}<*ogJU2M_g7ph+AE1zqu=e`E<=k26qlQu`X%6 z)Vn2X76D_r{oEzEu4RWTuh*z(02Ki%s8Ci=IoQF&e)v}9S$w^q&)(%?{7Zxx17lCU z>cafDzj_xhXO`A2wr82@O=VMQ*z(;`dpREd57QRXC>J=j$8H(3sRu8U^_>flpNA;7#GKq%qeY5rpYzo7o<@GlP1Do(vTTwRii(S05tIAAcFF18! zT1ph);<`z@&dlcqbqr}U>n^Sg?=HFp4;>#=bf}J+d(eK<<+&W?bNt#&(fJ3Ua~^yL zx+_g;cB>FZE&cctBEHt`_5q-17p&0xD8KpwFpAs;ml$JxN`UlT&<6_EEwoK1Vp+VMt69=+e|`KILW2E((zbZ;QO*BIT&?#ej}Mqgb6PI$)vw3Z z>qnA_9QqjX`2NK!=ML7<1WG|1FqEsNU^8w#K-=r6pub9m4{`^9oR--BNRg$q2M6#Jx*hJYUS>M22C~1i) zi??St?DddQtkUU$Sip)_OJd9?cd_~d8wr9p-%987+M;V0F(ap8KL7QVrqm}2jN8>= zhsY&OHKj9YPYI|qory8Yx1ZVT1kH_mXo=gqosN=SwaZM4fAp(F2gI(im>NiTy?aaQ z?T3^Y={T(YZp{c#OCV;O7^V?k9v`M~GHyao3*@w^JFl|S{iv|-{<6&Qr4vuK_4T;# z>-j`)N(bkJ^7vqtok52vAlZd7J}n8BQuXunhez|~ul6eBgo)&<#ycAP2>~oBorepF zIs)M$>$|B1gqz#pZ2TX^M3J5)c;MM{~`26p=nAz zH7CxsUuN62VVa8T?0t%h3N%UvF9n-)zbx|H+N8@fStRCIP`T~#%A0g{Hh_N7Zd)gM3bM*F_l+BWgHy#Dp)>DRXHxH$dwx ziHwXQMa(BbADIPV{x*ZNa4t4BnT^W9gSFNS`~4m@m?Gz4v}B)^j&34vQO$srMth7B z6nSNUp?P8~^;-PG!q7@#)+tba|BS+eI%ARPhp?QnMQ3ColV-5wRY-!qYuL%mStTnb zLOHj4BWZ7~1spdgXj}A^AujUmU`!_SfkB`R<4%Q{R2{O2RK&R3`)?!i zZwoUpG~4p$?Veu;13Xz;0R)C9xFWl3Bs}^4VI%%4xc>oo1*X)hhZ_?-+DhAn1&xVn zdf&3!nrWZ=Gl^8YVLTR!4NJasE^pVb+;Yo#qBi_;4sx1*tBZN(NuFrOnd_0yY~*QA zO~mL?rG(c0S5QpL9QhA3byq^Us4e7hTXBS`g0k?}I9s#D|^Dq?y~S{n~ey zY}u7rwNv?Z9KFATdbmfsJ55+L{~7(;w-$sEY_*^6)m}hSI`B%;{_=O$&(9~wn{#JX z_gu_eWHD6NwOvv-*=KLc1EDSHY8n?7;)tSsHeV2-!!S2>h{$6-7z zwIF`7?A^^bZycdO>=@pR{Od}IbhMbl-@$InFyprlsN`D;Fxo2Tg*b(A+z(lmjUr}Qb}{2ME0{e2FyA zo<{vOE6&%*$0n^MLtyy}Q_!D=V(xFAxmrbkvk{ubG(*3E($=v9aS>+sO?UVa1m z{97TC7UAe3;ZVdl`U81fmaI5+Cy_UK0#v#<*lFa_G2~e#no;EBL0v=Rz)R2C1a!aI z%#Uamme>X%lGE9I-8J09J+QPvEas9GY_g0xH}KS40|-zA(3vueB3HAR{Cp=9+zfH$ zC17m)GEM*X&ll52K(J2mdjgiEN=ibDt`>iv0n|ejFxNPV8*uG+=#GUtnQdm0)4xmk z2NeK}Nckpd_j_)Xshqw^PKxt+X~^cKykgaj2p&yU1Kf+mZR*CFSjo`C&cur^Oc{OZ zipXCKZ(D7j&ogSb_8r@8eEsV2*31(0L(4LcQTSkiBXlm5~c^U_-~!u#Fh`+%bZog9EQb(HDqhtAVT&M8)BitF#a)+z`lMYkI3r zQ-+M^A=ALIJLTS#^SFR2#OQ7p6-5?WxM+kba2Ad+>cH9@yd_h|2^zA48ZbV|bD5FZ zS?WOMPvIAxQ0?%z2n7YHx5|AEdK~Py_N0e=K!VxCIN)WSB`;qR*xBVhE%_ZoEoinf z9}*_Jt?RQO+#3^^RQQ*5Ohu=Juu`47v{Sho%yIc_e)i^v%*CnBoen?l-#BZx{>}Auj-JsBBK?`D^(mM!y+NoozX&AywP!c@Pc~P zyB96xZUu{Rl(CZt>r!U_xEX!Qc^aB@3un=!&7gWh-k zaGw3JR(XXMDKv6tzv8M&zeX`gEM{q)9WNA)EkC`V7op{-qlLsU+y;_Jl@$IYxk&4K z?+JY$7_1jRmzh?=VVm(TC}Na&*{NJ7^rG`y!Tl9PC{OqMlOHCvnBS05ouzzIt|Auw zJ0<^ts8*8{MrGz{N1{JpCv;A@yf6A{lw8d*UtbX^dp)WaWpSmpc|(6Y0O*D;sVmm% z`FF$jkE?1-C8tz!eTbyyD*gTH{&p47*R7J?e9{e%vE~hcKS*jAW4AZC!|>Mew>=e9 zHSQG!99q8r7y0{(6!uImRRaUn`KSnUKXNoFf0NG78x0uQj-P-5C+0s1?Qfr~=FSa? z6c48!xbVa>SD6QP&wBSbYXAORYB?F!53;T@-}L8~Q%%WwabCE)Or?$^`q>EG4QGSY zf1%}6KH$~Djx!tfSAL__U&rp(A?rP|pXgOEZnE}0S}AU;8`;+?!#hX&FfjhUab+dy zgKEDYD<3g0e?!(koZm4R86m#DKJY$qQYw{_gxAhnd6e>9L+$v-anYi zAD%CZ##+B0Cl?-^{M#?_1YglH%AJ1`<6ku&hZuo}C~LoVb3#gxiv1%20w=wM-iMMHOm)rb62($^z9ebyr`(klwn3!5 z*}rVdppo8VbMS1R{olR8|MT-uLvlgG0`GA>YHcN A!M_%kLAUK$|1Q|U;Jzd~ew zrT@<#3NC_J)p)HcELpz(Gh#3zo_D1QNAcOy;f6JR2d3A-GAA`j4t{(Vlj zdduaqCAx&F??XbIzXJyfCl~&%{jB|C$)C~Yc>{%BYH8O!3#3*MH$#7^{m%V<=`czk z%viJ&<(^ipXrBFtnzDDY93mU5MWp80_~TP>AoB}fI}XF` zyF8e?D{ioKR(W&OgSV$kW7JckSxd4dk}(s;K_Mj#@R6$DkBJlj*ni^6$34?O_M>MY>GSTadM8q*6T|u^j2k)TvA?hH$`tJM z9Z(^uB3IA8j6&#ePyG2-emlcI0LzxE?DZ4nanzCJVu1L>X=`5A{BPXs-(TbF$6VfY z_nI{dqIo-r_uuKfvM8X-J@DdTX7TpBKjZ12Nc-2<=n`du;wM+Ps&R)NId3mFUo(Is zzH~~mZl7p>c!@RGSe)212;vA`oI>^-0GUPU3e=8i1$=k^{P=&^{{u*WX%FmE@=Mvlj?>2& zv38+FV5IKJ%VtnX9^k0>cYyurZ?H0DKUCHG2=>C!&+kNHR0*$6V`>!%zsrIDA@hG- zKKfF>-rQZDf|I7t@0;xrt<~@f1qtzull}i@*CPFvFK!W!PTqO0X;@Tm8C~9e!}ahz z)!E;P?|)g>^+lum;?zPKK8Qi}eszximzT}yZ^muK-Zt0c1C^mQtgVsl(uZ{jgacA=-+%&3o*zI535r4Yjzkju2JXb2Vri~ksaEq!Rl-oBd9G=c8|7#p|wFyr0 z{PcCZSE2w2t9!h;^t4vZ`mfgmWHI;V=G~DnkrVB)n|+V}3ZWXAE;x?3<5KEQRq!~k z?=YR%g1ZQBDb2Ru4*swB_dow}`5ef2g!`VsT+n=TM#Q-{{{`7vy@kHbmPBx;XZ&@<^-Ji|AsaH zq-4PP7_XE(Uh7vI?)m(8w);O2`CT&hYmt84+{mx@_-kfpan59OBfT&9_t*a;%$3C` zgun0WP1pP-gAOhOo_u&~S@|y^@H_?3!2F87PX6`kqrj831s?y^*}LDrAM(2Q*Q>h# zqR(Owe)})M|0N%=k1bc_s{b1DZ-6JuR=ED;fcg&*-P3EtxQTts{1&R6hAhD`45^e_8XNTYd-FBcG%* z&-^ulSOVtU|DorfWyGHV{QUpW^Iy?(YxMT3fAj+QkA(PtsQQ1L^iM?euV4TF!%4r~ zulWub?cD=GCu?`Ey$U{c&>?&VaR$gS8seYrj*RcCdw6GV$oOTxRBwmM{WZrGr$Me+ z8zvML-omv}7+hx6k8l|&tpP?J0*<1ZRu;4MxAV{BfG92B65lhVItjKAj$++Lv_9D?Gjxa1G@TbeaVdt z1Tv!ZJSQKLQPwZ;iy?xNtvdaiX|I`kefFeRI^OzaRA^E5Rq!*Hoiv`I2NQuFo?f5sRs$I?F5M#4uJl#^;!OFcRQv7I z7t`nH#i^lND_zf%KtHLXxqqO{H((bQBl{I3KHoSEIXTp`~;c%>-Y8++`VLnE$U}gHQHDNe%ZC12#P- zhq^7wh1LcT*-2)$MNN95`LBnK%G zRhSi+UKu7Xf1;8`SyCsooe(ABgDIY9K@a@`W9~EA)sHtXZpc`8G7#H%U2OH*3M@$f z+8Pgz;oEAu$?lf)AXn!G_X7a`Q!IUjJk77<3Oy|-&`S7Tx2$kXF=_E|oS$0XJ>Yn| z-L2tdpBQwyd)2rsPY*}4NhPUxFcbPKNY^@gg?V|&Vt&{YY1or7lw*?G5xU; zZ_o7jdmu$SXeTi%Zxsn8pv~Sm51&;!xW2iUuOUBKXMRqSftc5_FrfbAqG;^DIU)be z%xLT&YsQM)sEZ!$ll8r3lRD?^ec$_Ku_uxi60$+M{8bmcYpbY z0FL%!%r=%51TtWYab^Do6u^c z>8?$A&o@0jhhi{m6F9i3baX*K3n;3PhQ%!Dl~|lf%(H(I@jD^>2W?yMd7}xruhC|{ zz&r14o6Y`wKAfqWCV};i|9$88=7y^ULz*?MRX%TlD0UDqut%FV^>paIOBN-C#U@Wd z<}5M^{3{KHsPyxAiTiy^IkU~gkHLpBUTgbE6fT4=R_&XI+V zEpm6~@e2EElkO_A?D)XB<-Sb6*~hE}BRCLJHE5zYWA7Jo7+d=A`;p1yjcA1$Nc~C^ z9f||L5_)p~j})f=r_u=vRwj16Kjt<+%@HxwTP-lW`q|~xdp-61Jk2y%;TXE4=?X2` z7uT~ni3C=2_v_*3+hzfeO1HU~-*`6=SCm&2PM^5j#D+&r>_X#wcduG*M*)dl5i`fz zpQCLWY+9p$=xdBFih)*kywH0R-^=rtHkwuLxxLa!t!ycKV^JY{0}eoHWh>?D(*Z#` zk@z=-gb69Z{)$46#jfP)NsQLL)kWyV`JU5t0x=M{MZCt5e!9JmTtufb38#w&e=7a6 zC7naX-5$x7{w%*|k>-_-x`Y-Sp%zux&_;s7uj&_J zPJYH^{E~9CRlnvAZ?|8BBm_%eN&A1t)BnJ6oboRr#4hQub?t#rf!@`8n{=8rV&FbO z%zySAlYR{W`&7pX%Qj(XD3SzJbX>MK<_P-?BtZdyN=Os&(N;WM=u$@)Xz7p*l%ZCc z&;Y0KnF&6dkQftaW5RQoB5Kivf*rXi4b5u*rCrSC6%26IA{-BpiC|z^gx{(5)JNn; z9}#~Z6Yrevq`nqCOj^>nY<>lNs++8LNP0nlS(6j+_YJRpB0nJei{tH%m>k@E5Vvh#~wdG>o39+W- z2pO8t6&>uL@5jpTbl7)e7mxIH@;?g~8KG@D;vNCC9=mUq=JldLiw*w$|6}hhgX`#) zEkVnYEQ^_07Ff*8XfZRR1r{?iGcz+Yqs0ssvn*z2p7QINj(cbBYk%{*JEDG_s5s|T z)vnB3Yp-0n;q@-5n8%jpGrkF9_~TG!63^{~pI3~{Fl%Sy{q6oRzi66eaQW{jctGhl zYiZ^-hl1NxhU0QbEzN7Zb4gBN_=r}^i^nRkK~JSx{W}l;6)0(VQe7$ZO4`56bSF4- zkT=vSUG0u0#ra^IVMxm*?p*JU|A?&k{fjBq9f*XxXq)v-i~7*9jE+E#I260uV-Sg3 z!ub0OMD>xI@?IXNK>X=X`UFxOHn@6dElwe$gJx(0>ze&c+s5k|Pz4>=4B~wWJSO(0 zhW%)gKa2HonfLWjri=_SWU`|LGkNIOd7OWXgPlW?%gqlu z`M)j|6Z8YM7S;gyJ9I!Nrm^v)EZ#mVyEG%noq+9bx_`IYABNs_opH{!pKdo_t`;Y! zDS}l}r%d+xW1(`qqbFR7<&M#K!hE2fGJG|z2C)C|ASrn|1B%d$e#6R`YiH%Z{(;5d zKR03z>+87u(jQWDlIvRXY(pv{v6U882dWr&LubGODhl5$A(m_EC3#6$m z(svY#9yUX&6`(?xV^*uBcL(ZX)5@S1s^lZvg%sSsQ$0W_G`PD?mBDcI@_1bZH0=T6 z=-?x&1jUh`>ABql_#?Z~o;^Ik-NyI0`2{Rh(FAAYIIjAVuEMVpAA;)H`*l{N_|V<< zj8GERfERy?fSUaY3<{Rn#+t%5Crk$`=vXgeTOY>XE0h2UrbP>5bU~Y4-5m2PRJ?@{^&q&`) z$zZri*ZdM}NEiT0l5Q)&-3)NnTa}gAaa^%bDM1=TdCcn)6EH@{M&9k*RCH$qe_maa zFrH6*qq*V~`un4o_0djrQ#iJ$Y#u3i29g8wf~Mdvr@gThMft-?@{zLD!?Zh0<8+%1 zkNaQJB?(-_`KxIU@w%o8pl^pVUL#BK5ot3Ko#Rx*ViRLSJoxA7#N$TcJQXp2zwrwn zI{+?JlSz2#Mu9Jb)oMM0b^W0x$|IZwJ3xCv-frA;t<~XA%aMFvMJVmAwkw+?yxXUB zhXt)2L)}44oqC}0?-qy_!HQyOih$2|WsQ=5CEmYngZ}4gry0s#4FtD~xga*Rlwwzj zlmpOpqS0!%1LF>_n`{+Eq+2Z9;@zSFeBg|3mW+evFY|xfzW;BYx`b{|f*X02JYPX$D#s6~`WM`d zvdY$@U0N^pS4HckOpm*zK6gOXE8dNY=(|6b(yO+dUH~>q-gxeL-v+EWAr4j)nmQ`A zTK^rC*c4XGgQ8Ul5Vjfm(ZKI4+JAlK7-2vCfw(79;4OX|{|P;j8}=>uydPa>jGUr; zR@07^nq={h*b0MXdFj0C{Y?;e<^gPvY4JMATTDO_U3JX)aW{$aF2y)=^K4yGNSah1 zll()`!HU)!P2vv1-=i*Y=?e5x{YgPVy{$HmKtuz6j5Q;@f`g{%R6d{r;|U%p+KF?} zB#2L`8yB~7!)O4VK@odf4m^9WRp;a0Aih_+`RdxJ&`oqu7vx7+>@ZoGS}y;| zqC>7+l^|gznNu`l&+p!qlPRdVp5UA*?8a>(zeKtS(|G}Gg3KxbbTGuSVlMb0NUR^W zx$9+@miNA8(xLkX&|7)}K*3NKDV7oWPecH=8rhXH{y_@1K~1*a)vio5vlK5KKQdhg zpgCQG5=+m~b-K=NsY(Hv{Houx54?N!K%x5BW`c@#i~Zp=?3`w^H6f}_Y_%Mh^fIpF zqK{j!m=ei8M?=nI#2-!dzhLnHP`|b!1F0ox#e%l&q!uVwS)9gepS}+_y3muCO!rE) z0-2pRVpbWVth4Klxqod||F8%xZ34h+p)%^yriqzMpC5pR7<<4U@Ym2lN#duV@rkbI zwyw6r!nA{O7*nqk1AGZ!I9FT_R!Zd^y9T@6;_5;P(@{01!QZL-|H^vzz#K%bKVgUC zxZEI$+i|W`Nol6S)mTZjB){<5>iy2@@wshQ4KPy@^t^ZSVpT~1EI}^^UEeqH-w3$@ zODr?`qIIAI*|6?~6&unUn9eQ&JfN7Ldhy>bfNoj`>A8>cGa0};3<7}@?Q{AJSh%ok z*(Lj&cHa=kraECQ8N1+apZ`?)^9W#7Oc@hvFMYwGo8_9t+cBkC zTptehBxqW8{xlIebzWz{7|MUL4JB_jO%|}-8(B*9=%MiT$tMw%P{l>!v;N8u`hcaa z_YamDnooC|x<)I@_~shWpV9+KmJc?{0|4g?PC(DD?RI(#XlN}5!ZBC%+kmzb2s~%+ zSUMxQ9#xCdA{3#+$BK=o3M=<*Y$`V4`R{%#?C)2!?^i&^8>V8HAa0m*IG(>GpIQZL z2b42_;`aw|ABskv|4b#`cwWB;Ug%!6?_yltlJP8pG#xk7|22(G!uWY03u2EJ0{JIY z-H@Rue-n46Xkz^7A4t6L_BiR)$vyUW$dfIs>pFT#%DHo=Ns%4TUgA{PK0*loM-Tn> zFvz<{MF~jYbIC2l8(Z@3f%qi0mgIVY-u2K=m5d<|JFk*uW%vNtUEUANc-mfW_2bx5 zWH5cbk?-5h%lRYcH8gP6PiCSdq>8f@y#QYR>x3$Iqlt9(VL|J`h)r|$gtG12*lGhV zbm1PVx00gFR^a+M0+&kubCCa+SXBS{=A<{hBt(91OsYdsaOS&=9`mgnN15+v?%^21 z6Z?*c29NC{bCrvfWk*3!&51(8cqSf}yZtrrBdJJK&*}Oouq;rm;rbG`ckyxT61>O- zq*BbOY2J^VovU_|#Cd@t_UrVC)H6zSVdps9OthD3pP(n^VA0LH{>qHW{we+b@M=}g z{glO)@|Xo!sYv7rv$LG+h9OE!1#qd1qdbr$id8Mmh1?fl!FGV8(0ae_R^s&C3@}L_oS$RFW(o%_xPC_HEE_A{L1n_u7VyCw(O;;8F(sNBAr z!tnz95YP$W9NL${ZA@uLv~mydaqk{mA!Oe_D-@ljU?`T4hpOP z`+oY@KhW<7xarGPQdc{m-1oZh!>7_{4Cmc=2B3{C_8DMA$O*vpHK2q8VyjWtZlcDt zm{dHWQmVx;W2zv7Z2apVoEU`XdSJ}^yaZsMEa$C|?*hw;zAa?QGCtBh6xeo)38>6)n^}>LyZ;Or@2);7W@2>@tCV%EW5efMTxIL$_6%LAj@K55Oyr z@==E4Fdu-56-0-UYOLIe&aAVm&VNZYDE|XR*upA_6m;v&IoY}7ov#u^N-nQnyNs@` zx!wHwR;As+aeup)m#}1a1k}^7z&ksUwLN~tQ2jlBEA8!s`a@FqzsV5)LqzN@_)_PETMmR zrt-`u6Y+d(#b%?=28_hLZGPGDX}q7(6QZNj>;Zl=3*T;x9#}w|fCX-~R2EN$?%nZx z#ftm&cnMh1r!jIY6-^s(;f=&D%GI@4oWE;m!nJkleg#?Hh>DlSODHr;UB%m)HnEo0 zVQ!d5*}4o%7BoWD*T)6*n**#w7&`5r8r*dMlTHstPa*}agAS?wXYRi0qEDLiTII&^ zqb!ntD@#yJ5P`bHsoIyrKFbm&*ZLD%jw2VNmj8h3*RmZAk^tRPRy zW9=vn=Sg+%K~Z^HgfERo5728qe$odRA;C7%%qu9Y{HJB|@1FslLjmIZpJ1`wtdH^% zf_m#$uy2a$q#aI56!T1pm6WW1e12tb05&3o!_ZEFpJNv$YtSogL2)HND=e4bbo#&= zq_XmF;=un8>-#4Q&{h74rjf@=4l(IOmJuW4Z2(iz_HJ?2g%si9I!K4Fa7$v7#pz6bq#(vamTv%r8RLKar`5jzAxbBodVqE-2byIWUV57wV=TPOo?uRh@hI+3TEqH(uxir0 z@%P#@6HBRuSHI&XA&<*A@6j386*Z^TJ%awUS3kyWGA;@E+Xh^=pSRO}r#H~i@0S=# zn{hZ?(E+BYDy{Sm9b*0uh)3GrYaPcC3T*|oX=XGXO7l`eZ{=1!)4j0WOsTB5^NV23 zf3K1UWPS%6?LCbDqD@J=U5j@pYhK|>7&ZM=pLLof;h6C+nfyQA%c(9vfb)D`W4HUa zGwBMzU+DJ9UoYJL3RV7peJP;zJq+3i``eZJyrBXzl&+`@%Dj)U&zr}IX3O8slr#lIMG8-w_gjBm6JCD=Rp$o#BY!(nR~!&&WlY0&q5qfp*M9{3 z|95@%N(tzCyzn{yN3;9?>P!VdRdgQj`Oxv#oOhrDT+-|FV#~iaAN)s1mOTAOQGe5W z-v2k@8Q=RK3DCyiM8edPWA8d`8Lk>ZTp=YK9H(60e7$Uqb!~}J%Vv3w%h9o z=j(`XqB^oa2`VE=^2Bu(V=eVbLbxZ9SDKls`)Vo#7M#V7SoBc~F-Cm|4`I-!uy8LC zKz*A7<>^yWy8Swrh7o1*N%Rrb#fUp_SnmS7u`L<97xpT;EV^4`rc%NovAJFkKxtEqNHP4FHx0Jv{BJCqiDxjotSSe>LgW!k{pNB_X8PHl%^-%wnvGT zmooKR_#Z5mY6zr7DcdSDJ>PGABptFJacLZ0OX+h;+oPxsy)K5qC7R$O!^8PjLk)(K zEq~BEU5pJI<9qdyKPP6l^Z)jh2Zw}B8c|R2Bp0tRQ!YmFfq7R%@PmSgprNl(;%+E| zu_EDSlGqkX3m#^Jw5cp~9Ey70 zfgcfqz}|Ry$Tw?`nL_G0Gj6!6g!;GXWvRU(a+44`!_WO-bL!zMmZbwF^UzfRm{?)g z`na^7pCRQS`h^4#0~0L+c-aau^5L@m>tVcEPc4NdFW)J7|ODNqzxrNF{U=a(h0 zf?ZoYdlOx`#U(9uO$>VrrD{_A&*Y2n9dxn`aWK%?kwKk1ORyPXgn8+kXwSUN-g@TJ zTEFGyW6jZv5?&bFH)h!Poy_b@X35jJEP#=da$KVY5?H%(v|!bL%Jw^IK5B!b;whXm z>9<1wcW{Quw?r$KMEu|accTc4!OAqqX9B1G5Db1uBvIxAUU3ujap{!u{oz@>0gvOY zwIB?d4EH7{4sQJTrR;(#{r3K&86=A?{0Bp$;IPtO^+gUav^pw8&O8oj5hRi+`_0xx z#PAEcHUw-^wRy%uL6A{=KS`Td-3(P+Cuiq}%C<8=Qc=WWx!P&T{nbdG98WB=<_5I_J;{my{K=9hBzm%hp(t6T!^A8XFEd<^Qt;fELI4HHK4wF$w*#DY&yk8gdP?Izw4j9C9v zaC)P)45i!=HVey3IdFq*w^I8A%iEQ=VkKy?R<+&H8Fe8ZJZX-mp(wTFC}1(E?{hs_2f2H}Og!a;mSfW40*Sa4{iK zLl>490;u+yk|cYVfUWT3>OK!y;LPm32UVcON@Im~$I~BU#>c(1Dq!y@h0!>7Ub@wS zWgbr4Ipzao^xuvXaxll1RnAXf^^|i<^`G77;IZpS>2;o)o-@}u7!;Ech*X%FsV_w* zLO?}zOOOSq{M&c;zmTceaS?-ry^y40^-)7GU}`%^7zZ06Ov%Owe7I@?eyO?)u*VZneGGQ#iZ%dBJF{)WB+FAiLv zoug@f!RbHZXKvh_UuQ12ceHVryd2MEr4Y7^#qMGX!l60HrN`9F(bGq%;DVxlM4}dA zQ5+;t3jJA{nTr7W(%9ZSh& zo^0or)SG;Iwg*qrNc*wX2kdelY!`m&I18`S(Hc1P77J@{zmcy;Hy#g4v<3paVKug# zuY1~D!JAIY`H$y>3?;c@0h1A9sTirUm=%?7Z);f;gZ&>E^_ulaMi@dpVz}kls0%p- zc{QK(Vf9bFi;R8Utrzp}#xN?nRl$0k zTFPJ!HulG^2J6oioGcutg4#*gZR`rvsSo}W7s-O~91asa(MIphy5nE2vvlel?3O|_ zGD31~LdHm3>ZckAJj~=v)Kw1UulO5aY}0~WiE2Yyu?uZ}7dMABwr*6N@9BQ*NjVFH zwqt`q=lsNKl3j$+2aX)eRFa!#mLhNIz#VgHZl6kCx=b4Amm8d;b%H@9PB&{1b8!*F zq*bOfQawe^oo^FZn?*~r06PUoggeU#@z`J+qA-Oo98td+oEoaGWQVNaG!{+2@V1rd z&~j-*SD=0F;~qH>i)rQrA86G6yy&EkVkATuXdNP3CRlt?{7Xz$_Kg4}G(0q3Cqj}7 zsYTOP#wMgFS8g7(Dj3u`1Ldi0=iRJ8(bRn!2_2Y;oURnszJ9N5w{TR=wNsxjUCE3? zHIUFtnjph-mE8-MTB|>P_nfKzuv%!rc@W# zI;U#8|IDg9Qk8;%adCug-rmsR0|O^RmxqIRFDf)S%8z8#n zPKLV0O)Rm|(hs9!LsbybHDwGnFrQAM(_@aPH`q)u#z`_X7kG&L^6re`tkJpnQQ9`2V$O_6|MS0%jrCGK@f)dW$&udr#+cXm36-RvxtuG}PME@-R zg`3H>O*#FFEf)%cK%CA-765O5B7?7x(Rgle%#B0oeZ$;+zo1G~pP69>p1r00Z$c0S_ zeHjfZ=bOM{SdCldNk0{2vg`msZb<4LpJYHEsc*sO!pMV2Sw>+>p%hY`tw3B4gY8^? zxqkky9&};bUYl?n`fWNG{zh`j{le(EM4dLZwDBEuk+!n(^rAG*Xa@ww@~UE@Ap;z>nIKnv3^aLwGNFqfw}R5mOJzy zrP*oeZ}<|17JgO^H|H|>bxHd41=#EHn{7E7L}o(4cKWNBbi^7~jG`5+ zc>}b6$773(kiowjBxt$6DWTGf>M36_(7@Tf_PSmVrTVB_wx9R)TOUwAOE&L$1PSi; zf9s=Kl3LiomHSOl&2B{A_l<;gDHSuNCIrH0!W?l|g66Zavo|Rew2=iBWK8ZODZ*W} zB#aG;6O$j@_XGnC-%h$P8ew~t6j&k>bpeh5k>G)1+@e$(dxa&kWz?{ISpTS_>vP;=>kig9uhqYF2fmZL1`rw59?a;W$zRRdZ&JaGNXUjLoi~C zM8~)G7J%7>-+joAc!;uB3;)geMb4XMr5?(Uhz~L6i>mb|1y-+yY9I!r5#lcAz55ILd4jIHraX{HeHDXmmvM~Ef+tR+U0|d@J6pQjYzEIQC z=p37$wKNj;vci~gG@B*yVVfsd)m6WG)Xe$_zX-BAH2swfoOd01@6%KnY2 zjFl_M7#_fkDY{qn0xMJ*MWu)F&2K7h;J}yv5{glkk#DWgm@Y^xzL_Vt*SZ;jv(H>p zd1Byc@N+OYuPa8{ng&I}1;weZEV5TcwAa-z8Sm@OjJMxfrUfLHYD-dX)$~@xe?uGm z3!QMs^%3FC@f3V*`0OWTZ}2kd(U+a3AD*uJj!n#WwvW{eJV$Lu9%O<@LIY$dKiQ;h zSSpwrTf$>wQxD~*Y6!fQaPhA&@|!of8=vL_MxAb86e(McY$h_MkkVaue9qZ#`GQ?! zm(4$s%~BbTXxokEO;)Z%wlDVHZ@hEw<8WB5Up?>BuJb5{Avd8kTcvlL57K7N6nxzA z8-=jUxL*vV?r=c^2-<5YM?ym51pqQcK+?w=0>q5eW&6Us%6#_p;PH_~a>18#Fgc>#@S*U5(jvP%d7FDkS%EX}4i}y9OPo z<+Rw`LHZuI!Si0ygpb8G!DrU#F}gijxg`G?YH%X;No5}oZV6Ss1g9!3Uup2Ms3nS4 zwGI|(|LytwofaLZIAyw4ke*4!+yz$dWq7W)%kv<`2)l9N2vJ#%NlS}{h zbq}ZdK8cpwG?&t~`n4^9ylhsWub;#V=YhH>InSLQQ_84Gxol|}&{PiHVQFx3Sh)YA z+-~;=+oG$}+EV8P>20VE`5~|n+EcGty6V%8CCJ4`eWT%h9)C7TEDEh)OH*$6HK10i z9&9gqLKvR*S=VD=?3`JrV{>BtE30bXX1#@Fi+8x#NnCqQD%bm9(%bbf^v+8kXz5jr zT3e5=6|H3#xM2&HlTzNRZPuHGI<)J7W5ZL2;bC|?GUnlJW=oO0?)N_DUYk2*RqwC0 znVZd1%CN0SIN?V-^YF^(8&4M{VfRlHB!k}QeJq9=L}B-5`}39)5oO^0!V<1$>sL3C z79O8#jvCu`FK?N&@e9MURNbCuMlGAKUY4k18$zIwim=_J!RX+bc@4!L!A$ zHX(RW=JMFks~7E0U9+T2dI^z%o=NHLsPAwq^IVWs z`}V2<{NpNn9VtlkF09J0Y#YHcn->?rIFDi$Y9CN^=s~cYxY|73 zI-#wgvo>sBmmr+k)p&>Ah38M;3_9Fl`^^UeC(>1GfTY)|ALeLpUFjck}$jHsM21ANhnzI3Z9ky5Y43~OxX&r+w!OOhmw+=LUbHnkd?b9 z*|+Q@789$Q9Q&yed%@Ng&6=vEKtZGchHIB*q~~#ZQ}cESF2#fNM`pWhfzipCGL(IyUoVC zKP`IFKu#tzV(S}U%cJTnQ%r!R2^rWO3f`w=MtwP`D%rO>!XE%yM8jYUf?npeB{ip5 zPWY4OcZqS0+$H;>7kro3HL-byeES7@A!huuz}*kdtsZ+r&rbOlsxLt&smT%qG}NQ!>}f&u|;FE z#-C$-9_fut$29lq{_uGBWutR@-Dt)2T8p{Jq}p>!T!TyV8ro0WABg5sBxY7M-st)Un**B<@*{4?kkWkh z>GCf@51qobqgDaR;g<$=+AG&TQlzmgUi<)DT->5eR|-5?l8Bvi`Tfbxi+{4OQvD6l zySDKM?NX8Iw5?LR?M}T?vG-(^S{TFE1X8HM2K_MorF4~8Ya$j10k%;^x4tL3rFS(G?7^J83z_esku$AE6&Y;5<3r_ChUhSwhJ#`{3R%J|db%W2oznhHi# zYGU<$*ty$EI6=maVo?+&5jk4x1N^zGt3xv5&GW40g*%_?jhA8kbF~$f zVJGv54o-BM_o)^5ctu@eeX_k$9oohCtPfZLOgOx?w;o0KUP~H9{FyIHSMbra#F&(2*F3inGw59AytNE1whE zd>@1;LfzK9f1Z=bjm!}!OB2bs{7quT^^`|8$#+k?&h=i?um;*PkBI!P-~FUO%K&lA zm&uL7nlN62yFp6~%9!5Wgy$kV8Ciks2MHnhbyx=)C$?{IkvQfY8sw)B)vL^XrMl+m z98`t2JU04W$Mg6%Ezf0`v4?^#%@+GzENkq$uqN(~myCVf5CQv^^AuFUfR2k}|3t$b z|By2!e0GCW^F*S;u_8yxkauc2>zT+NE|vvDc8tn=e*goI*B)1RoOo zbrDX{N-Iz;Z3d5l4mQXZr!;LqzJrZ}lEIcVx(0}=EG#UFYa=W+_5vsM!_fau=|=y| z6x2-`?cf6;k)9>jtTgd^C|{{vvym`G+U2h5=ugTpCH$>trH%+2vmMsP(xj6*{#mIi zF*)={=lx7jU`OL7@lcyhG%@efk&%K1Y-3aZZRVF&IG*2I@S4|cX6jg1P-}9R0~%S6 z%bfc^i4_OfBG)v%gWR#CDf`|hQLQpcx7Hj~GCh|zUP&FVOLWSGV=Z1Po5mh5YH&q` zGaUE^O$TwVE)=dIkh>C_kKF=mc4=RN7-J8~$+P5ef)*uPNbnxllV~}LbxAQn`Olu< z@!yBB@ttc~v92_pxUJUfC<7-rRVOV1t};kTMWicn@+k^0C@?1%-d~1swI!H05*5qh zuGhQVA2Z@ewwKyXt*(dfGj30_xPK>(I1G9@AGX3{GybASCQVHl&|U2;-s8{?W3+wg zjHb>|R`7p=@f#KR1EiEV;Ow@G$AywI-S!|NFQb`Z-rt}6tkmpZ@pZ7Irv*x?)LR}| zC*2=&*{#o)gv)erKEDcli|y+#eoBDXbUvd(=Op{GSf%4^Em87_$X>YFV#8 z9oT5U$Z#54=Lr`Kc*Uw+0ilNDy}q0^A=nd-PxaK>T#NnkXdRcyU$YMt?gG0;( zv)hE06eCB=-~{Dy<)anS6m0=K$ALbthB*(T6w~P!cA)QUH$lbG(eb04!iI2y#gw8~ z!WXN5vI+0@$wyLk&0&Hxw)SgDJ1ne9U@6SJd%nEB!Wwb<;v9&Y#0}L6GZj*z;bUoH z0JeX{m>{vD04vVVk`78%z2j}P1?nKdBK(?n&kCYaesZBBGI{kF0k<0-H$52xdx_VV z+yss2E@)l&9>PA}?`uSXu%_dWq$~#5(u(xx)MTfu z>2)NyPd4LzJV`@)9T#w?7bV*;^FN&}xwGIh-TOR)(8h%3Dq?^{uhVt1BYDV9b*80 zK%Xa!hGz?K5JZAwvbDC`Vv9Z?Vub?%AsPA}DVPpse&%jI)2NAF8frP6ct6QC;Ai+5 zYYHw#%XRD0>T3!cZ13zi^-4P^=w|ZDd-FTwHYy)Edccgj-;B~RIMGnO8|L*RqjCMF ztk%*T#xswBh_LW%L>wZT`7wm>lmGMH_{uaBlc`n>M6`qt+~5b+*v0}9niRW7q=4oQ zP_AB#nIL~dL+&Ch(!@{1hUPX@HY43lbYG6U#mAGOA%>@AgLa2)xWPV~+;erB5#Pp0 zofS!|w7f)at8d~*=R$#YJW5%K<{>CUWK3U;uT7(hhPL7Z`|I^^&B`|VSe}lN``1X z&EnyOtmf1n>h>Ft$CVzToQo!4f>lA1b}KMFuUU@TTX;&uM0P&v$?c-!`ii);r zUQy-d?EVn(QngIyuRGD4p*Z6BBZf}!jr1bWQtI(!Zv0`jTG-$e@&fAEqAv(7xndm% za2`7+VzNXHxc1zSfvt}7*iq?Sd+X{*ih0g%4Id$4#qkW62|i~l3LH}=@4!CG_z9D)=VjuYBABm>OU#s*;m%91bQmj7Y)GrtBsx4!F=WAH5jxDX!&(hD% zJD~(I!SROT?7QtrIPf??O(J4%GI_g35K18}>#VmsAUGf+tT@$td!h}0MKa4D4tk6D z`4euJZ@k(Ge0u&p_tO?5{C&I>Zb`3c=W5!Hz*pPK;}-sYS{f}4?z`xZxu^lte(FK~ z3MVD$e9$N$Y|JfuhC{m=uAltShT`Nf0z;f`Wzd?01lbXNrPgSxN`|^CMP~v~7T0(` z!y@rp;J`>BBDtfwKB|%7Kr?_3hb4*_Z8xxgcU@?pun-v6a)bFy%a$lSuQRkyY;KXDAg^-<#%9za;J6FKZy%Rg!n;f zd__%eueRSU=gncL&)Lt%0wvaD?~#Rx+Bl7ieGe^5=Bzw8jG){KCe}e>-xPn z$_&{h0W8NQGP+uO=)7&bZXNV48pm`-3@VKEanrrOG<0l0jKDf}i;P9Zlc|4`t76Ct z&fTr_ny+#DVxM5|kPBK+j)!aJP$!=4U`e~j<)dW@W0M`3rN&qGL#X8wxaxy;oYi3{3NoySnl3!PG5P_)Q7 z6sL-0{4r0(wfGEl@kCjq|7m$&nO*;J80?|D$ntBM`dl$%?#2kVKedIgc|XX> z&#*W%!Zs0h)Epu1$b4c= z8gy|%)d?5`jYQ2GsnGAJ24E*)MT+Cgl~n~HsrXoxSh;wi2!H3`pd(q^G>JRJ1}>Ik zS_UhTA=n*>!a2X*Zi{bUFK(?btoFs2>gIba{T$sq~;*@CjdUO^S1u9*ok2 zZz;w2Xfa1cC7w!-#A`vA&!gKnT3zoKwP=T-LAPJ_m`GT}mVAD4D=x_Y12<2IO=Uqw zpFI?Pp5UXn(GxzH-nvutMBlvxV2{AN8$o ze3MMTW*bza32rgb1QxLGMqfSMsunop4@&pc1-hu}NqF){#myO_wUcSnQPM|uiFH+b zYh%P=XseR(C9!%2Cy}GwA{d%^R7I6@opX5{KXQCY}XetYk_)&UJ27>pCbxC6mPgCtf{0!)&*e z_%6mY1RbIcQgAS!9(7;S>YRr7`~3FDy8$UP)M0y}&Z6RGqE>z`mPIV5TSo<(<(*lT z&#jd+TOn^`rPT6FF^`yidL0x`$tK^o5@riyJ)vSI1Ng#X6d!J~M+$5}LMlWBM3jp) zSbaeGQ9Q~&V0>eT#}*Q-xzjpfh>&1l*!-*>jObLYT}H&bcu7!;%aey3d*S^x=ChPN z+sPzMz!I}wHyDJZDZH5y#`RWrh#jrzqZKgg~vH~pp5gAgBT_(hwyZE!-Op&w? zxnaJUrZx2)4l%EyCV>`{M*@+Ce?W=;=VjlF|`SB_h?CVbDDv~)gh`IbRFL&7N(R!N;w%fJ}(g>0=- zd_3+)qvF-JoP6Yw)vDDC(+O$C;iO(=Vl%U!{YL87t7vBM`U#HjI22rs{b)+i&yqIUiKf;O>;jXu9bk~vTKqy4eb#pqbrHk2Nz)kS=oKhv!$l0SThRKdw#e-`dwykEi%lC_Jzk^{+96hmVDB2|-me?vZm@m6-+w!MN$dxC8o+a=Xx<+^%fl%LYyli`jutOyG#e=%)F&kif4X;bEMQ zv~fFxjGpO;C>`2QU(Du>hF-5797Ckc&K9H;OKYP{oj0JK{U#f3D(g^M+9LgC4G-ZoMZMlZ4}dKAzH-~XA+;` zU9;dH>3rm!e~w8bvgF_qZ8Am12!&PnAdA6McQ>V$aKA_6@7Q-#4w zS|=z9(=@b<%XR9?d6cha`jcq!>o0E3b=U7?M~F$~T-Ku{FxD0!3yq;{gQ{=Fc zs*b()I__M#-WfL33~h_;D}D4Pf*5DE>Ft-;cPn%NgO!mRP=xzCZN@nKtFbN4az6kpLL*fE}tH_m) z)6Xhc*N@nETCB3CcItARim><^>KQh4eU>`4(ZY)ZlgJlA*_fwD0c-(SEyuBqPHF?N zAqzZTa=#N^MOoCjcC1qkn$>HlmeK#SR8IU+`DHl)@5`xtWSdff3OHS{%@h@{LwCt( z!s((z3m!_-erx5+{?2nFuMzavkkb1Sfkgo_Wb&sA^udO;A zAG(oj)xpL`5g}n?cTVAX=zssVRz69`K3yRHNET182=0R4c0&Zh8qKJuM59oFKDqdN z3I59Apc}8xV5lKCKV+;3n`4ml)(-rn02&kZDKkOqmk^Wwpb)>!*RLD48+B8q^sYMv zLu~<*tYaSad-2)#0y6>fEh=*H!p=y}nZ=fDU~G_}G4#Tz0kaW#!Gy*ll^f?to@`E- zCxTdgMZv`iF4d`DM}$EW4FZH)`t&M^=YLpypmK$Y(eMfB4ehCRr_Z zlAr~k;$w^zY=tKfZnB*GMirE#A$&x&`NZ61zkr{`Kf+M>frD)40?)wna;y{`!R3*Ze+9Lsk(m6gj@HT)xqa+A^#jc}oDMs}|Fvr@4rS*&uuU19A_! zO-zGOsdbTzEBGp0Xa_~KdhufWgPQQyZ^?Cq_PehH((PZbSk2GY(4aJ6e0)w+o%{I? zAe(FkbA>Zmc@DHC$B7N#nDz!_I!s?K(6;w_hh{r^!j4(;!p360q^v&Zf(!+&C+jW2 zXc7rK#T4v1easbMHPYhJP*dp;p{Z?#4{`dHE36EY$!HvT>dRTLZQlp_541xBKHQPs z2YhW$vDA@_w~Czva{Q2O7Z=kfEKRETBB#MBnp&4UZ%$1UTIw5H&S?>0&msmaoO_)O zzrIzry&5dM9Nm3O5_s|reVmwCSCkeo8B!waavhv7|; zkP-WhKnzcYS{DnY(ku%~V%uCrQR{fET!S9@)y)C=fvNYtW!x`Q`p0v3hCAuVkwQZl ziUV`V0UkPzO9}S!>4{~rn;KDU#mRitQiQkMcRfdL+U5L=f$1SKO$N&Sdu6O94n9YaU$9*C(_>|i(@-MfKIiq93adyOIpZ2;|yVi#? zhh;UMaS-#p+KTT(kIx=f@Za%{zO4E+y56Nl!s66_o3@$4-hb0O`#9D^Dj7uYbwba& z_Bsf4*D>(DCpV%;;iMa;S~iWbW{Rc|CXHgmV&G+baDr0BF}?=bHX7I<|6%F&Q02w! z6_<=U+FT{9e=HTX>bx%q>WK|kYhkk$B6qH%Wh=msWc=j=MAk&QhNK{CB*;G5Jy+`l z2H!x`hdm31sLExu{m=(tAL5--b4b=|=?Or-(xx&>q6&)$H~0!%h~EXF`ZlNoCc>G! z-i5-u`+k$v3p1Ei@tJCVvdLFKYAbs+NZcM;N+DlWJU(=jI1`E3p8l}{8qz?^{t4vK zqB9gv%o&PrZxaiO%EKO(V{vI_VSo$=i+K>KLSV3t|x(zt$ zCLH(-yO$G>kyvfW7^N*3T-G5HQ&h+*qP-sc;Um-xnY&?T>E94=a4QZUKik%x*9W_c zVhA%BasM=}lzoWi3*nFvh^bdu;FsS2ymcAo7kSUx#QXSvQTCQWaYaqPZ}0%YgL`my zx8UyX?hMZ09^BnMxWnM??(XjH7U1T2&%L+md^t}}?N3v+cGdJ=)~naQe{FZC^_afg zYPl8-5O!cY7O?<~41+*V;d4cT-&MS^Nl95SSL=_S*}fJ@>ezQ68uL5#C_d4@%97Fl z=x}DldTP)n_|W6>eePu0Wz_1h9831ki~A092#x#tiqu!ybgi(5v!t}FARSC>>2ZL2 z>0xu(_o7sIRU&!7(Dhllto<6_bpCd(L^(T|3!D5+{7BFH=^T5GJPNl7VEy!3xfX0} zzMNPb75T-c+zhMhtY5h6_I$5Hcp(n7e3fkHo%hz0^fZyqUeH!;=W_Lvje{yNDtRX; zhlqXUkJfrDe2sIkzm^#m<|HpW{WyUJdk*JCD&bVd7Jg*)!RjK6GDjuW$41ko%bW)> z<}X8~(zY;toZGc;jq%acl{4D{zNZ@@DIz|>#3uDd4eb}t1?+syZj8fAnYYrlPMcc- zjGl*j-VpEQtuQbfkKCvROpMXk81H%@3&!m8jJr+FH}e>NfmmDtEX-`&$ElY?zWS%N za%-KT#>>s5GY<7kAvW9c;8^82K3aO=;{`xloKuwDFVd7GMOEG;5#vN-LR(UXw`WEF zvmOqw*h-(C45W{Py|LHbXcUenPyLT-!%eZZ zOONVy+U;f4(=T(P)Y1@hFeN<%)=$`Vzjso4y?D|F9-IMPEBzl|}2vv;QW`KREYkg3d< z!mEvrPQB9R7Qt%X@8oMshUV+6a`3U$e_jpRls)e{(fB1jst^dy98f4W%wFd!GKA|Eyo(H22LQlMu1nPd{`EBEYpQO(? z6puz|qLLFdm4x5ak%%!dGvJJK4lOdV3G9NqZt~y<%PGIMU^c&dG>}yhBe3O&0jN-| z^Be}DwgNxi2c)Ooi}gRM+SE!tUcyayi*Ye4Z}2$+MvpOwW!e;MMVV7(^6Y-J-pTXY zPVmX^U90ZzMU#G{`OIe{Rw{Pt1RkF?^0#zO>90kM&(HfC zeTB};4*OQhGU9IZN^|AD_wecAS&b`*P}wq28P^6FFk9-|?CHb|xu8cZ76q0o!-+d8 z(j>~Y#9Ed$pW~X?qg&-&OVitrFcG?+jSC}Yzbp{0*A_4Xf zldwdH*Xms5;Y#t^+q>cf23V$Gp)`#n7wz@gWdXMWV{9whgl)MdFk8z z@^ki?oE-Ri>>&2z>6fi=tZpde*RBRq)W)}3y_c=~9_NI8#^%mmCt7_ZM;>`Zx{&Y$ zq$+@jcG7Yxa^msJcYRiyNuuB#*fgV)4OJ#{I8pNjaFZ=8!dbN-8`Bhkc05E)#ApZX88YKs5OrhR8+v-ANR>K`1aWntL=q_<680KS%U z;x6>bHN<1njMSSBog0sL`PfhLNeA0@?zyf(H>T%K?=_v3c0`8=t0!m?S0&0^^aq;j z08g2auea$*rFbXaZ&&{AzrF4;4Cf5a!C`1^!)dH9q_pGo8hM=)CPl@j$flHJVF%;xKNZq+W%P4Ua64$$F#xmyB0gI-*!zvYwqEi1oX#ed6Kb^ja9UKh{VlJ&y}cIJLJcYL95z+W z@3oH~BmbvI7wP_uPQe~+zmr_DM|Q*h_dw}XHv8q>RSR-d+yfN~#bJKf;qs?TkLp%V zo1=kv$9i<1UEIs*61AY#itkma&uNuT2mQm{fLk-k#V_wnzV&UgRm#nW0JipB4Nc54 z$6{l-Op8N3myZU4*LSqDCN(RwVVZBnLCa$qM3|zg=aYPN`k&{Xo507;c&Z3`iPM1^ zWM;$m?bMNF!nd-Bzv?=kR zH{_TOLsD{H2xF>0Pp$|Em=vn-8X&qOlb67$Mv!s9xgXTnSGK4#myW=8&$GUjIWWl^l&{2%iy9-Dm#d*%HqO zd!Yt=k|oImMs#xae5a@}!rVo#)vPJ`3`#m18@R0vxpX?0u}llgk{}cFoMGB1b8yhQ z2z~z1pzv%zl-wmD6NC~OeQxm8XL_PWpOyCKsgrZxq3vD)p6B&$0K_8%k9DN`5{FO2 z3ZY$qk+k>F)aNUnvrgyV`o$(t@yuIp`xRWu-++4ql*?S)FxcNGIrGs9?7G4m0=iSv z!U;oqjFv3$Mzvk{yVdayUv-+?O+LaeKRm2k-1hxhkhZq6irlM^`9-CYI&wv|D>)re zFVdHJo11+H(kHS22g=cDJ|bZKh1wGxG7g9r#=X3OHTMJY?=4$7V`hmUNtejuIlB$;LfPbaGAC@)E+Z+{_2WD?RCu-Od|P{Y$Ln_`SX2M7PE4^sP6 z@>AWBJ(&rK0ZXgV13x-&)6Qxw4n~%*q|czsRwi6xrqz~dv2~gE#_0afs0)N}-~kb* zNmrUUz8-gP0t%Bytd!KHpTiQa(-Vut3jx=~GB(Wj-_$Xp-)QQ-#d=jB4eK|_pHk>- zRH*Cg>*p!>LGzl9e$P0esl=^iCA6{7>cjn)0Au$53NXTg%r}HImEV!9k|KPYb5(c1 zv~u~DFvyVgFQE9~8y+F@P)0N1!ebu|M532O#Y4TOkL>M$g3=|X#}P=*oJf_qSB7^+ zXt@*~JAsbFZWfl_!mcGXSW8mD$I&s)^EAx#t=+;p$6tLF;N9Km_BYFYS~?iHWMtKs zozAxOKq;q1HG%9{8y*2|Uz49pd8LiZ=Z#M70OeqVt{yf}0wbxYfEM-5pczL^Bc{yH~%cmuw zAuPNiv1D0&@!nBQkeQ=R-w{Hm=M0p`b}^>^)7w30OShc-b%@)pPn^@9g`K`#hy|Bn zb(;jtY+NA-9A)Vyh`r{Et%aX7oy1lx!D_7m+Qkeip|1MM4-qbYAX#?+A}L)SOoTc+ zocO&?$(4m=M{n*fm(|UTiWLvYqvxNxPah8~RJ`pmVDk}9RLX?qtBo3aU~^%5TDW>^ zU%ko3Zh#C8rq;@1YyQB*ZsBTwMnCW>2R4-v2dGKm;=L$Ky$m`uS0YWEdT{QlIQ`;Bk3c?W&-o zeIcS)|7a@X&GOGYF-0T_6vV3Fh6*DY{o1OLnG#ggMKa6JI{rrP3?sLsLm5A(h`2Dg zmpj}mZqO?siLb^Q-!^E#$HBPrHLUz9btXB(X?s(bE~pc~qyBN@DnXo>LXCdBcX_}{ z9OPt!J4_v~L>Gw5?Y$kCm7JG1O2)rN0VqJQ8Fc@lC>)KRod(rU7FCbp%*op(5iI7w z4#SYagMMXtIlUTPc!X5&j;;m5N++L=d4?Aag42i%1)c@BXJ)6c{R2 zdF}-&&W||3CMi=Qn#vtPNEYk~E6Wl)%IFe{(ZWZ2-4k+I#b_CEbhx}cEAb(~U-B6e za316(7&``Hx(J|ljPK9+`#p1Y#Tn&~^>t?r7SXh)X6*jBvKk$6ubc7J7 zB=KKV8{hXA1qI{taq4V-PO0 zRi!mr2L!Ghb4lVCr`g*URj0JCCA#;?oyH&Cb!l}Enk@J#H<3`*x=G|E0oq?j6tmri zYU)<^F(F>*xcdI-xc4Chdrr%I0sd8utRYw&s?LTel3z?$*d>|%h2njBP|4WaR>~`C zkb1XNghOlAhLpNGGsoIR5@66#OHvlTXg{?&71>?plb*~RGQ%0ZH-C3(xUU_C!l&}f zX4$ft_$q%zbLxhzQSi|wWW>V+ahahUkK_KlqRdcB-_WRHKrHK>K-(BrQj(;s02EDp&yJ?pMF#`A=yn8r21h=MYWJ2jMINZfs^ozRw$kcV+Va7Asq>X#EF^h$ zpA!;XwJWb4S6rV04ccLttofqFq8|(9!2Z=#dTIjeqw&IX_=&%Tu><6cY1c&Fn*v9h znL&R#!3u^u6nS3JLS{DqDVOLOo{W+F#eNBS`nz(*FY1*jvy!zTOda2wZQR?0NmT*HOtv2?BhQp#ZN+6MB{y<4c@sI#O!s(I&)f8=qrLQf&0fELD)Rb|s5$$aXx#ud8kN)eP;@CW z>p5Vo5Vp!jcLE7aBYS&8kfu*@?|iT<#Xy;!rsN~k6!F~YcMcQjZzL=M&`}_RHIxK! z0mkU!my#%VL1~^wS=|Y-I3V{gpNJzYN?bHKDGUajN%DJAG^;XPcF%VG{yojs|eq z`bM0JgJ5zYZQfeI012;qucq|5+~jE)Nf$g8fE@Y(RhQ6+^^{cxG#nsW8MU;zGfk7= zR~Zzn+ddMHp#~&kB2O+%%R+P?6Hfw+md*naJ@<)uW{@3wjZ`|QR=47B zRb)$lJaoX7*#*sa z5gxkcvHlC^sOTRc(Z=r)3~efD1OV6$JUt@Pq<7QC8Z?J=4ml<^K=ez$Wj#F~KutBU zx3?yTc7FUCiz6``qfd_9zi(>B2>T2@n}(soa@i5ctQHI!hcm3rgE2QQLoZDt`<2{` zGbaDG+>E<1@400*VKTKp_^hUbCoK7A{>!|odmMh4OJW7IzGnMk5p@k^PsRmF z6aXC#A34^jA$Ra5RLiE2*0fQCHm2h@!PR1Da5$c|Wrfvis*&Wp?dk}Qmhh@Xoq8>k zBXE-bTTyJJuxPmLv)^yN1a>=d8!>|+V(saIG_v&$4ed-N_!5`KLO9w~1G|1W%PqZfwguM0_*;@%jo7nwjc9w? zA?bHPC>f8;l54NLgJ=$s|D?IsNA=gu-k@O|I#yXJFNzaFQ?`^ZtTw}?QsF`%(_}8R za4mR1@Rn|iIfN{WO%ho0C)Rx|AP0~FX!=3k_*y(km|zQgwqI3tx`=Vgs68|$i}3!w+d zuq7t;unBdfpsS1p*8(B~)mAe~ey0azjuc|30GDiES-{U4t!}W2YH@+=7G_mUMc(Kh zw|3(i1u>f9)(;1jI8{=Wy@7WHBw>p*bT(VGg&G+>G>M-lVB#T%X#aAG_;dzuqiK8c zVF78?c9)*Po3O&mO=kLWBq^q zZy?KG!`rYqL;ht!{~yEEL%tkrs)vwN{#WSG|J$J3mi#YVj|1Y!{$CBhK1I%5e>=7R zjsD+B4*!e3w> z<$ZB#|2jqtIM8y}+cNx}IRDjfH^uqzI_qjyKDc zGly~k@fw6P>j|q8TaPv(nyH~8MkzGOHMl>2MS0KJCG<;GXzu?ecfrZUX;cUpo2{SP zK1>slLRF{v?XN?r1pkQgq>|o}#=@-3&1Ix%*6#tKJj=1L_2XFM_SM(baB{k@VNwI$ zm|M&4g3{RcBv&riI36LKKFtnQ)5_}sym9Wk|0%K+`n%V6$q_686cQ;^vXeqKLu0R< zX)Gn^UM;jbN}=shNa3FOGnS5m* zNvpt6gxPt%(R&#D14eeGD)HFQvxby;_7$xMS99D6sq&~EwK*c3{6@nOVBkv;=HuT( z8IMT)kH1PA{}10SqRO6t*Mj=?yY59ap8k@l(}`tuY*CDnk{lewS2h*-n=VD3)q2Hz zy;Vh(>4j(Uj1{7GCNL}Qb>QP-qbJ`wOOT%5a3SmYUh3Nz1fKE(w~x(uMRUpAb@(UQ z=_mbfIIv4r!Bt-4{HaycN}?-i;k?CmFO*b9rk2^u#CuJt>M_zkVgQS@>w$#KVYU=;Yr!NktT`?eIC12w;oM0h9a=Op zg(;_5u?@d*yc3pZM8x)}*|Fi(0DCeQIMRBKf^2I_QI?A&+$C(a7Do{Ji_x#S1D{zL z4KW190vG#J?dwb0;i;0w4W@|UhoQKL%$bw;TWEi8Q zoWeAOmbA0=8b7+Y=Nbs7rvs!?sD9_(V4bb(ftK#Zx3iv~{X0x)ShdIwH4 zet`e_1@dB|&BdC987w6|pn|}RguE^RXne1zTz#XAVLBI)wMmz0Lu34_1?}1_W$9V) zP)I~y^9#4bX^aaxTGQNmhfHKjGB-(`&~*<3>^{y|_eW6oDvi++y>D4B&Q8{UUv%}* z!K;^B?p14BwWCT13P_??c-b7%xeC%IVRi-@LBtXHmi6q4aThAI;Zcoo+~vy`ixR*I z19s#nX>gko)q-^NG8|7Tcz9k8Tro;N8#kd5(WvP^;F{l_@{W2*MOp;Hjy*z5 zb6e(_-{dnVE&QlnF-qp=Z`H`^yb(}d9#O-23`se3r26?b{11s4?H-U~g+=O9hnEeI zFNMUMr6qF;;}xSk)J?f@m4q%+K)`)4eg=36T$D2?XXerKb}lFNn5mi7zJ2V0S(p0k zILy&<4(Q6xMm+&uBUS0R#BOE(Z9n+l)D+0GI$0mzGF?(v+85RI*=EPHd7N+YtqdZH z&9G0LjbK0fV#H=)cSaAq-`21)Ye-E3EZBOG$BNIGvUAx`to)kQs$i!=#W*!3Z>x09 z`pxK*bsk3K_iVF^J4F*v51U7|W0ApF7sB_YAw&+(3_qr~lSwgZrN?t`R4uffT?$|= znpadAMC-rYs4mQ$oE{eiiIas3e3AWc^X@0OFT1~(&}t6gxPi?8O2j6ld^cQz;h7>_ z4p{mlVZloAHM|@Dn28$Pv@G(1c~PQonc#s>%y1U8I>^%r2JgC08$B_d1h9*1x5!|F zwQ7?g7JgO}k=MN_$puKCBstV$cdtSrs0ag8!oUFLJerxnZZp@lLACd$%Tvqx5Y^R+ zaq?)>4+*|RE-F}&OI+EbBpi3u9Nf)i83$6b0xWOjhJ!$-h?&(uMV0Nqv0zeHT*qLI zv%21_;q7ycHPP1ahNZQIYbnW9EC;*iicK?8S8C6VUxvEDJoZRlE>*abl0>hcU>kia zA6zCVD=o23>83t$+Yya!H>??aZ)|}GoN*u*fw#&p>3K=x#_SlKxXsh;Co+8yJ_-Mg z4jJ2>BfX7Yc9dg%&348x8mTj_30(A0Mk_ZXJB>`z(<^E{E)Tco0X#+=s!gMmdp;*w zDikP&{%Dw6mAg0rc(pVek_dTu1qGI3lGKXrsEdpte#xkFl3xZ&__MGeczD{ zsy*y05wH^1*WWp^DEM3|9we%<>xdyCyCZh;d9#&^gHdo+$zt{vbm+@)?YOK zOMm+L`!`eYa-YE4szti$u4>sup}$GBTVu4P>Y|5CeZ|<>u{8xD+qjv@Yx2o~ENwN( z(ol0gP34U(M9tG+|4GwiVir--9ZDi~XJdQbF2^{l@2f!*C1Idwv71c#+h^uy2CMPW ze^pF(pZRrXGdu~UIqkcts2a1d>@QAbvV3AOLnMrnBb7By&nUjorr9B#k!xIdG@fwKVrLO*|p!!lT{&R%FP6oaN z9HsS3dQ#{2UlWCD%4M)I;VE^vOeHi%-Jm(5phrnUqxND+NL4mTG%hyGKl(t!{@|S3 z>#Y{&$syoRXH=q(FtRj9n4@a7NNiSe6+Ya!Myqx$%Qce@F&Mk&64na3&;h>ya#VtQ z<5aswOam>F;-;e9<6chwY+x*v1v`56wLGa=EmNaTxc=T)wiG-PU#2*XFYA}c(Cyj~ z3Zm<8@$EMjyPS`_BTqiMx@x;YC{2+4{~rsWE$v5?j!s%Cv^l5kr-0uUhr#4JYpuC- z!TJKBZ!TRC>W(z#AB&&mWIAnf=kc}HSi$#KG;1&f2&^u+#%D0TNXJjD5Wn-T{~Lw! zGo<4wx~ZJ<>C=2cS6+aNiWK?lFJz*>Pq+$7DR@B1SGD_9Px0!9)ebnFHb+I2#2=!q zL>;cEs7ijZKCfJ!Il5}^ojICsknQUF0xcZRL0himJZ%+h)`w|Rim}gO9m=c7Mn#6J1`E)#BesZZ;(dR1|-u^uQWAsB=S?1#Jn2S)$Wn-E@MOpnC;`& zYo7+CCnzSr!d>&?l~{G=1&HD-b77Bzd|Qu_qZVprj_EOwHoS|KpEfYo%7eF7-?=Sk zCst>_$Z-5#OW+X65V4dI1OMD>8Z=uWi-%Z|if3I&T&l~vv$P`iW!KCsv_bJ2!}NX@ z?UJE;y#^{w`o4@Ie2dvR=x=U6npPK2Sut_!FRoBOV~=2%vM1A;Y^wl zi#H1W#7N?&_>L{?4iqK$RnldcdPpXrKtE;WXjaD%J;OHvIFV?2taF11v`WQ?jgxDR z03U-$j=2*p4N$@#rDx<8Z(nCsh3M9>R0qVs(V@z zIpx^YzasWHoaf4?1k`lYJZWz{Py{2l?7EhkPE>x*9lLnfIR9;X7DJzz0aW#yWy^}e zlv+>TKZA&g+Yke?b|99yr!7|xR)kU0ddNi3NOJ}^_f{1rlXsE(;q#@3%`xm7BfM7C z@@#12U|d|bZy#Tb>PGldaFEkVQL&!ubT5?dlzyz>@kn{BI8I#r2{}TO7{UO2Whw=% zSRGq(NuhlC&|-x)2Pfq0<@x!#r`c8&&yj~sl6skT39;-KF#lK!J5kAFvt+$hYc5|G zafS>IgN9tC0JDZ+?wX!LV=3?BCD)Cc`@?<*X~p;Fmo@KG&;b5!tQq0)k*Tlyw}3YN zs|MO`Hn81W@h^6+ssh(NIbHcm7wu;y*v4J!h(dKK0EL(e0*xkjrxDjDT}Hbd8#d9l z?%MXZaih#@5O~6Hx-B&u??~{0U1$aHW}Ict2ze2KdBh{HU_W`_{x)_<#PA=w_kTLS zn?}c9o%eF$OXf9t9_@F7C?a$sW1EN@M3Azw)d=}R1dW}y72vbxXrL6|{lILu8{3=b z{Vv}8ZV6gVpgp*Oc=y!$HV8R}1fENY)yv(uEinOI4dYuTtH(NIlu=f!%Iikpe}eW)mebvm4eB z#gjvD%3vF@)np^Te*HXGjNj$vqd0vsR9j+?(;(NfZ)r@?pohtEoZJ`nOYRUDb0BxyuH-}S|)P0)i^{qENLzZTu zW%3ub#!Ix8)1SJx92pT5-v0C51K~wA@XgL+x^V99&~-3ayKfMUzC5%;^>HlU&oC2His#LKAlBX5F@* zMpY(XFw)z}$}n^QT;UWvuR{V2P-&-{fjD&|44l+)Mx)NZ5BjAvzFe!N9nD7t(CKg) zHfd>NJ(=V>-oOt|JpTsMXbzHgt5h=2@tuO0LS$X{+J9P3#@yL(Ss~E)F3m^tm2atD zY|eR(<)O>#L4N3}eylhqBXFXLv^0z4D}z-5TNV*1I&PYvo)D-!_c~rUe)yv=j5+YfKV+V=PQL1)bLcg>T zLn>u(QX^=PCPbP=EpjNUCOIY$fPu znwoitw#Cv+A0^zA2WuH)==IY{&eEqm31yv1RyvT5yHq>Ym;pJ`YMMOkhpz@px2qL?>-=b`lb+a|ltf&$noIlb3vypX%Bm8o**Rt8FZBr%+yMb~I^quB9 zJOqYNgNOalMeW=ue_a^kqT`L#uS-2X174F=KTcVa`M1=k*w4sW zhCb{Mp5C;+f1mq8-`!f$VD&*ZI8?oqJ@g4RD4>FQWJ$FvtE6F%{Cl3>A_!S?Krrn!&Ul}V^~o! zc2JPe2rxf=L@3YRl-UZ?)eNtt%5EU6+160?T7owN>pNX^Ou&zQ?3IH8R)1Z@iYwA(8y>9CF6^60_ut5F1cj!@~6_DufUdJkS=yzD>sKux(GBkB~Dkh4L+sIMzIwDE1Z1irGnKZGS<#HOWX z;B${L-Zd`RE;+nU+A34vgRZCfny3!vCt@PAd^fVJl{`6b=$42Pp2NkSH>|V&jMm<* z9A%Kn!<{7n>j*LzUW$*X_iQ}}s5b7LKV@mohOSg{P0iAR2KoH&w2cowdq`8I|F=Eh z|Lpz!(S_2xX>MYx!|dkj6tcYXZS~nd=UI09*hKX!bzujwtdotyjo^Eu~0!U!;i^Tbo;L-AFSkxq(}C?^jqwYuDUc*YsXOeBzhM<*w~OSq_y?)fzO8^ zaG?@Q2(c&iz#@D&ew~aX*pG|Mt#4)7;Y+rX(Rvn@)NYrW#RT5-h;^3MVvnIZq#A_}AjNuaY7p@hRNM2VL$V7gtgE@qP zum8nEv}L|7k9Zb9#E63unlKGlwl2vdG-YCJDMsq-?H@v^JS^@e5@3fwS= zqwq=&P>q8tvs-7J6xgn57UKgos!wM|2Dw~jd!yem2|Q1qu1X>0B*d&K^d)yg)B)l1 zUK@=%%Z8=$9T$EXT^`;84uSL85{{TUjsw;LPvnar;Sc)`9yaO7a&34;SF6qcn_j;{ zf2P>p{MK=zz^3BqFU-1UZe%;&*buB{@e59vq@RgG6WkWpsh zIf-8TIvwO{W19p2SKitmLtexlGPT@rv{!t&mM562a>OmI&E0_ea1BMhv8T|T;id-_ zI7(2M>CnG9EYHsv3ik!1bQ3c8<;TYNtGIB+D^r0H9T@;1lgt3j&2m(%$~IVlY>IKm z>bRavxi&MtymU;HPv+$}2jWcX{iNMrsh3IuZ-4DMHg%-bsfnm{fG;*X4U7`5G_g9| zS?_Y$TN0hyie#cbGjJQ9G8yRv7fbBzC!-}2HsDLiFKW-)QTB#*X||2XALR@bBB&>0 zh2Bhz@s(=5wGN}Gcw0B+TFEAoq#b3PA4FwMlWk9 z1)YXtX_9)=x2rd;{_)Rw41?y59fyfa9m93#()*grgFEhgDT|6-IQ2Q93`|<~%I#d0 z3x3e}SBP!>QD$Qn(=>G|I_QkArW0m-2w1j8=C{i7>U*;CxLwnU_H4aNXWu7m+}HhC z^sKjZd8hJ?d-EGaRW&Q=+E_MiH3!E@#(rGLzfY-bY@f0gj}5g^O$3r_zN2+)18X+DyOb zl&Qjsl^09KU~5>{-{s|PS5tFrTYOv@+nX7^R6=@JX@qOg5jWDP-zP%kkzeJlSDHC9 zilBJR3J$8f9Bz($d{)eSrTVO_xwCMCIizz?e;!kXKyE*0#ON~Kl?^SaoJR+5JZhZl zbxLNj%F3Gdn7S5f?0>@g+QF`V|L2472cgn7*$@);IEp9>VzJ=%?{4fgG?Cw34p`$t zX}`O`WADKboVB)|W5w#zy->)-=wwr`uO-8`b9CG=v0WQ#kp`D7BF^miF8`|UsP_6i zJ0;)~HsD%Ou1WdTcK96eDntc@AP2x~mRwYFNi2{nFn^2W*_E{-^OE&b2Fei8A_|ZA zCDcr{9)K&q^v<+{s3p?Ih;>AW1dn&nwB>gARZ)h=ptayRpbVVQVM@fHI-g215!r=h z4Y=$J@wMWkFl<9660u6C1g)Z$N!0FSEd_rqR+BJTm>+t=2=B-Bg}0?)2rH3BPYN0_$F9}AnM%+f z4>lEzbk4SxFDeHuso?DNw^Z8p{jMgW@HJfv94CzRoWss0^o4Z9cMQZVA#svg7a?%} zYGyAKee+8M{M#3gefdSSc-?!X4wlC}4Zb>ONjOgD?SwE~w$`lU%&9DXS?2ygrX*=@A>QDQ-FhZ^X(rL)S^&oa$?b>lx8FeV}VW#@h z6f!~srJSJt2rMjSPoz2Rrt75<$yrbmp$s1yB<4XDd;LQn+5FP1rn_jbZI}#WvqsAS zSO{aoSu&i>NpF4K5o~jY4n>=}DV}rLXk9+-BJP5lfgk!AO~&GvnSQv4WHzprUQGgf z<6Bc{Qg8#)PVB_-0#-40JDna{!p_ls(|pC2XaseE)*g=nlgvbts@oH$tKlC{B~jEW z$SY|^?mqt2?c@;_2@Q8|xY-uB?S51FG3T6KeXv8b(eokK!h-2=zoYg6jp?c zbdMMR^YK8;zp~&<>~0}lItAP|nv)s)y-BL5F&_64q)1m7u6#bqYP=wUm7H2eMJX}4 ztiKf`t^@Hjssu#oK!6m-6#3~R`nyZh-0gLoRuJWS16CIxhgptaX%eny2J;i$eZ)-@ zx|`XP&;@jWtO8wEW?}GhME1Z#p#PVPJZw-nNMBSlNOduwYbNVCvHJouiF^s=3y!2o zb*cSm=z=A?p{8E_%~re7ELY35!eSl}NB9gppe z;M^~4h@frvrYhyG&ezMMU@>wD=(A-2m+!~;<`O~cb<~{cS7f>4^lqt&Kuaz4MPGha zx2Liwzha;|e%;zRgsUZq@B_N9&*3dacGce-jY!m!`N>I9C-W}b&kLZndWXH{%@7>3 zh!l4rr?wjvwYA)!=J#!t;Es}Ig+)K5n3&SDEA6}Gx&>UCD9-3RsOstt`|AKM3cBo; z0=gCNG%P<23r*>e`GXkVyZ)cv-ojDt_>?AVwz2)?L%m4vhNet_zoOG4_6@rg*T;&? zTMBbr#t^CVXU`F4m!S@%%5_GaArgcO`UBu<(b=y+AdgA1{g9HiunL%E<+8sIqjcf2 zp6kf3*3-!&0*zX;JBuJ9=}tb^MNlBdRmYMXdSnz3s)W=G(mP5kacF z#g=9!vMR581aTMB^74TXjkF4eL;XMX72HH;Ri+IVZ=F2ZxDgjd=LdcBp0@4210_bf zN?-XQS%pqJR-UD6vYK`8f?McGD8I~3o9lE1C#4oz$I(3=F zk(INRr{$c-`v`KMmge;vz;tir9EUHR5VO^rZKGOurjU5s3}a0fbsgmdjAk+(&&`~V z1UT)d$|l@ei}x3HoQeq>r7=6*a~6JO;4s3?ES`m6>tL5!UlG2Hp-HnxsiX^@ugKZ8 zfGjr8jWO&vnv^dn7Y%Lkajz>@n#Y+#6UD)@<>sN5c;A*!8wwVzkj2BYxT9IfuJAd1 z-lwc3R(($$hHlL({#?si?NBIQ?xmRQ7O(cNRMyj8@nU6hcaudNr*+ z4C_<)j0P4G;*jl9Mb_Qd?xk`9?3WqBs%7Td|VB`yPTkyQBZ!isgOxD4V>h{=<>>RTbCY`s)mX`a@)B z4%1zEo#zzl_%*sO@OA=tGMRZi5z@0ujch#n- zk-yipM(^X_Ce%0TdeU~ATfEl2u1n3;@$b^%t6RxI?NL6}j%#REGz~4NRX+9)53U=& zyZmVWv$MKlB-rX)r&7UGA>WhLuk_V>nlF?;OTWWGr>&Z+b@Uz!k!*scGAtoE;W4y zE;}ZdHhup98>uBrR- zGoSNFZ-S56Bt<3xg5cUcdD6)TR{{RdR4AQc?v|S#wyv)CIG3OkAH2krYN)MA-wQbq zxvh%kPW%M!*lo;*{ljcYTezeORJFsmL1rV_@`6gJEC9Zh}x#bn{i zwz&|tm}6%K2H9*u>vNbzed}r2q0RO)jpKX+mj2)o#c1RQD%zv`r}G+gHBU`@JGwxf z$KC+`01QmVZWPc%6E}l8$(m3LIGPQ^Tt=wxedf0WQ%1)omcndCko7b$7U@UE4{#U3*OVF6%5vgEfU}-kN<* z*N3@6g@8pGvyLEUKQ{gZN$$O zcabJO!|0zQ{EI4oo!1VWUW1qfliu`!P>Lo|I||||do(H=MCyeH!|oisxTX;+9mJL$ zgJf<_9+Gxt>Kd{NFNK|+q$xFB0TY1<)=9D*ex6CK=>iry^DGlV1=Zb|i?b_m6ANja zrQ-gWZ!ZsoN~akDuXnTlHEQXFIt)bg%nef29tn!Y$j<}*x%?Pxa-T8Q(E_*ER~PEJ zgv63cP&Jl;Tj^DV8a|na8Ni)fZR3`3{+M;f|*F!z7 zm4o)iged)sjp^Bn%9lhQ!v@*@8A3HAK?ul!Ue@245){>v~-MTWUp`nvb z;J5T8Dsc`D{_LI5NPz(TT;z7K)u<6tZBi2 zMuqk%BpQkw6|1a&P{ls45fgLJkm5oX^vPb9m`Zv7X~#`LfN^j)`?AFfIBr=aTM z8-=JqN|*D&F~ruNIXixrb(?J5ZT)_cT7nx$9`tHDnQQyviFs{-?`rU-*f56%GK0X* z(t0v9NrM&_iV(j-uNIW~W;(0_5hJp%lcc*7@7=!ZxBC_v4EN-=h-za62yPX`WpR7z z$}}?PXW8fapYy_7FXz}iBt*TMz4hue50iGfwZ!!7;d7efC{F$JLv>^z{Gwb7l`;){ z3Bls^d?WA44``w}cv}m_7?*OmYd7b;L`FbW4BbJS7{g(`6) z^Q_2)!xqTF@dhPe0sSyvMmo^X0Hx+Y*+@W`#(7m%Jjri()UM$rYEMe6Tj?d`WT3;9 z?+dRU*arxmANqWcpU|^APC8-6l;xB(RqF{1SH+zua|7kh0?a=lx zJ3Eiq#e)a5X*=A%2B}?u;~|KbVlG7}RvL(^VY!Oz>2k-gd_z)tevl?;_cqKUyGglT=% zY)y-@29&wW0Yljq;1!qRl)T-IUzB*Yp@QCdT;jokW4oP|^e`5Ij!MbV_|6Dj)H@{2 zjKnG$3i@%c_{DzrV455Fo=UrHf-}jTTSPaOWDd`NU0JR+aB%bgQ1(_~QHA~6sGx{Q zDM)uCEe!*Vf^;d}-7o_P3_S<}(y4TJH$x9dclXdSba!w5-*xT%?Smcfd$11H`Lmwi zy5o1>dX}WR6dEf`7ha^BXAPvdm|tam;MLZ1RXz^ye2%g_TcNjW5c5q6qZ$$Ts5ANH z96Mgs+Drn~MKT)j9?>(c4T?*K-wrkil*{(> zl`9}kSxah74?#-K@$!x>F|Ulfh6%DgzQCfUNn#{PMFVK#INp@eZ~!-m^%koR4R4)0 zJ{~*!5ukyrx$^0EY9*cDG<-9pX-Xn3b_`Y6=l&c`s%BSsH&bMPzeTX8Zn;u#P(5F2 zbEm-CYMf4KO~ZQs5OZTjw^6a;R-NyR9rmM1C^}h}kE@rlryjaPi?HxKB|WqLPv_{* z??!vf$1&>$UpU%}9HMOsn2j-)k~ZuT=M)p-kS2cMb0VP__1r422|Z~DX|A2MoLdA( zf~US!BI%nC+CDVjW@-Bj_xM>o2SL9(x0U#uCi*n1L#whIC556Y4H}tRSq;R!vIG`d z?svZbz^Kc|u;XJ!r@I^6zn|9bakicO8Zxt&mU6vHRc1&u=R08`R4=(+{&0Dl?sKEC ztt}hogX4n7TE0(CsrZK1tt_J*l=WP-bzieM|4QZ2tKoJ|Qj@Vk$u9MM%NlWQiwf_j zmdcwwDq)wAkAA6ThKbO-8O^Yb{O+=WZBp4)O-<3$xmZG2>;fTs4MTLLu~ER6SYA^6 zOsbiD|6e^+kTy%4#6V)^#}$iFguDCz`h66UPzTa56uOGBK}4`%Jn$}9TJ*SEQu=cemb$%viH}mM zr>$Nm0xwfZ=x3}cs03C2SmAJjyY+)RWzDYaRMKzfY6}qr9!pv)A{0nYsMUQ*i!8-r zyY+HI(@n~RK=g*zoBdncLt_+uKJDMxR9RY@RE2(rjIke(FJkbr7 zscgJZCfjcEwbk~mkufU_;8&?})PNi1#@V8w{9>6C(TD?;v~>%8g>R>YCnw!t)TQXS z0}GXcrIsAe*Ho|ehIASZIg7<0M;4UtjI^a1?!yOW;2-sxBlEqizd3MX%IFPHk26#M za>HYAB2A7Ec6(BdNNBl_b|DkM-LHIrSq+&~*a^JfY9K?);fwc-oAdopJ>xK>K_nu0 zsrIc1kz=HEWJ*-jGl|#SjG65~ukL9{b z_A(?>j;|-n<0QI5NYaI7}=NQep&|M zXeE74qaW6+Q$YTuiYAO0@9?_x>8>*>74AEt*-dCSWbV;KJHh>2&o~=d3NxC%yDY>p|EBn=;QjKr#dHbftKa`jlcmSs%wj^E^a*|&;0^BBWm0(a6%@M& zU7*?R(jH*2qNd5=R`AmF$UW>AC7)0SLMU-GBK+1b=Woz0De6VF-T%^JbC=JIa6W9G zm2;xwUrTq9*#briVQ2QbBb(znhava1MryNKq`&RdlkGJgAMo?pZDI%RV$L_7>Drs> zAC51w)n}#H@yMmS9M(Lkgr0LwZIV=Y{s=>v_M-GH8b9IaM*SgUaqViPsUT^7KZ}O3 zfoYw|0M76rZLrn~vrQLW&%veOW)F2rlEY0af_hw@=#Oa)hsZQ5FfM)id)K#lHL)FM zRZE@31pjO-NZPVc{av&k+I;w-uWvv1i>Hdv6;Hdz>JJ2S@L8z6L9=3T*c5gJVF)PX zzL<(6FVRmfk9kVI2=Tdg?C&6rf>0;BxE#^KV-XLg1pE8*kYF$Ap-Vc){=o7j z_&RI;zSJF$I=6bEM1Emp^J;>)R&d8_oU!-i$4zAROd=ns_fD(|I9g2m3Kl5=XachH z?eiz0Nbd*5J08QDpC3H`PI3ahQOVf1vh)7Rh=UUIiBzTbhGu;$#NAX+$%szd|9-3j zI|qLXd+RwVxo!Zr-YcL!IIvW5TG=vkY5BtT7mak!1=X3JT7%k9?}WDQP%${V{5BCI zNik;D>GPe(xEbgZWV+1Fronr-^$>P4*%LGjn>y(0L{}DC1KnM4Ly%S;MN2NT5usZ{ z!vO>-W3^nRADTuQwxRoz-*g=qVkIF~1QU2}!p6m>BeG&Hr>j(eAD0|3kM$)Upz~lm z9b|`Qk6TinNy6oX#FC`pVl|MxFp{$#IsLr}i`iw(LYM1tOx5r%Pf}EYN|j5Zu29O6mz=0 z0=mC8F#!>ouB7NG4DfN}+xhiHI;u@DtdoK1gLKuBxZ+#KmVBY>h-@4;#H#0pP|lFE z1EpAD-lw-sjc=Do$SLv5Dc`Yox$bPRq413V`OJF{)BX9bXIla)x2((7nmB^nW?i_v zR8h5jJeCza)1ySl&mmrEsL zMh0W9rElSN58V+}q}b`e?k2LEskYU2@e4JhKKZbj<}z+DD&J~rBTj4r*VOjeHtzRfTD)*}lHcldTBTb>F&FAJx7cA| z7^M?KEJP`K)+-4>(sMC9_)Q1Qq&c4ZE;pSM{@sgEv2SMqo(z6&tUxH5Ut(a&_Gz*? zkX#PgHSL|qC9i1L`Y zt{q*DTX2B)g|Z)9nh1!+esw|yG+KGjgJWUtM;)$_^$nxu=|)d;|DKE5_b+pZGT|n1 z4DUEgl=lvy5AB5xig-kOxim-j@$wt+&679Xv@kdEzgDVb?N57q@oK69C@+iubTvEZ z?BwV>Dd_Q&C($;X{8mD&&-=7MQ1N+`ED-K6AO9Yr00S1H?eVyqls!}Nx7Xs)xZO;% z3H%@V@_#<)tEkIU8?>hrY-Gz7kruklDZ1vHHhJw=fTes5Q)<~MZS~)X#X4I2ZXd-e zvG6oial#E#%n?ql{%Kv9(if#~wiN#VX$}6raO?m0i0|zQ{s+4v$<|!M*8R=$??o<; zMfo@LpVsnzKl8JLQYq~}j1ufXww4$_O39niJB(ft(~0HFLbSFLW+;B_qj>vib_2VCu*IZY38Lr@5;sEITgFe-9c6gTcbxVqK3DM?Gp0eHOq;D++x1RaYLXg=vC7X3|5TZ5 z9DtbHKM@YcF5-{bt9KAF!Eo>IbU%20#ob%2)oW6`QJ`ISWP{4DN&q@iV-vOmT>4=D zbVbs6*(KOWwZgMJH)R2WA(?9Fs5+ih~W#g&MR4fmW})GrGNRJ zcX_4TpQE$e#lad4Ux zc``fWtp90)V*hIAnceOUSo>+`329^e(Bp`SQU7(Jg6)_#FUY@2Xgcm*%~*0CMR)nq z*HXQYRf)j>Yj!B!ggN(sS4~*QGQ8HpPf8Cyj;sH5DC?~fz{ZslEoz*USoJFn7MV}w zA^uWf`6$yhKe&*941G?D)Qo*)`Uj5KzLAB4mMp|sj*vxBu%UUObic8*L%?=$il{bV_A=&ZaWVb zN6&8m1|jun8Y?w6RBM#YRg4(ZU!FqmSS(AXKN|}%o%H_xul)5{sGLHFXum!S$(W=# zGI6Tw#|iKK6a&ny-;+@hSIk|&nD2hR{YMb;Cmeg$R!B@{D%4iBJ-V*dVhn)0-SMG! z^MeD~Bp-rZeS%G|hJAVA&m}D7r97JWznEFmDL+(|Ha0GqRWP2C1k<5*FsqW$2 zc4++>Uv1==SSTw|C?z|4$VO-Tr?d_XX!2M2rY^Z(mTkweR_{~L?1>Gbc8q7zCRknfM!#rV6s`R$0O!h2T529j1Ga2NpzbEluk;P4m2J;cD;e!+;IZ*UK&Y zWll0DsZ|@>Gv9+9KkXScg zaoYQ!9+K3<+eP9(lLUmU?A379_FtBLan*6(E56F(9@c^EeXyaOj1WdWW|SpC-;mQe z+=FCMcly75$oi|ZTdq}(1k=UHPBfYsx^3NVg?7i?`(v5sSUP8s)E~H^ms+g!Tu|i8 z%$lr!Xr8i(cvl5!?7tbR9M%Q_jB_-=Sb54#r(za zdTdVT=n_L1UnO!;;QWk|-|9AsJtA3g#&l25R`uy~&U~S$PH{0_yfLKhKH^yc%npUZVfO>t zFQcgw(i1T>#n;=V7W$)=w-^G7B(M%}#F>6(I>;$Yib&iBU84Ji@z{25=-Y5O820>K zgEMO5HtLTJ{!NtjZUkXDnU(B*0Bn9ohgM1Q66}k(@xP*XGTck-w5Ir-lY_*SNbwi< zlSZvA<+QlsOmRQ&8NTU=LWofkspJbryWvABxfkx+aLW>OPJxaa;NM5J?87c;vBxfK zKk`SN22DH=Hu>9f2h7+3qA|^ z?S(7uji)s zMJ{@c|25kY`#)?4NPHcezv9BY2@xhI`KjQw*WICn1^g^X^lps?2>yJ^D;!{kQH)qu&YtHfgy_EJ(48^Lhk0rlDJ}hnITIcUF7d0PsK;2N{NuhUiBM~9^gtxDF z6z3Q(TL}(YKe#TPZM_=xH)f8CwkRR~jBcF7HqOrUIqR6V2sP!jA!U{xMUv~Ad8GQT zUySpw2!4%Sg~pl>U$W}c9NcEMy8gZi@O#Vi2DYN89{c)Yvx3DV9{dR@uBw0H{PyD} zkRWH+EIJIPm&&-$yubHgTHS4Z?w%Da<;;%MNr}%9pT6teg;Y(`LmK_^i9jVHJvIdru?6y$GQ^>a}RHh%vMHVPn&ziWZ6sqhTM!KMq#!WXYONM%dI4^1X}}#Gx5?M_jh|=1K$J z$;#fb8JKIJ66-kAlydacoo7XU|8iApJ~AbX^7L=S+f~Ehn=$(r4)fQDt4A4qm{hVZ0GJjuH%A!k+}i7uDnF2oO>mR`FgT1K9>OF(jBLxa2ofPX0hV_ zCq!1o=-)^V1!UIUIZ&_n@3G#n{1eH?wHD#Apjh%Kyp0fq%kKkW;_EX>FCP^_TGI?$ z3V`9~K)(aLxmm&P6#&K&FXwofQX4)6s^vu3z|JV2Ud%uMIN0jd0ARJU@{^ge^hb2& zA38)vl;kyEa7@CQ>;f>eU1lRSS-u%^ya!8>VYL!~AgeWSrTy_Xdm)PH0hTnrTvpr; z(`0qpcpmY|EPo1+Z(=VYq0^&r6Zso<{ByL!eC?W^*(4W6J#A%}gS_IDOG=m5a9J?> zpTketam@?E2N(1h%}R~|QdMv1L(2yV8!KOy=0bKBAtxM8pObl04xAL8J0(#`-Y2cn z|9uvdh=65QeU+r1g!1!?MEXbqwNKVYlJ@W5Ac4PkR7A^{i>{RIGA>5Eyu=8Dj3J z_7i&U-eZ&)UC*XCImp5axs~-s3`gLo==1D&e5Xm%nlgv1X1|-Y7mI>Jn@;;Y1}>iCvJPW5m1a$2#nbh=JQ*PKd`(SZytdf3)ruc1&R; zX>rf^=@~7AHv=5LBJOOmy1J>JmDMhm3fTftxE$U^7^tWMuMhMGrtPs7V2ztmFEflc zlN(gbjrJ)FTkichQ$&(U;*gT%=b$-QQVuM0&NC!?$`uxEk9QViwlx-0`` zeGDEiy!tNDW+D|v9im)2B?tS~iH;lxs$CVfTWR)L;ZU)4Q319{@Tg)%W= z6?tY4HKN|Z@V?Yh2Xl?#W4AoKx!g-t64pMsXgB39qh*#SjegBgTupASt19+W-pc`; z?#`3$eN0j{B~^y&Nnw{qv;eym{XMu-!PXnj7&o5v2Mg2j(}DtY_rCfE>QSL+Znd|S z-=eeA_Yafo_M5Nyh?X!p@k?Iv;mZ{E&@o%n%FMcI0z&$JC7*A0MkW))kjm@tB)ntr zV2n(EU{27(iwv% zfJb~VL}P&>{ONuXJ+_iE~`$V zr6JT3;_OrY;l5$rxP{Ao2t6kwkB zngySGC)PdKIzp(6cHxiNJUJ09Sc1`<#wxQ9k%$)xwfwbA# zw)99(JbzB1k6n6{wC|U1qJ@Uw%b?i(8$SgqifqcDb7^BwTk%neL7Z(G(=RZ992#Jf zYw5YVdRud;YPmg&p35??UZZjRNG}%=yN(qU&qij`W~Dl1TG%D&GS-B^qD$vaRLAcjNWKnIo(YX&-X|B|tBwkuJ>WksLjn-3bbip}=aBUO)A1cop%CAoj z3`=EXpLRmjX36y8(>NW;lXKxl=(x63`VPizB70+YDgl0F zgZU2N1Wq8=qf`M?;({tu$enO!+al?%*`FV&{}t>HL?!)c7asTsCKDIb@t2AHi}haC zWt99~z)Ui3+D%|~gX?z@%z7s->8uTn6+7deL#$EZ7m z>a17yURQI;i>zA?7Ma>yHPvS5(`x`DALB4y74%-D_(mH=l)(jas#|PP`>@&6t9|?Bv%6AM5AIC!WyOn+xLc>oqyl83SJn9`LoV^aSwdMa=`4;N8tK~ir%9)tyCDH zGc7@y>&F<>p1x}SlOxzbLNGA)tTQ_J3%+7XhF_AKTK-2qK6O=mK1tp5{6aGwQf2ib z0d?=fT^S>{@z)gZv6wYPQX7AI@=eB32{XXd=*lQysn#zCV%|yhPxshrhzjt@-j3%e zGfY_P<(@OAvIMZEaOBZj$IHpN?iLkC*YXKULE%Pp?kGC^BpD zWqTfye*#I_X5jKbJg z`t^VUf-CmZEFVKu4et}g)sTlN>vi16QFCXH{wZtqdak+6ROzG{B{U(!YtoPN-*v{;^fFob5UGgR1 z{2?G*ZxqDM%OAI>G3fyp0j5lU0&C=K+N0v?_EKa{-dgU(^dyoatB9&Lh2hSOK5xGA zS}WEGV#P9PHj4PQe^!B6P7A}5_^AHl%~N7DSQ+ZmD?Azne|D!XRymsJlGzWjdHkV` zve^KPEyi8UPz2D&VVTm@^-TC~1zt^5=M<%?)~5P)Q*MLhbnWeT>sm&C#sND_E8j|o z{+FZ&#glzJ>blwe$P8<|h?|2*Vr^IV{LH0gV~LMoJ>>R;9PzXRGjV98hx)7=bFJpoWxmSy@Zc`$LZEk+saAz`u@k; z(^nZLMV(|NbL35#?Tu!qHI(nf(?#I(Uquf8rhnW&zW|o`TldRKF$_$mr+4XIlUhHS7-bxQcpPx9eWd^&5t&a2*+Dv(u%cEtC?(CR7UK^`PP3c-|r zC=sHr>DC}s(vF)>8=V`+{QH&LN6*00FC=0R{nyN4#ERN)JLEvH$qZ09YtxrdIpbI9 z2+CY*CF4(u>pJj*l)D+W?|2vp99z4cX|;$7pwCH1@dJr^G5y`-+G;1=z=hh_SMp_! zgMp~8Vh_XGqUu&-`Qr~wQ}EQonp-Qe4Q*8W*>uRA7DAW9=a441J16TyU|LoNMO6(y zvrBGv#A9De@>?GKy3a)$msVbf*Nr2Te80`}bfqL}2b`xU)QKr&J{`Yhy=Cnks6tAL za7iWI*4YY(XeKOHgpOmzX$*Zx)OYTZX>nD~v^No?;+JTu|0HmrS;xLu^5NMdfz*`h zBtOATKar9zb=ik5TI4h^j`chDzQlp7ZN&cJV`J6oZB?LMj*2G2;*;VF7Z7@8GWAox z2M!eSsTMAJ>M8Tdu%;GN?AATqfkqscKM~Hiqmi*5l4;b04r4RaZFuM^25BaB4s&;Z z(G9nLFKSf$L*BZvC4}&!%j(@#67ZH@!XlBcG>4Qohp@1-L2{*ZTuc%y>`qHB4@sTx zUEFeP8jq95JIhP=bRG)CD5O_!sH=%jfF4ZAu}gw()~PK?vh5t%A(o+td+Xd`AESd; z5e}5DK@;NS2XIC73cXNqzV7sjK3>{|Y7z3cyF#PAXlqUyPK6eC+Ux9OCCw?V3ge9BIYS~wptQs~%v zhCfIAOd$M@Kbqz9HpE5Et(BP(1le2YyHCR#Zgj8j? zqKd|+B3Cg^r+m_9>2^@t@Mc8WYY)iNC?%3l!S5MD@xqz`tU?A@ngiBblp-zF(afD` zv=Rb2Jsmb09>3s;0n(%w({gTC>kcFHzl8|0n^&uUW>iFBb7E?mlYqevRP`<{x#(X%)a*k&=4~UHzq*{{_J3Z; zPcyjMB0%XVpF8wFphlZ@`HHdk5D650yrI%}+AJOTtm~Hc$;`quuZZJ#?!>#x{DX>W9W$EGoGuM{c=Bqi&Lss*T)FtysLZr zvFjLPWr8PT248<-9&?&xP0nu4#&k*0^YRXtRnk;4v?cCKWj&w;Jx=7}-_cS{b7oN7 z@i*w_hL`;>>kV#?M|vGs=A7Mn%_GSPUM(W z%H!eX#Vf4x&45W|jVZ%JZ)TT88HJ*RF-d784uY4;{|dQ&$;_ZAt5@>Lfgz~XXU_Ch zN87(%UQ~4oT1DRLo3Uu+RM)A7)~5Ft+d9foFZ3KQ$G?%=)D;=L;C60#Tl?lVSp$<~ z16{K|ddVUod#==Dz7KtCf;wL6O#qxPF(5x-^SxkMRdL)1U|qaG{&4xafol2s*YaYX zJ4^FZ}hy3K8EF{Nn4-U%+#dJaJ>mtSJ7!wGt41mo5{&&`f4A0evT zG7;b;N-6k=(Wh$L)fm38?l8w55cg(9;t6un1V70yj>kH_@;x@w-pwcMwnP0U0L(qq zX%sO*1xenb2v}3X64d5OYyUBgU>Th=`Q1@@i`2LWw#oZ-hPF!r8n!d!LzC*DiMapq zT51@ot^~?z5vSw(4;?iV7XbK!o{qHfr=)rJxaU06I&x1k*+*@9X6 zP9m2iJm2}(seZCaZrRcRAr>;4Q%E)+FCFV-wwthcV2ufc>a@Cl3gwN~T^_#LS3#+C!`|Lo0}O8V1> zYd_M3pLdq)oju|kp6#MhMwenvxX2ay2Qr6J`?e00272ZBJGAZ#0zA8WJ z#`(L@pkrVtm%~c;_tMFWo#h3$0|QLpT_fntMdg=^gR^oUSYom8LEDxdsdMMwSe#{i zX*Qc**Uqk7TvmBo?YQf&`)pQgO?(t$nn6?oBTwF9dNI5&5x<%E7lt|Yy%1CuBe&Be zXAYb{ghWmw>#hvF?^?3Im;LDk<^D5GkoFqtEZfQ*>6+Jnzrx69Y3tM>e8u_8UbN&2 zDP61t+VBsk&LilwykF&s^gbFXb9JKdEps;<^<2;_f7m9A6tiPmg$PQdZ)>*JZ(RO; zNwD1Sf(`N`F}3RoTIfD3LO4G&DYjGdE_|F+o(;jgJ_j)_wgOmQmn9Ogk>X4>?}z7O zqz4#fgH_ks&_YZDIW$1TB6btmzqA(Mr6e&oUo_WxI23vlW$c=4V~wv3mm&LGKCT`9 z2s8lJ{eqSmJLnaY&)taX>17#0ITh&I<2$yCKM)7x(I^clLb}eBh%At6LOzxq)h+%R zX>p`#&Rl?&aI}EU_W|eESM-GvSv!K@+XGToNx%EwPNPSp{?p+EjK%2VfcQ_`^&!=YX5+QD+MC;!O% zZYyt!54UNXItn=$5Gu#;(wYlG$jGSu2On zYF})iOVQx)sIFW>5gI#s%A&E;ox7K&^KecCbUVjE)3)bA0^+1wn{PRPxtO_5V?}-{ zMUyoy1^fIM8aaX%*~I&}Av)}ZR(nkeKSTv|r^`<=jkLJ@TBH%`jqTO^7gh#IBtf-D z>n6_Gmf<}dhHg)eHjSj+E~qBY`Zih5Jz!(TEwEqJYq@Sh6iPp95A5yane2GSF9!&)I0)xUAs0tf; zb)H>yv8BYfEQYAJx|^-=Gk;g;6RirhK&+nCorKaB#U87r*G<(MT+U#3P#rRUSIH7} zBBD6QHt&;C?>9cAA7Y5Dkg2VNTC69uWcb3Lxx)&tgcato<@3J!&#*BrGfg3EU{ z(FJg44TLH&=^8#xW!xStKaZdaKV)Hixo%fuKi_?tmmbJ9Br#1E?=C6aXEjV;>|~Rn zLutlspKmO8w8_hBO?0^#Rm4HeL^n~@SvV|w?C6KdZ63WB;7fZty}y|827 zY(C$Mk;SQH778@&BMFjiauy*AMJ#{@ za*QNxJS9EW*XyB{E?#?}JMp(pB@WvxqtOjSONC-y-F=$rPj6BH>GtKn9akU^hh+>i zPn$~MILGpqxV_*%J(FLDUcWx{2if&ea$H3&C^oxl?Y1y+)?Wt8w@2M~NNyr~*zDDL z?pIvPIC3^p8RKEKAD0Q~yLwq6v(N-yo{Or0{_QovjZBnJcm3Ss9n-e;xxw&@jbVbn z+uuSaKUftZqK?{2fgbG(O^4oX>`G)(4SJ3Xu(a^hYPgF!O_^s2E1CBF_j14>Q6s5( z@~fG~kJ@Jqr$OrECxG7*$gA?^^?08W_wT9SY#`9VJrGj9L9myhHUhbFx9n83V1CZ@ z&w}{QQ^vov|QODWz18{Kr}tq7Afm_ zw^6~Xq$VE^cZMDbg2HDOKTxxvzC;RL)&6&g$M^}A*DXC#z6X1E^DzpY@x@cJ2_I#V z76R)1_Yi*I{1{e4%!zWM^&16V`jPuThtPHuXgY%yd%Twdx?>`Nbz1{*R9rf#Dm%*c?b+w^{(#d)C?JcAssd4%mq1T2MU`gs$SO3{!f zj(vP>McM;ZaGS0DR1IBI?Acu7iZta7pLMU@jhDtur9N{kZ_8^4iip)FGgS+jC?N3?+*A#EOb~kLH?C2vy_}eU5k@+dx{t~Tgb~4(;I3XjSVJ= zVp_80a-?;H8#7&59tpYUo8?r0+81Y_*oS>LGBv)Gj`d36RBHR8AAb>++9*mq2dlh% zi9U2Q2Ea)uqv&G;6JiLai&>EdTqTS_HWmt1+a2@26YPetDItCW|B{plgWT#`>Ty6< zc3){Y2qn_=wyW^iDj6r<#GNlgA14=E-X4NDF))${02=!`>3fHq%DVB%Dt!dmen*V8 zUZz$SpCm;SDY0oA9u~;y6iuvSC25kx+cp_S@V|vBEXT^LzFNm(Wl%(ou^&*0CY|lL zz=0Ea@0<(WNJ*>6XL~Rw);QIwEE_?0^YoTI$}LMq@f`t$InRX3Zr|pu&E-)S?TD~? zv~EEo#jmBw$M!*}PqL$r&B}-DUJgmR#eXYIhOTYJ9_JDSrL4iqF(UH*r?db8*QQQ| z1h&YBEiD7C)wxe|2}}$3idUj6WN$F(dgg{&KDwHJ#75lUo8>A_hb@ZfBsGj399%U@ zKSUIznkH}Q;hjsZ19Uo`PT;XfZEi!A1ee1;H7!n|K2EqJn!FN__g#V6{irwc%ixW=sQbN%`87z5Wt~6{Sboaiy@K-{EqFHN zcN`$j5vCkyST&YI(em8W=I2wc)F5fZbPsy_~RUEgSu*;@b-;*JJzq<$ap zQ^|%l(UpV1AoHOXiNB+q?v`phiEz|r#7@sab)uY07%ozNB;hVQ-D97OGsW|tVe@L} zi~-yU+`*4CuYb@~QR6@|T-@S`Yj_lQLt*G%jsR}%!T)tPzYd_P@r=3t>a$zv>N)!B z=#rJw>!54KpwfF0<4i1VSJ2#Gp&=j^rtjT;Kg^?);B%`Azp({Rh*N{6A^8T}?`M;?1qIS{l(kmB%F2i>Ld zN@JJ=!LhhnT(ozU(pd2jwy4=P^en_vTQAd<((MTd`ne-W&YD^;zvFr=?k$((BO2Dd zLZ-gAG`lvrrZ#US^#C{ITaPlg(x)Cmza7k>%`fELz&<(+L=23y9A-J+ms}u-m-qLF zS*&};#qRIJU2mq2uKS3ZH-E+JK@QTw;o1WOFv}6x&ElRCuLS^dn{setY3Dw#N_DkT zoqY9PoDepVF96PX8nE{%rg9%FeZ03f@Wh^}QJlI|f>lFY(;D}Zc$62u8kF<--;{fA z{d!t6MWj9Dmp3|JUF=QUFDwo}8nl{jUEteb2U}8pZ z^R6B+UEXw4R($XtoUJH-HOXiGIP${zI?=kc`N0Q4zh`J5HVVIuJL7zsvv==8aJp|I z9yhD@>~tY1N8O@(Fgx9m2|GsnxoXcDwcpjeb!RwtKA{5D#W}6?9c=!zvqDd4A z@x9@_R#&}=P4J2eP|sn5e*QtjqFiR5%6rR_2L%O@Gim?k!)*t zc}U^3CjfGeP0K*c7}^h<^)cB&r`IWOf;`g}ARU*E_f>%9UGUQ>JW}7~1Qvh|@Rn^n zpfRX(e7G+=m;ooXdYlD@UjZ!7Cg5kbM=mA+$J9-ObF0YUz9tNRmrkGROc2l2X;1v} z3BP^ww(#)Lfm8C+x|3n^aZv-05{Tl25DmSulPB$XmZ;_SNUR!^e8mA7vOmA{kDXZ* zIcq9+?fr=>m);1zy7D>8hOZv#{y6fN@{zk>Qh#bcw|%;K@Z?W#fQ}3+Vgg3yYqtKm z3I{jU%`w%N_pbv0+5qbD7B8Hf5Vzf{xL#|!t#`-yaEBA3@@9*kTkNy^XPCxsnGTIar251oQlH_((Acb~ z`q0iSxiqGj+5EX-D_0$8@oR@PN8;U`tM@j3`fZq$y$`~P3i{-^ka@(d_UY+_!tuGw zG!d_5`d0lRxp1Q)^%{DgPt!f;U+`4TT`~n8%mAr2e_dSP%g*!|yAl%+l_1DpKNM{O zwn86XLT_C7PlY)>1|o{>n^t#$w~0O1r`s0@q`}GM)Aa@#5!j%0$H2AajPA$>)AG$( zY^5*M1tBy#o&W*@i$r@dt9Eb%byVz!AzR> zEeUSRvOfI&lBv+#)H!3-=xnoA6#fHzC`#43MzwSnt?a#eieG2fxS5#0Y`X_Ka>-9# zwvo3LH{2fu!S9d-UCwpbLu~#OtWxkhzXN-Z9#5d_4MDO%Y3q8a{YemKOV8p6n84+$ z<+`_1p^GDKX=q+1#t6*A>iF?|kXGckdrj zozHoUE$6ax|1dVw@W{bJz2_ZCGEv&qV7W)z!otwqZ0*5BDL?GC(J~Ud7Sq!!WVRSn zv)n##>}FLS^3>4;e5`!BU#JDRE}afez;6v)Z+4qfEf&J$l{=H9m!Dldce$?^zwTs^XY*q7IGXSY)8%VkM-ms<}9mec7C z;D@{kh`~jH6aN$PN-S#q(!blYqZYO`EKnJGHR!#8KhyH?lE@1g!&=txn50@~Cw4?= zDR#W%+Im+iP@kzIK$J9;QA=ZYmOVsxpwBIE<#|+(bbY)`FJE%zPRrk~KShTl2Yfbs zEv?(bH$&$iYx$Zbc9H68{~sUx|GoyPWYUwxa_;bi-o3;VVu6k%f0o$rr@$pn&l!tg`C{3+-?p*t)(&j6$}#>$4O zx#)VQ38wvncFA|nBh&3jb&Q-`>n}lmpz0Gf-4G`36g^P3(-l4h=q%_AKyHZ%2wa)B zp3je@IvlpQm+@@D((BgGd@e&H0ZpC~Km@@qDIhsN#9r>m%G|QLm?1X7OM0$s!@{$= z=wJ|x_tA`Mk5%r{A*ISwmFqcQ_b#*h7+bEtSTpw<6T@> z;eM6ddCs~Xo(IKUXJ_;s&8ctz1)D)#*i6cV`^;Q*|F;KJd*hjV;j-F`AvaMypkBfs z+W^nmD~)9JuLqZ@>sWS%H3!%#+z?$dD^C|RiivW|3Vf~4`*J)|sZK3q<;t{UX?wJ5rjvuDypsDSUZh~}fmcQ6BP z$qYZ%n$_#C5;S_r=bRJS zih^po^LSueqmzt~*b!b!C}+YX_^eZ{t~liz;ET7Gc>ghpvs1U)zEmwuwB6IWNSx(D z;sH>dIE6)b9<((4_2^OKBd8#MVLw6K`r}km^ufCeN{+O2&o#`)oz(b)28)rrXJDCL z{V)!^@9J4HeOM?w=;$atJY2l=9r^wDiq$`9Lz3&X1>o>NRGC&QnPCIkjPaB`?o(HZT}T_=>_nM1pJ%d5#+NaIstO;4mM(75$8JpaBI}ZilI%DJ z%SMrG$j-g_+s93}>iJJ?UM-GciWDyROpX)PD@NMke0NNlrsG~`-KAFqQwy$A`h2aG+N=)wsdJ(~N?jEW?%sL&=%6=|i0;CxUGDQb-|%6)c8%A+8nZci zZTr_H_T)T$A~+fOm_G2V9s8{D{IQWfh5Yr_<9FzR%uqaOv+h?y%#}Q*8eCY-j^<}0 z;lsQXr3GIpv84g7XzXh%M6N&Gi!*W)JLLr^g7_h^jGjKTQ=^xZIYcdcrKyLgjHz&$ zbN6?1#O#o{U+%@NQK{o2oyeqzvEQjq7kpbSH-l%zlRp9H)2~OvkFCo)3{s{4f+a4JJ)Zm{qj#FWk|vNb3$B zHnpKv8SiE`7>E3ERh_uYwe4>@YdEU2NZy+pt>tgE_C(9B{m1C^$@QCUjgX)EHh*>X zRO*sq^49c2@A`AK%ZTUHVYmQ7taT^m|6uPeqvC3tHQ|Ji5CVkY?iSpo@!(EyY24i{ zScl;5)>!c1?m>b}1Hs+3(cs!J{hTvr&6;oC=b7{V{nq^K)xFo=T~&A0bzN2azN6`2 z2l7u)NoBv^xTh$W0)_4R)oe{I>0B)XGfI0NS zrp#PFUp;QG43bR3MAULdB+45;OjIs>YHlAL!fnrX5C%^qU8-9q3g-G)>DD~A=KRot zW{~VCpxP#^R5tria-6{=mL!~1Yj1OO_V(>`>Pk~rzTbpM-Bq@F+KZLXW$>3Fe5$zg z(6k)C*wM^#-gkLma(`UhoVa!5lJ8h?ep@QHJ!ohp024FFtahofOt6ej z?smy$oI#@cdR&mEsXKbUZ# z9T&dvS)#)Y>ezjp%Pg;@TRCWS-+$HJyCeI(oq!FE$Y(_pq7*^_B6IKmJ^Sx0%RirF&1HGHF0CmqyN{A6 z=|2~Qu9akQt@bW<_~N$hs0w*#>HW8gISz#rmE}-`m7W1InX>B=-kgOwLCp~_l zw4uG(CQ=gvz)Tv81UU8hB6ciqno_@~oc#{~ul+0Q61)LW& zY`xZ&SQS38Fzp(HhRZoRvfM5Z!>Pz~^)4gkT4#QBo6Ys3BmWXeF{dhHkp^I2KhikH zum!ltNti7z5#`88t8?Sz>PchSilAdH7Al)zXnb6q4f@2%Sky3@)b6RcrVy#vnV5Sw zOdPay{~O-7zWVSoJG$OKt2aX7*2xO-ZI;dlyK&f)Q%bhwN^Rvzx|dS<)y zOCC1(tkKcL(2vgA6*Ef2rhBG*j*?%}qQCLbCYFm~n$goUv z$xvL(R1ijUcL}-=qC3^Rj5jeXK6yU@1-`cza3YID)hT1SOt&^=Y8_2^TaRmOtynf) zBvDr&?}?3tVqG--sJs5T2@6I?>eD2U&ENUuVMpm{wee|s&2ME%q&YulX?d)}gYCBR zNO@e(=K4CjURb_?;skE~{s`Ir6Prrdnjfr>F0;XMYZvQA&oe{c(04Uf@Z62`UvpE%_Is=heXzXdVMuxZCKWNigSk+C`zDz{EmezVd^CYD(TYZ8il9)Wgb zmjp@&s}x7_@*0mol{&(2DdPP4Yp!mMiSdOuq?kki&W^r za4OONQjRxir_FbQ;-{8fVb${S@Ag1gw8sOgx%Y3CT6 zp6WMVwV8;!s|1{fZK1dKDT_Q|?48_ij(X^OoM)HqdFa(SOzqlV5H(h<`NA^Av*+N` zAnjY$ZT7wopwZ+dVCF)Zk2nwmyts;HJLB*nZpWlFzzcm5YH%=A2m%lAn$)>-3stxT z{!sUPp&Utz10D~GQQOmz!SbO+)H+xV zHf9gy-*)emJW%zPYY$fy)F{ntWEBpB4yNVj?5!O!6<#%M_JUKU66`OUrsv1Ksis;* zx4}tl4aEz~+$yV=BWrcl0|QQg>qWfpc)OLEYeuj3lU-d|{a~G)C&#B97EislcuLyy z)smy`C<0AK3m#AXhfj}|ormR>jnaNkpXIc`)?}OZo7UeTZo&mh#FGSNJoqb%77%K; z+m*3?V=)Vk0Whq6F5Z4g188;ThUIiRM_rXH31W0C=`de;+FG@$vI?{-mC#^JV=p>= zbMm0S6ys}daXY>H!$Fbyvk_0D6at`QwWvCjXa7m<;SDU2=Rw4t%oS^X9oe!O&x6%Y zoA*>d+}7@Qc1D}WJu%d?W4iZ*=VT1KUe)0EDhm*;c zM#;2#Ru%z5aJNO#@!&BH24{DY(WkWzp(8Q|MwS0oy%Q&ATT$_T%ivJUuqVHC#+Uhh zWpv864>-lcG1j`4SN$cMwi3fcY!fOx9gbCItBEh7oO*~sr{@Wt{0(+EGl_qzteei4 zyQHzLdiW+-)|rXj^dQdmi<;I{?=(xwW!^O{n!D#jcZl^|o`>cH8bEEQqK|9YVd{TX=eO$gewJEgk>gR+s_k_BLP1&f0(@*-7{L-FBl$I=|!x_3^ptmu*?=b5| zG-H`vyHvXHQuWavTW@UAexkOlpQ3I{2)`c^Q4IfGJfoi8p4}Kf)>MVzYT|O(*Ryjl_hJc>WD_|KW=AMJD@IoWhBVn)?3mjUIPFWq-OA1#-vKSU(c~<#8}k-M zphA2lQ%}&gZ66@@#HqKgKt7a+orG3FqqkvAuk&Q#vJKC@E(0@nK;MwF2oygzw8D(H zG6h`C=G5IIHaQ7#TJhB2Z6P9BD`IWmth7WBVZOo?-f|zCGVmM`c67nx;=S%}T}jLk z)3@S2u7+CFG&25Qgv|deLPoEV%fGcXwU~i{#chUQD3-`*Zxt@mTh{GfI(*Od`pA`r zZrMp8mQnU8anHBzyr`oURvj&ew31i^cNKlR#snQ^1c_PR7};SKAhWu<;AY>qy|uCy zPXx|ETZlFWFJ6VZ0X-voG~jmomkJh~8bfM|`}*a2ZS7 zWJ{rG(QS4l0e>%q#mkprloqH z^I4fP)HhD*J~z9&fT`55>=)f$(Y|{qd@Om@WLYNIqERe`vE>y#X;7$`o3e1v;XLbc z`h!>!EVq{t#%85(-%J5soawzHa;5EUoz4zP*2A-Z&A+p-&N$M6ypqmUvs>!e9=i!u zWWVsTdp#PjEZ)gY=i!dto~OzF%-t*4=JBqnWlzt&q77v)0|!N@DqQQoCm;-e&<^g& ztm|`Y&T3s5RpC0rQWVr4U815h-jv&5h6;mESnS7&W&QQOuDJItO$xSR@tWPtaD|$v z^>A+)YTP@FNcw{-JtuV?&6Ncb8-g8_l|7i_(+`gP^G}RyTk&T55?0zJ-M5dyw?F2W z!J2+I%VYk8_IfgB$UMbR3D{3Hqwor{cKOKBl36j=8=iL+r;5NlIQm=}YWhniUoO56 z|Ha_Um<*0Z{yzriBWrNp!g*$yEdhgzSYkrkSdo){rG<^kg z6dn9>TF=&5i?=PEjiiYuX2rPwNU65z8$N;LQ~2brQpPM)`dkT-R+`0CPO8&}*)EN5 zgOEnIHD)!QO+CH4gvh_u2Z*JJ>;pb$$@f0(`dL+RKzgI+$g5CPsR#rB8#Gg$QXK(8 zbvPdv>&kG(iB5NIZ`OP-cHY^)If5?1hsjv&J67-RPWZMT+$O8l(V-7pay8|D1-v4Cljt$i zwt$&`l|2`wKFx`9-D=*3_1!v+`{3p2wA3Xo+2dDKKk3c9R}Q_oAjK zt!1>_zlVo`xn6PgBx9G)F=tlEqVNCST!H5#ZF+>zm6bNtkd*9y?8E#o4E>)!OQT_1 z_{Ni=0sXhG(tbc3UTM1?_`$^VuiHKU-`o7ha$pkwzT7?{T2lUR+^cJm?(@lNgNT&5 z@!z$U{>8nF2oQEo^xw#tVLa*Rye@rrV_@^fD-w)ZLI(8g8%q1^5LWBDaU7q zvj3JS^Pj$~zD7=J^E{4K(b4{Ifsp^Smv)F~i%)8Rs?L8aApU0pYEXvT+!hoy-2Lwu zo#QX}q9zS9{2Sga_xsD{52ot4xc`RHQz+o>HUF-z`R{o57z=LmMY8VL=RsBrhP zyeh2yKYjT32DQzB+kEZ`KRNjCfAfESWBwNIUV@Lw8vl-W|KF1OuL%8rOX`2-pa1_Z zDJ`EUTwGkuwxL&!vGF5|01oNRE_bcy|B8VBb4gKF7a>PAu!a|4q2~|m$VBDI5z`{P zx{@i#)r&}s17L;@C73GMJ8}E^ZFp11)Hsm3uM9c$9xYs|%*AuudXcjg?c13Y;3;lN z-bqvCn76TUB;u!*&HlQZadervZ=5AuZ1=gjzdF=uupH06Bkj65)va@#&DKFN1<^tX zd3_pD{&CFVL75!IWRtp|U)yE2qRseO0rA?>wT>)@RIbEXIM08 z*~~;n79`NPnHiyZ|6JR=R_eYX>^g6R+r%tg1u+G4P|bWIiX;`Yl=wnQM)*^qX=bLvnFfc62-vxn;% zhl!PP50UDVD<^+B&#Yz9=Ysm_lC_oVBL?u-Y^;Vx-F||UvL(`IDLmDL+z!5UjXGvogc316E4!PMD0MgXcb#smdNv%sU!o&8U!Kg@Y_%R%w%s3;(MU}! z%A&G5J{hglTTjs!XSz(v(k+Y#MD+LS3s|Jm{EMjkFImb+@~w8>==q#>nWC7TpWVat zvEkz;d?w|5*nX+8r9PyQGCCyzM)6^tF4L*o8a{1skpYLRyQ4Zi>Yk|SL2PfFLDQr; z9O>gPbZ<%UJ&zZ!?#4k+VB5-?ZzM0BF$d@hXHFXmZ?TjG&orA_HTnx<=Jj9{KXB2% zeJ$=i3iR9?%Oc{jTSs_U-V@$4-rSoM?|=1EWVoa{=)d*?P+DXy>HSn+TU5GTlisoY z>3|Npov1AQa8$R_O^=q?rSp2r@ef;-nCXB$7J<^KtiE#}j+VZAa*(l9Pmgzn;vqZA zfKgxfl(II>ivbMDa95N(kmiC3~13usdk*8z_1Uq++tH zW040*k3qJ|cFMI~>X+0_`z9v~?K?bMr?POP)qkQ4;DNQ!_P!ghWBn=l6g#4A<=6oO*zk)UzCNYxC0?5{ z{YlPt9Ro<|#)NP_ptrq7nt6d@ntgYm{A1kzy3S=e#bNA>sOlr{-lvAo$^U?vy1-Oy zDSg^0JFI7x!WRT1&3ZqZT1M@e&UAR|m}o)FRi_nz+EBZ=Tx1XA&|v@jshma!>z0|J zrXe!hcO30Cwe~BQyLvy5?Hz3sU7yY4laRq((Aw# zT0qJ5@5o*SZ{ItOjTd@4+xcByvL!+lHt~J3As!6lvY{J>PQPCJ=i2=y^YKL|guc{Z z@;rUH!T(|dqMHaL5kF)V3p~D2%@h%cskE*@$+@2{#y^dvJM~~UV2!C{A?{jFw))$| z)O99j1WlN`f)DYXQjw-O9G^5u^l#a&upXKYFcwoSI6nM9WK0=DF(1%B_101f)(K2` zziDS(C{CW_t9cPf_V``rc;5)6)8b0|7t80+*cXGhtTU`rb-`_OsC!(L9ye`kE2fqu zW;@sc+>+!SY$B;j-E2O`_E*{l9tlv-@h0=)Xy+pbXc;}MoN$cw%zwtffdn4|`2 z$+p~W1-V&D0z=1Hf255GjCtu+O!z+>%jOB`|A3F}y2MLq|BK(3Ic1l!ni3JBtv2^+ zVdR}rK2%bypQ987Yb`A zyrS^zQ95wx#U^|)d(mrIEF{h+024m6e&XjxtxQ*X zJ!b?9dS$u>%z?e1Gqz34Zz)~`XDEgrV9|;Go2*iD4n6NA|O( z9}$NqHvnGrQALpj;@Ew>ND!+i|GudDH*}l046!zZ%@?^vufDo^--Ya>jMO~grfQ8j zp14$U;rW2VnB72+^qdss3Nc6IdF;!#`L>KpsSA=n3vx3?64;&n`uI!uv>7LhTnZucAz12@1qV+|NFN^F&o zpofCu$Y+)GM{qq_a2<4iVmW|8u>AbQP>QA7MH^V~ICy%Z*mUK)us4U_*@L!eHAnWj zuDL+#F@}GUJ{k6}UDw5cHE1F^TVALA>ub%o_?;HLlRf@7#vKL%G8Q|Dq!k6RL&VV* zQ0wHjPX1AIAr<8*owG@gp+P|&ax*imSkfIA>w#FKu_GHxh_;7S%=h4#OG<a{~3d4+*6ki~oKl%SgT`Si? z&?)&~nJy-%`HL8~r)u=mJeoS?c<_gq#Vk}zd*Y)8SJI$@H+Ai0efg6qxtir`{Hf7j zu5gT9*RICzG`dTsZ!Ki*w5ct8-&Vx`&12V$D8u1`{RK!7!@${;LzJMZ*VN`qpBWdg zl&hctwxs_c@ltreAT@gQhZoeY1}5J#3JKY3Cb#L{R|)9ZiukC(aQ|5n3!12H=TT9$I+_xbOX;vW}_Ab&~E$O%9xpy8|5RF+FW-C z{B8~X2ZR83Plr7wQ)_xh7w%!u2$xn7^f#_NRt9X~7=eu*M%x&o-M z)VJyhK{E~b$T;!P$lM)?D^%M>r|H@Fv3pZe@w6>Gzlm#w7@y`+f4wlJ{1r8k{6gR;Y&2yb9Qp^Mcn;hCl?H#CAe}1{pCJOHg zOV#j4HM5|1VM1zo+!A@(VytWyo6G(aM7Mlo!7EN)7zj~WIYj-zA`|8120C^d6@8Hn zB;PKKji%5Rh|`))I2#-mhbG9g+#Pm49ip%_cTpW~MzRX;B&jJ09M{9){+=M{X1K-om#}?lk^85c&NZjCJ>eT29iLe+j~_&FfDNj*>rY&hcQ&6IBPfR{RvPuF zw_Eg?0Ti@6fha{Oi2^#azI+-1-Ld{fcE7YdW((g>#J-Wn|G}Q__m1Q^Y_g%L z{n&oocbD^OCz#cDBV+ZFHeK4MGe+oPvH zDazlZya`EidB`kdxr%o9QxoWccck|D^cmI($FEFfxWGociTnab7ZNIOCY#R7Le}IM z5!!~2duI*5B{rsVo-Q5liyRYT63NSyx24A>`kX-JGXeo-nHrOVo7aK?7h%$Zo4dT7 zcOTy+!X{c$%rE8B+z|>NY)36LTV5*&hJW&>o&`6Uqr>wJkHMTF2v+rcU$xYjvB9}! zX5bAiG+?O-J!f|OGUA%^unhmpzgo7EN{P~Y2Lu&Xu=~xF5c1B_G~Rv7qz*B2aRCJB zW6Dp2X^p?^!~M%Yrw{l|AWV5~FG5BRw4nAOrV4jHml&f%X~@dq>4V5! zqR7ye)bm9#db3Lb{ON#CbI?sNJ#B2NR)>jhsX zA`K+!C{YwKC!DxAmdP@D0D_51scO7{4K}$O>0@v*!4}Ke!npse(~`z{l%?R-Nay3o z-1pJ+0|lW*x(z~VkP|HdeU+TxS-3J+!+AZsllst}(o&6SXmK=g>Pe9^a3_~~Z=F!+ zzy?~v!YNS4FUp3DgiMh|7``AZB}{n!Ibz<9c(lZv{EZD@_L09V}+?rA;bZw8?It`k^hs;HSLj^3S7 z6Nlrlv*s%lS6@PN!v#U-%Lehd=t{@}I%%y`&Q7K8>t``sZ#cN}DQ7{kr)k*~cMv#> zt}hlk9#Zx{M}JyJ9}`|culE^h)xG(D@?BDcwjPk##|5v%G zYmqvx0l>~`hr7;%h&%>d?eO!_457d!Q!~YrD@yT65Y4){3n?OD`5fu_UClh-#y;OFKVK@p?|6c(J;Nkd z$i@q)fBOswtZv*rhAaDSLO91IuHd=r+3(b6Eq64KQBfkB^VIP9%Fv#u?gXu4>%Q|+ zzVni?W0Nscn~h0lq*-%-=kh;d8BgCW-t__2Xv5{y3%cM1T>v|;fS7u3*mvjGi|{<# zmR6pgPM=pp9$d$TJSWP5Bu6vD9mx3kekjCN^uZI*p`)R7cj7%H(Bl@_P173aX%93k z2-ng`8bR=9S(1HdYMrT|^!;8@9`=Y~cOTZ`LiFzh)8F{p$v1g#;IdxPi* zev1du3)|w{`MBqrZI_9htI(MQRfE6lHnn4{?_b5suj(8M{uMuDm*0`j&>ok^VR|VX z{qOpPXEkJECTsLe^_NmPq@Q^Gw|Jkr&>z1F_3CwU{N+>7fTu6g#zJcyRB|>ZPcppb zH};*Ll!Xifcr;xxRuk8J*G2PArr1h-dAzqv#^8pYAUwdz$^c9*$T789O_xef!a>Cj z1Yd?x)ya&1Oit5CxSY(?xs!zEf>fSrnK5Y_v4bHCQg|hc<0H8g;k}x9%q~w4ozHEq znhlKjRA>GNOf^)JU4NYSEebkzfE|6vNlZg&*}msPo*friJM$mY(Qg=r2c<&h%NI(x zOXZ`60GM^>vc*Y?!4c2!k@I&x{D*hX+}mFTJiTI4!5@`Uef&0E{gk+Xtr?MkM7dB5~;#fRQD6fk${2($pHg z=lS+3P)er!s|*DRqcG+Mdu+6x5CtZT;wwp>s7Xm=HI`D`ozp#Y3%lDtTcv&7#s$3c z{ySR!J8%^$^|4wG_;gLRH9oTTbiC#dF(tj25`j&zsHG+tcQUBZNS8EH(lSS>LKhwl znmbjJB=~Vp97n}d1WZm=e@_DqJXvnsmf8S+U3^jk%UZk3A}H7s+yfY(3|poQutl+J zGHy|0+_NOpl4L#j*A{ql1p<5kZez5As2c@+h=}1c$n0GN+y;SiZ4jlrg_?+zawSk% z^1nu+>py6XDt6y{HU}zS$IxWGIv&$L83e~3_#vB}aSWCeXWtWbo)M)azjeaVeyGFE zY}Fx8D54mVHZjq>H;KwyaL?t6*PXOMHlrlV3QrVqsg z*1!?U`no2@4Q!>0Zyb~6HDpTUG20vfe$b>j5&Ak%Vj}BVpkK+r$adO`No|YXfL)pElBa9t0QZK1}XhNZ% zh>sRe;JuCRa~{37{o%t8lGGYYH$j`Bf-65a9@+{SNJxDd09~V|t9yB|3IscZrCSc< z6PvJWw?!pwaUkco#w>8WPWaaHi37Q&o){Sc1J{-&wD)5)ja(r*S8vp-3ithUIATFr zW^xuHczfbK*T&y-_nOt~gOw9&N;JQ);qMmp}z-W-+aXot00KY9X}^@BE4T z@2A^~^=+f0<+u8N>gZam&mQT)8{nB@(Kkhvfc)W`vN{TfY}Dm);r-1p!#XXp_V=); ztB7*bsNkfG@-}ty#s0}q`CmKQKWdJ)@BxR^Iu{N3>nTk?X?g6|;Z4I{4xR&}d zpM9V1_5|dn`vW&xRIZ$#x?h+*#G`j*<<$SiXL*ZsYkjov z!~2hPwAL)Ufa9sBm#Kd>9ggZ_8saIs@O_h3H*uS8@8NJ1pZ*z6!~zw)dXI9h!zJi0 zpx?1H34F(WR;R)jls?cD*}|de9^cjL3`Z{$U!`-~&THFzI!2dXqnE!6Pmd1DmE{*N zr0Gng5MMTkzRLWwieP$s?pK>i8W{?8}qNHgLDNL zRkdg#%1A{OF!qyle$oS1z^2TJ}$T8USc3>E{HI`1+%(`#fvea5xr zGy@*Hf8|$i#g~ko+@yoIj~ir9xhw)kFR<=n-Uf4hC%yV1osP^AE#uO4qaHek$^{c7 zRTL2srzhF>eKpUOSb{^fKLKNXK4&!nC@tQeoIag!9Z$}mZ9crb3i3p`g{lXWGDoEI za)Hhha~{h&@5}yThWe%sZ90!(0<}x0=WzsYsN)Y#-nOTB^fr-Vr0^I&?Z-Y11_^Ei zVG&<#q}0FEpUXr#dE*8Yf-^kaQ^gCbx5OMZvrq0fso@8)NYj)I z+k|&v;q7>T(zwBiGC#U%Q)6oswyl;}jF!~XChOBT4UXUep%Z7??7fh?BVxUzM5P@_;%W)pi@}Gx0i}EJ|K3%q;Pq%ILH4U9rhc&Wk$<*@XA>< z(V}HdhPQ<^v5ncC?n3!S>sOjh-oHZje@%xx(8e_o9^!oCMmzgdr3=wKx(An*)7Au> zGm*&-rsA4p;8J)J8UK8JXu-L;72j``R z`elRq@E2#h6{3*^b!MV+^F;MILY53P#?h1(DEn~A!>zN%blj)AoaYPBIojdaXLChm zusxg=c0R_13Q{xryG-s-H0B^z?2~f_eawwp8+%t==<&oh5M_CCT-=Q*po)lEc-l?K z^ojxYmuRvJ>IfaC-V(65Ee|H}1MKDkAXxxfqUZE<>z!G*QR7$XdFj@VSmWNabqW^g zDoWXb!-W|W`Qe9*lHfljF4j0Lo8>9euc&DJH}L#F9m=hQ({@7AigA&Um2q~mMI7t_ z4!j8B7L8O7N|xS<&QrOUT}Z;eFFia#&o`jwS{&0{-RuAiCqf=j#X*D{#em6$Sp1in zGJ`8Mkt;RKh_|jGHCW?xLQq-Xb_&RA^Tf0P`j;KNwhsoP1d2%NoAF;VsUN{!a`~1? zcX~qjqqu(3RIiScKEDJ4^rD;}Lm7r{L-fISUEm%Lw9vU+1UB&@^gNt*FqwroMW(yO zD~TP8i|d3$J;SpIEvIcpxe{KxW_h82oSUC}>CD*u)U5h2xYvG$UgJ|Gigad6-IXZ& z9SjBkrw8*H7B)ige^{1vC^-o(Z!?uUJf@^_uJbg>y@WJFJ;@uzcZvH7}_ zh}6w8pi5~`Qrk+QC{82jz8G}>7YQ5pJ8^}@p(XB9vr4$E)#yIp&I9RTQd2qF+VRdq z)PScq7dKL^{aAY&5axZ=l4|o2Tt@~kqx-|xGt2L}s&(|))3r&_Ae^s+A}QI*MZhI zn(S6_Nq(Q67gq;vecr5nm}97>Qj|MI;WDx?k@dK($C^~~ zj512CrcbS=OkhJF@tMsIFGt5RV2MRTx5jaI(YA`FYOs`-Rb|%3+<)!HZ!PC>D8~ss zQT@I)wH-oP9z?6%N1ehZtGg@r+TQ#Nb2JB2{`chVZB^HJH|v+tG%Q1Ag)A#7+jbgW z$gVZ>vf@$I-29Av*SGz1wrA7)Ko1NR6P@MKn#r2|VmWV1ob0NsD5EO^`#{PHK$$Nt zuwrnHIu!MHHOP&P1OvEMx9^zYu0B*l6m)iJ8+n##|M-r_<-=#8H(`#oP)URhK`#8Y zg7APM{q|RuDiy7Iwul@ys|VHCb|J1lzHm*TuEicHyTq0_cwW%qL+X(FobZa*7zBxP zRJ=zI^9bU4TMZO$n0%)kHSkSRKXnLjv+TMeKwB)YHb?9Jo|zS`c{|6|(wwEG!ppa0 zrk)bLX?en>JTdmwk7QMI9bk7>w^k`i3lgEv_^+}wOgRCS?c?V{E>#)hY(Fo4qky+;T^^++1X~MUu2068o8q6`6 zSmX$##ea#k$0hEaR^JLz(N*OVGWs1~f-DtgGA*kff(s@rlRYuKKoAfD2eO0<^bbu= z)c2(LMM~BRnKR&}@b?j}mUGYPx3yBP%La@9-`X1&CNfAD?4n!9Sa+S5 z&k^@j#51}Ci@cA;zX@hg^pU9!U+{XQC(*4M|J64`??M{18!!KG;`i;+yY zvDz}$r*CgoP3RoA-=~UeSVZY=_d%0-84&GjD6rHie-*}EGR%Kg{Fj0%e5BiG4Y9V| zqWzn;Ci68jD6w!>Xurmcy2zI)ydW=G54rGLVkT84jo^DC0heYe;=A1sGV+BfD9opC zJ*d!@sGEr@dP-^R$6xG4WXbX%0j=@``Dcb}6GA5ISCpI?bxj+XrEofo_o^1LiC&fO zsp6w8n3u5(B#a7OOX>_x25~RL`zswPUi;4L)rqBC%wy&d87%q)W#M)uF=BS)LLOP5 zWP^%qaoYVhorsPk$}i$3!Cz)x#pNWf#~-T4rReQvd)+yQr!p0~ox22%({1-g9sTPf zq=Gs>F{3#A)WhAU>g3W~2@whWL80U~%x#<+M!buRyTi#sAVjC1LKGCTiY)MH2k_Y!N2g%fRtKrYf(j zMicJbq8?k6>=LU}pE_&8V;i5ONzEzXd8JZBl6P+IP2S9lJTD23*yQSc<8OP;a{MCO zo=b=lknu&ZV9LCP&aw-vMe0V{*W+h3Lgo1#iI%J-U)lKOZ)FOkf_h!SZ?p-DKTJqJ z0LU{U{HHC7z2AH@4L0n*;OUI=NBKDM(Z2IGvePQciY<{qw4c?HsXmlDaDQ@$Aiat3 ze#Q`{A@Z%tcZV_Ea=&xWrz7-G>q)qu#o{+DhU}SLoFhMAgIfga^1{Fp0jnJ zghl4-dYE-5l|MJRdqQ;@BfG+*lBC_*VjMeHxjP5v7t^!GruJwe9Li4byYF8^thTQ6 zBB<4-T61JxS(XwJYBYCo41aE0kS&^HxJQuy`;W7LCcUzQ_V|XVY)q@us*696*+kuh zzPl|7eeVMCc^M-J-*rkF!J5XMsuO-_qT0<@6Gdr}F$TF;+^bAAItyfuPys@@K&|)V zBngYrAzX6g$cvYQ0_*3PDBh-S-mp3LaQeUajGH_Bw}Y#-C|5hl>4C*k3TsMZIk9{>io6*hIIwraX%-yjtf8Aex`;h;j9ZHfS2MNq!rzPp z<5m@hTMsh0dcy}Twmz7KYr0fZtJU#ae>n&R-G;V!P<>Mvf(r%z%cPcHDWFP;3ToE~OPlXG8-(auPwnVf=;oaN3As7~mnw@10P)_;c>OVOf* zU+H%c6Fs1&pNu4J;Y&sO)btK#&-}Ck5++(EN1GF>W-(Z>F#)cP3g6|_D{J+OjkGmp z>9iLbx}PB4e_`6E=+f0N>7U$n;0S%g!Brvk%@GiN&(0o@ zBqa>pqMkKhAw%~tx+p^`56z`S$WDBp=|oTa71bM1t`6>*iInc`>Yb(4Pc-Z(=0&pP z3d6x*OSEQV9w7J;!+o8I$IkhO0{zQdv>@w0+PgAEI_uUWZv?F$c1L}Y>%K`R-oz*o z3Qo4$Z6tyTGznACYSi2t9;4oG7za{1sC(8t$*FDPMfk*OY;SEFXu5J}$TG7U-Q|B1 z68vl(9PAxbU33hJo?lh3kscgp`64Jp{8QXwvG^T2HkEY|5}EZK^^VusNG!H7urkzN znZhydx9sW&4K08)#j2sy z4~eqEUy2tZLtj29{op{{kH;gw?+JmT9i5{e^5Y!HR$o(3DJeO@r@z=7s|V|dM4{Mn!OBf$qv5oh>ZgLXdg za{dxW7VCzxy{E9RoU5qsZjOBiT#D-KpBved$8^SNAKQ9eCJM>fl_jDB;Su8V#ulq~ zTvnz$ER!wVblkKTYS5ME#%qp{ExRZIpk>pft99XEL0^nw`U*e1(#0lN$!l0Gd$Vg? zTIfLKnn5SALA^V9WF-Tac~_gHb~*#se6kA_88}e>L*@aJFtg9uH$-=L48E zqt?NXyZ({T=q*$6r@IPQ|BPx`7@x zT332;h5ibmhXrfEqxOP=Z2uD+xZkZug{A`pk=aKgcpVAz~8#3T?U+a|hFn*3!{Ci6hpSdaEIVKs+|W zSvH}DiEPIKXz}V~d9+ZG2j4-sES*W>T2dunPZf z1)jscN?(eigM5M*VM5qqc9TK@SO3RIV>_8hc3KO*(@)m*l95d%s=P7n=IY9IZxlef zxA~L{(6)Mk_?PNu)fOTr^{bnpaOAr$CQ+ftd>9eGCQl+C6DCgwfq;ktKmlU7UQtTr zU0smU;<;y}>M!1KoT1CTxLgWyHJq73`;Dv_sw_>u5Sr=QmmDj^4UaHRgX7IoqC562 zrvXx{5c24VtnEzn(t}7=;$n9WaidGE_=u7WRlvRcJo_SmbiUB&v)czQlW)A&cCzEW zVR!(XsaX|lvKq%5lOTcbPnj;ld>vyp9z8zgJy{qow;hhsi4uB8juNbtI|-1%8@!DC z7u#kL@^QR63!OOB2>{Q2z&GZy6LP5t4#C|exVtnE+^unk#x=NW z;~rdsy9bBHg9d`T2X{H0UHjd&&-vE7>incAnBB7;z3*#`F-ZzGcUFU>4!;;x^1F4f z`Jhi+{**`S_(9EGPi&Y&QLIkG7!Tl6|7b%L6t-A!{d1p|AVOK~-`(Mre7u2ju1%LU1q8;fY~)nrj{rtIF0uqUK>0^8?llZy2M zF;;OAIOsusTaRn6e*9kd(yCD+uy$603V{0C2ZwFkxnw#TFG@XYeR1*D(S%qL&auJr z??9623c+cj-(|ofEg}x~9m?JIMhP1i#s^7&U!jF6EILn*7O=dn;Qd#9(tHpzlKn?lEdMQ z*ZLl1VRLEeq1@bFKJuL=I4j2zE^iIYP^@`R_?zRD*=()klW{$q+a^F3-r^f^U4ZDo z;kugUyRu^i{XfSw%FVo_B>=6E<9!ov&$+rDZq(A|jvXdB2*0<#^FoS(g|63aU`Kmg zw%o~Fl(YRB^zyEQnqR8yIP9K+$$H2Kv8T`dq&o>I zrbe+CXU4%t6lShqBP2RNpiY0Y=>P zuifdG-93?GbAT0PnOUfdAOCSBce~WJB+?uOwV`CMU-3%BvJ#Sb3IBvVUR3#>%0Rd> zw!h!?#RDRYqfL}T7{E|p>yuhJ;W8zCFrB5`5ErbYqr}ob z2n&-w4g6}gFUjnxcVH=~uR$5+xSh!5HbE_m&}_eQxNbuOV(9HxRQ?*OOm&j<=X}9L z>jPErt#JQ4KGdp*7w!hN3z;@=1@_rxj99uMouxB47;>LZx6`4`9;&woB|krK6}P~_ z1mp@|eDMmL+V!h}Suh&}U9B~o{bDzplcOG&npMEy=&&uqPKQk&!a$qvaNWBI{lVtSJtOhSb zxw!ZexjFCegEAb>f|z%b+HZ=6ik{M}CDlKM%vV@k%b@ggQko0kN0fr-zXbzISjrVa zE*%S*SK(>&AV#0X;M5(>Q+zAk3cTDH{{pY1763yTVMx@bwk$9(OFP{%o zn4;1K#|Nnt9EtJ7+;C8>DxNyum_Y*30yiOAFumL~|LF9wwlI;a7?$3EJ)%Hh9iZk!W4-*YuyPc0e^Yd)M&%OM*Lh;1%$yTwY?RQ&Z2B za5P}TI8wYj3M&J%j)EaRi0x#z15STpzO7>Ns5%}Knq7xeu9#CpbiW+t$vh5A)H~z} zOX5u7EON|#)|ZH&EVjPryY{#$En)@Nzg|GIVf&7q%xY7T)M0Q~jiFXld4Sebe!nzd zjc<=j-a}VRG3nRdp>CErEJgO-p^GKvoG!|$BB?VPS&TWgWgrzq))}i#Bc6^%)pAjT zyt-a4!Rb?b&{uUeN&VU!urj?OP+KtAc~1&300kW6hUimW%J;$5a$>QW+^|5`Y@wrU z)YJNXrR2PMv*7!3Q}P4jMkROmKXlpX$`>K02agA$S*1#*Kg`cgz;H;|p)EqHPP=U^J;Kj(Gj(FiT_{6!$XZ+);c0N` zo6i{+ZL~-Y&Lw3)Z3+FO;Y>!iNgznLZ@xgW6U^Vs!U}=XJRaUpcRv@ zAZ=t6jZu5ua$KO^-%KV=D!|~dI0T9zCQJb*vxh=2(2_M=BmA3SDz>wUak%VuqL z=$2l=Z$>pjA;er~MzBXcF`O<6Y^C~*IG}Ul~`AVl4npz%h zGDU)&cr#^=GE}3|2q63z^jPh;eNExKRCY zTqwPh8s>1}NSe7^+S^CG&9U#=0b}H~76TkP=g|m4X92=DCc-J&SH~8@zZWNVF(g1^ zB1p}da37SZG&oOt5ToqMRVBv6ZBx!zrv>u|7}FBB|3JQDJ1tw255-}6Kxw8UQOaB$ zmM9($5%Vt2W6a}EdC`7QIeR0(U_EOsvHGcw5)SaC`VzC!8;jjaW3}6_;hM^ zw*tp z9V@6)CWBsDkEF4Y{6xR9Y93=0+(}DNOS+) zzlL5XVB$Ar#xBMUw&O*C6|*PsLH-AiI`DVRl)rU2Juh&sb&wqgq*tr~moHJ~bC5=p z9hm^girg#;=$;;XhD2m1%#sVP#{Ja3f1!_Rl+%W3ricxsaqG`!Q-z0=U{2z>(QtD?HE7h zkID-$>XRH$VXBb@wv@slc~=+_sb*?{gTTCBeJqlTaCCd?XxtqJwyeA&5g%2Om-^XGfkG3h5%5L_vIrDDh~3_UQiM{=5!?Iha3)NGlD%@ z^P-fVPP)zXV&+t(rarl$yvJ_{g0vsb)9+)+N>~h8!Vqwk0T@FQRzYF6Yn(&xV&wMB zi4;QP6c;DHtC7bI4Id?CS;|xZ%RO8u6{ro@h$IU}EQ>JA(GiEYnGwY=ga1t7w=(<2 zub!6{FQ#>}!+V5yxN*QSL*XQcp8ULovShY1XtMf4cm9Fyl6{pBuWp<__Ao>}%( z!9s_~S*7wRffPoJo<4cP?JLs-6~kvVA+^IkOmt`1Qz);7XLuJ&Z}I+Dyn^SIji7<) zT(*1WSaebYJsXR|=c3l_yO5&XXakRXKs%&l$dpyZM6QKRTGuqxlwDu0@YJ@$cDJHi zX!_VxB-$cJ*yJj_aWR3&Wy!2u>zf==I$%rGAOEE^zKp+PAXJ4Zb^9Rh9a3(!9~4AR zs{NAckDy$9BM65#Ouj+3F>! zL}2IXa#9I>F8P-6C)7rKlM#yaT=AI@5f9HnNvYyLYToI+#KZ6Tpv9V0 zQ&6mrwd8A!Ksxrn!A9Q@O&3iWC*Z2BvA@0fXovrczQX z;gZhNa6X4B3Be|B>?q}~5&3_NUnGQ(MhQbwYe+3`qvfNNHQt3S8NWra!}$hIWd0Su z-XW^+I|D~}2Wk&)l$I)S{9`8jw7=X!@JPE!oF`SiUBi!hAC9LT;{?&MdBV+qquq=pQhLhSj5(3WqLy!q^T1bi}#rYLvg< zT4@q3+!EshXDPF#V_|y1+j@l; zqO6HIoz#ph$Vi~zX`t%J+-3-;YxPX4+Pg?s*_vnZig|@;9w(ubTbVh%m{oLewF&)h zO0f?{dCe^#&Ia;Sb0eYv=2Q*CqykA|QIRg8$el0b=r7Ytk!3mW?Nse$f2cJtl(p_mRB#-VBt6AKvZ=wI*K~PbWF(asB&9+CH^*}E=qOS@5#Cd(gqX5l z*4n4SJ;mQyoflq~%m5eszhszHZ>9BY&j)E6m8`O(`q5~sCdzDiuOA_jh@iD#Y zXaOq9ws?(tr%V!LvV?Kqrs$N&)`l^&)@VUJ(I#xER;{x2%fZK~3XPPdVHz|=Nu(95 zk`nd;X;|9Yr-U*Y>hC^Uv(uIGODg)kkt zjxAVlmg37EH<4r6E~UcRmyIPhgkV*J78+V^#CO$m0!&CeyV+MuPJ1Z(PBwdNNC(B) zSs}Dk6nSc5FD*nXsT_pt^K{9=7>BWY9b<*61dUL|)OCP5FZG+u`Zx|)1gshAsa#&J z!;vJMp>pHjXn08V@)>dQHS6v(?LdrjTCn33eg5OH3?DPd%KjAOS^maseS4b`bJfXf+j4l+(NxXF;|GP6c;09DM zkPrwEQGKNnBcm-!WD!+4R&s9nP(xC)xLB|W)lt3HyYFqZQK@@&=MKMTZ}gvzT#lk9 z79|U0A|;(_ibO-B{OGUBtr`{mleE4vFfcrg=Ue!yOBDdbBLn3{){%uxfO(2dnM95`u)Hd&jx?)qA}=v#3$bY ziGdFYIv*DRH_rw``=oTgZg-OgeBzvg6C3RTgQulFrD^`Ed=!Jlj6>28tQY)g z*Gm4)7fc0`{j9ebRU>IJYm&{&D9FEu)rIHk$niCxTY@^hb z>4${T44y$d19wAln)t%f0_p;+Vy=6)$25Jiv`Av7{X2Xw$-6YjQ@dx;ig^SZJs2u> zO7x^{wKp3A9mkduuQ6$dQFXt@bhG5io6Gg?Bf{jhQea_SJ+H(6@cVYVO$ z(<}O`!@P%L{rC;WwH$CAxc=0MdhRX-KJE&CwAT?s^KdespO-a;UQIv$+l8y!DpsV} zYphdiv|%;no~SliZJ6}s3M!@Y?%MNamp54*yx4}`tDJQ>#=(#*Gk`J(iIJbjSc;9v zJc4(3_82YGZ7vI1L@QQhO8Hfo8A0I_xjnQFT^hXa&NhhlZ=t%oKDsEw$3{b&+e#Lc zj4$j(EiQ2yaxT>Mss{BZg9a*>nji68fLYggqop{izCt&DWgAGBBO2v@iLj^E+(+K) zYhDr6bd5aQ+CZ=#yN@}(J*wAK5TJ?vJQK=B!+ch(LUXAUOMd~m)bn2X{GN7Gr+SBc zfk>qLcD#^n-+M6Dr3-peEyS{~uK_w!LD+C;V?|+m1fGVgx8D)zyk!T1tNpIOSAO^1 zbN4I3ydHaL2jKW2PI>+1Z}D5Pxl@YHvNWkJ@C$(_8UrW$u0@<&C8|1j^=j`dz(%!l z+xc&ZUSda^Vt8KGpi15wh54j*l^$1lmdiI}@$U|DcjBt?&z8L+Mt#TVe>u?XBTqmH za5nx2xXSP{Mgf6!7*r$E7u==EC-mv6;&4~OWXSek3nCmvt-D&1M>dWzuTL}@I<9AP zVt<(X*ZW9b9G~BgR%{fG&oq`MZZ`NlzG5Y8W_gukPAP0`Egxz$kJ!*ciZIqf5x=Hx zKC!hXwe7#p7Rqk4Ci;Zo2!@eS#SPoLgtPM;WSfAi2+91VRK?T!@PTJ)TtmiZesG<6 zZpS~v6*Y42v~}%Tf6Xjn;nph~wIa8H`{6Ii{?`VlcM$~9KLz22y*v6;>7h;7G_rqc zi`w?67>_!<+4yA}qlGqen_kkMascPz>lwn=sWLo!n5y>gWjxJqfS}eBrhrCxy9wEY z*^qk^6>Tpn2C0-W*YVZ6LGF*a@-N3vS7ELtT=e0Ik^XaRC#?By(%F2|Un&Pyw?5sz zaXs}KB(v{_3m?~yOA~xvT^8%umt?tczcJHpxD~7DVd*Ry#;=vPN=s1r1ZC(S@SM94 z&M#7bEN7C~7wg3k(nV1Q&G&Y|nRwUQxI^otw-fGN^@4S=P3jFOg*>kcD5g45V$W2a zLCN&G@7&g+>9kebiyVFz!^m^5=A$24;uVu4GF@hN9c23SW;uZ1vD2Hv=;#Vv;MO?n zDc5eAn$JlT82rDXgWKG};Kl3M;kJ#mUS70~LZ(}>J;bxzmqZ$*8P*_TzTl{Gbx(M( z|7|%t=WXu7mPJR7dA2-;>O=LO@augQXK_NU)Vh_Z3g#`1tS+EH`(a%H;6RJ_JleMQ zd|Hwzk9k|o(X{J7Ds0n<+_oNw;=@U}p&owayX7pfHs_&j2rmmF#8Rc5sB=qLlt&#( z#P}D{^Ff6FB9f}rE@mZNMoXGDuVq@n8MUvF7NH}fVp=yuU=mp`xX*&}3XyS-Z+j4A zE;+R;dt&-|eCPv;-(oKYc71;ArM@CVh{M6`EX^8T?aVq z(VBo*Ho$ohA18kqDfN4aw~AQPhOIg(7JCn>}e5t*&WqQ-%N!GB{@)5x+0l2U@FWC z&$3r8i2l_jhV0;p(q`A}7*yww!g0(A$>FV0BBhPH0R#i-zz{hBw@|HJ`ve0n2YO_E zf>cuhcFWHfVxfK0oqzHTh*Zml5zvTwJLWDYH^yojbNXW=+`nwMN1DJD?eMR0FnplBp3M%? zBh5-9P>jTbplgwnAcWRaf_L(5P+tt0U3iXzyPGjO8h)fu)17^xAla%#>k>A<1Bdt5 z)`RsHc<}#FX{C-6l@DCVYm zbC4&f#<#^_*b+gth_Q^_GC&)r-bH%#_2bgdxu`>nK}F;kN>_#+IC`>&Qiu@T*4lBN#>9V3qO zb5v5Km%J3LiJ$J$-~hh;>%$dP>1k3aiH^&v?Q^zo#6qS6-$8T0^;$a`n9%OKsa)eK; zpiD!k1`$Mr3^#_?PZ25#uRu#DD5k=X&CkO{*@VauezUvSV|&r;I{QQy)EojS$Ut22$;Ygsj&}`GD3D3%84$T!~oJh!H?Ib@l{Lv~zeJ%L=h%$_PHOY=g~~ z()sKM2OQ5&csE(0S6vTqRmnf8xl`+B811BJwF`V3;KC?_40d7{T9?VkbY^Eh(5!c* zKTPQwn?uMV@m9HmUaFTd!~X|mH7gRO z5dAgF@(ZK-#BI6BSE?HXvN8O&2y|17>NC|W4-f_ZR}DrQeC!JY!aC&dbop<;y=HT> z5)B`_<}Zs=bW~+yi7F$W1YP|V!`Nnnam-(iGiv46CqJpeERuI(u88)t>zhI5J{$AN zQO@@(%T8+(Hytizj>3Afbty*%ge~Mm>(0We9@t7LAN!)e%29kMEk1W-WRYJ?{Zqu| zizxZdu5^51glZ;=Ja|@6=>5k&l8JdF1BGzn!OyG?v$ExZu15~9Hx7GTeB57zoO2R+ z*>d<|d0Lz@`((X%0nI_?s#RJvmj%KP1qs}opv^zO)Us+^CvStcO9&oT+ktmB#@Mlj z*XoAwUr2W^!WUfM)iD=-Tm1syapT-uwMd7eS6C8r)mUx_qZSM!3! z!C6Urz0G^GRDJWUZ(C7j={3N$28US+qRe9IalQ1Aq(<46t7-YU3DbrMP*m~gX5e&E zRChpV`C`8bY=$=-gSP5O>3f=fWZ|~Dc>??(@n1=AxqJV*t5^x0hfZmaM%x(y4^kQOJ34^bY=u z^jIbpbMOOS1L@pVu4_4rPIzPp10e&1qsXD*U6A8+$tXJ8AM5>gaseGOjSj&U+3*1S zj9U$Q_65OSFk-ED%xK|AN(cnM~PcZh+ zjDn|jH$LKrD0JkvfQdLe;!T;NBUQz-+A1v7=U*xT+IHi(No-|GOqB4lxD(Uh0mYyX zw?k)|(Zj0JnbbI{Fi@sE zX856_MyW4R@_IO}4l|?+AFqib4oee=wic7uxCgB@e(3JZO0M23Q40A#(td%9VQ~+ zrmF=j7i4Kz?YMBiKV1wdJ3VEe*||?UVXVQ5+JH#ht_fYirej>f`k2wKI4MC7nGr>& z{C&)qk?BcO1%W};8P|JN7f{){>1Qe(C%S6|0LU({FTw{14xeK1m+)%<_&grddJb*y zRfDK-4S()db5*m{BGGZ4BRhk`5u*tqI@CXru~T#kS_ZCb*nrIuJDAwUWL#!u?^w9Z z(iMH6>fMg2R|X-@R&>}e`c+|-oI#B`+s?}K<{_9UGcRmTmiHwUb1{y043hGL>vB;# zk$E9^9OIuXB+ax><*Eu*hO5s~UA4IxU@;wPJ>xq5kY!}WP{E1i^+eHfK|PRPFu4BQ;le%eM+B{aqoh*Zc1#D zK+_Bbp1#QwI=2+q4GBgcB3ZW4(I4WJVH~{)Bu^#F!&|5pmfG@P$X*|1SYJ&=i$ zx0$1~80yJ8I9qrHi$lYoKW-8!^qsjO9$ zWn=JSqpmnEO@lf#b#z=Ik;%QWtXX6!lGTkr)ByGsd1hv<3-rgPF^iRG4dDad^8()C z*Zu(*dTfNO#^G;KZlS_3MMfWfFH;q)Bj$aquiRWrIq3qhwNXC*=$~E5&b|Xd-DMK0 z^s-tAAeAkhRb#6xw>e`Dx5X#02Z+uSnv3tWF^$3Y%xk~7QyIu5swWwf~ zuW;TuV%!wSgi@MU<1&+S69|5l7hJ;mH9ck4b%IudGIIWr?GVC zMImWSLBR>4khfneK3yk4ZD8v+GUy+!AWCf=%Hi4*fm2&V=tSy-OeLZfzTH9z3G`TD z9o(5VWBnW{CkM#V1DY_IZAkbNA}drnr2xU_7QD$}_?d`gANzIg&Y9eLArcxqHqKk! zS|-*jXsUH8r5>K>H12Pre@%a2mu1Ib%W@s%qvjZ3`k<1IJjh0bZP=DqX0$2soK~b} zEGB1!j)fph3yotL#rZZJ z<4A0_s|ZXd@tN`0q@EUT?1kZ-;wcEZ`D47MpzXwH)vX6Gvn+0uPbkrgkHS}qr}ADqlnQ&NXTrCoqw zR6yvrofz^lkz5Clf4@jr{zatRWE-`%pgh;Qh@w@EB>CJi1XH*#nvf_e5J91Y`88>>xw$F(5|l?nrwfT8kM? z2gi6Tg_aLNNMbXI-JN!<6c8Pr>jcGCw^k#W0~*Q97cRIr3rEf!Ek8=UK>fXwC!6?= zUev3-k~!W<0hldZ=$fJS%|3dI8QF|!>%V7hyg6f8`uGebQpG?Z;~i7P68L?)x(9|R z|3SJOhPs62ql>L3#^QPGUxK@`x~JKE-*uNrCUB1M(}r0H=ha--fuy)J##cmKW(4 zbpEq(siPx$8gFEJ0v>Qhj*1(TAlY+m{w%C*^C4a2l{r4 zq{Z_Jw0t`d*TKjU$`#ddH^8EZ9_Kk`f>w(^(O*l)@a{Y{OBu;|^RE7i{%Sbx*%hf5 zK!ujJ*o4=EKbnA*n7)i2{+l5rR?C`ma4SMgJX|4eME}Bs9!EwF!w7*ay3Yhg^)!j+ zsSyr;3|+A`F&ch;_?zlfQrhyy47n8bQ==KQV%15GiPBY*R*2#B4s(nSi%C=F;5P$b z5vLF}Y?bmG)Y_^iQ!)_mbXG6!o?NS`C7Bf3M#ZnmuVgqWsUDtrdxvrGQb27YNX&7( zUPh$G@w$9yeNsdp4Mqw1r00Y)aQ;w6BMl5V7;OU_AR1!Ng{U}1SN3gZFCW+%PnarD ztup)exx+PN_XlU+o>J=4xIZv3og}7WN&8aSSe+!5q#`mp`TIQNQuV^=jiAeGZ0fPo zlaVJLeR_Y6EGYQ;bPvwW# zMuUn6NRFm#WhDi+$xw~Q^R;kZoYR3H40$$tN=827mmZwm_CNY2BhX>}yCCXU`%pB@;%Y4_iaG0+3G6R;iwgFbWxG7rG4Q2LhiO-#?m zvf`8YuG0VuyzGi_7$zJf%}Puamq69X-A+eyh*D~>oCA85j(Ts1*eTZK9+MiV?R&lw zP+==9BdiYCOKLo_w5-F@JHr44E+WjBA`lBLpOWx^l+6ZZb}|r|@ochR$p)b)O7O&*HwGEE7Kc?iS&l~F0lFuSX?6!Ylb5IR_c%ox2I=0PfT%}{PkLI8goQo#3JOVsS77>;iWufM7 zl17`T-$7|y-@0%}i~^f^q__-nV)WAi+IQFT^g~o|k7GP4yD1`|pf~#08%e)18UILs zwD1^F7al~|yUkz5U&}D~oMt0pa_?Zf(8L5%OdRn?bs}RN(^RvIeCcIl&!}%oD$MB8 z3JA$@LV0z>0qBUvk{A{Odei+t7-!^|(>i2GcAZb#EZtDG#{feXKPWMCxgX{rg~>O1 zaYzIm-EaLZiKVkZGPDHQWcyBa9<5<;&!dr%^Gc0u!9I&*2xylsKut_xqp*P}r`E92 z#`PRUyi`%=^mgqxCAzo~JqpL@@4Ig@D;uQ!d(fsSsp8CBQ}*`9p`nedl=g=3GB!wWQMU^i8yY6+ZC?~hx^tP@3-*#Ba$(0nWxa12|X%2xtb)dOjK5AYZaMr4kZLHR^>Xjo} z;`_ugG9or|>ILJZY(`rL}NrGzvEH?FdT<-7DvSYiS)N?dgQ4QT_S zE{_LDf;3{8iHix7Pwh6cMV+D@uxQ@dDg`h0J$WTAbwU=UB6E);xR4xh${H3#iuMAI zNMt+*hnq5+)eY`pVeLwtV>K|k9LHVSPJETHkCYd{;i3l~3?|i44qm8$26W`mrFqU- z=?2p+^AbN70l*^r3EDWE@6ZqJDgKNa+|-rIm_LKZ!=O6Dzioc$gfomtJZ;_7GpnNg zv>5xkvl0UnpKxW8361bt03LBNnt)nXGr)&m?}Tw}kge6L!$V9cstfT#yE^2pETrQ7 zpq&nEWtyaFf@=TZ`;tF96sIJ9~7004~fCr%Owix2QjU^zTDbu zlA4Pm&TG7oJgp=AZ68KBf=uu&!p~zib^Owf%wslVG3o4`>yJj>JBN?Qa5n$jfTD?o z4ygCFt>ZEMy=#^RmIJ$Z+9P4S+REvay@j7VYbSf3cO71KNnSQcmK2TV5misqEpj~j z2v^TwP(A!BIaHTRQ4!F0>-LSMY^fN=8jC|YBq+HfO*Bs1E{<^863Vpj_{F22l0Ii) zCvUJIv48x{@zpv72ilM;dgEoM%>YYLZy#Y?rRHYkJaB0?zZBRsfe-zITocox{S6lN zP;HJ8V&^$cZ z)b^uu;p>&X#5hr9{&Qn~i-vDm zqlQPpp+nVYHz`}D+Jv2-k`nShjoQw&7_l~b%{Q)Z^+eJO6QL)F}e z4k(D0AH|$OT$Lhu{$oF?YhM)A>kF4)?8q-9HVC#-vEEYx5qU2~JxE+hMIgjJA^h1D zyZ)j)(6r*2jW*UNT6bJOgrr1yZSluBG`;z zn|XWV_}i{BdAIZTz}!nLMf@j->zLd~mD`u-p|4x`grt>hj~@qeR&8DGfqK5a#G z_Z4@x>Fk=TXfYD3+@LhL(=YcP|Mwg^TjGF<>kRgcWPt`c3J4UhI_xv8xs=hBcGZ_P zGPhQ=7ctPR82blbLBI`IUcX@1hIT1Y`Mno1x#tFjTC)B|##qqG*Gy-5Gkc-NM-$Rn z%KlOLd4yBT{duKO>Rir8MS#~V)jQ8BnL9UXp}#)W{hm5RzHVsTH&JTpzhodLhIfPP z?RU}8wU5RrRhD(-awNh~U4Hh=T&;|0knA7M-v9H#=st=GY=`#7^vdR0&ST;*sHa>hCowAgs!A6C~72uPm6Rxx{Tbwl4DTyq4o|oKZF;K z?*nndU~OtO7i~k{zpV|>R3&m&4wq#VK~>lAbk&UIXYDB8Q*6p0yR+q!vNPXUsg?g7 z!l8;6u>5DYvKU|O*eKb=nb%UZy5n1M3*xa+Yi62?M#l*8S6-hb37aTw8-_aca>8Wq zjtgBE+x}llT1SV&$Df?&S#IBbvFSVuSzqUOqzG;GMq2>==L-ikm8V-&)QHzGDT3mB z#Kn##smc9wYKvJ{Y&O;UZw}qcf8NyN2bg97?iOu#y;T^c<+)`(6Xy^OpoY)hO`F~u zq4WH>(ZKvTc|T|4&r1?U>Fr2MZ{-10t+OkI;gzO#Dpwk_N)o#qiV~i}M{NRP1emb> zz>i_`o8s%own4uq(`K9hdCvdr$2KCG&eJ}nmyFt~P{$IbBxSTc7!x9ttzpRcU3VO2eKH_tjtekWXEiCmS{i^rb z#AqQUxgVBxtO6x}?@w&CZVmT7UWH-$==C>f3SgMoZ_vv>2eTWHHu5P9l@rFANsn>g z%|93E{;hr)-`5d}Z+VA;Y8*Dk8C9`Vf^K)B{%POrNL)nJ6eMSX4}fF>xT{j>g&85w z0R{uM>v21qW*c9fKc_2pj?SIQASl_d?z3o?+B%pPXu~onar_+((1Hcjsq8s|G8RHY zhW8_yDIwL&-ZN5Lvi}PELB~%G0-~w;@i5kENy(oo*Ge0@+dr=_e+JS=#9hp7QL6@) z$YCGH40iKdL~N>;bXGtqh;VFdR~0HxD{IgCyWUQe17!K&*It#*D`v7chl~X2@e3XL zAy>lfYI@K`9?C+Ck3TADcs3gAlB8c_AnD-fEC_OLenUvcDE`>2W1xBa_efLa6l2peP0;fmAr)Kz!ieDcG0*Ua!e?Kc zq<29?HKYULfuV2r^MB6p|1*k>hR}z{egV#9{pKxv=i8x32h=r9F#K3cWgoC!l`Z?* zDb}>st z*uHhgg}~ht>FLVb_&8f5sP%Sr6+9uXS9K5kO?gt_ zt#jt5HO?%|Gie?ejr; zJd;2E1pCGvsRFxmhsnVyU_#(>EDNFCn`$0}{tF#$TGMCu=Q>js)TZypeDr5?A79YN zt^p+#pKn30yLYIaQ-`I2DQ&NZ)h{C%>z}JV$9+~l3o@k>_jqDbl=RX+P|xwVY7^AX zGrEzT=(^+8Jr)&?`kdGp{tnq&{iL^r3Dd&>x}ic@zTtyz?O6$Kwm-S;ZY82M|MkEO z9Sq0ZaAQGEkP7|cYm7(0apv07u8-WzVtMW*Z>~uKqjSsS_^aw131;ORTwSNv-#?yp zohz-a*`F)wC)hPxJG7kPmDFk73Jn-&JE5NPv%?C&-}2TNznVyj;?bXQgNf-Hq+%od zckT9)cRzS$k15!L67|9j0!_p;0}{hU9xQC6w9(u5^P@o1eh-)>s}~BVWwW`3%)lOC zZsc#S`zrU&BdE%OWyu%`8n)Hst#PAd{@JmaTi64iczJ;eXJaKR9Kpj*}#ZhlvbzLv~ zSnuE%M94^%7FB1OVt^aagA6YAq5ABAf*l;>49BQRI_6OQggf7;y&FOC#;ymuR*`F* zu;eb6r>U{#JL0}*SwZ?LUp}#3+&CjxlMA{1lFKw$Y?2NT++p7V$)B1Ja+_w0a@v>h5q*6-e9}i}DOL zQC}X@+VzTXF52zg@glB%o>35TRH#(Y_Ri}uF3fNz&91=k4)QPa82|nK{e+_iFynR= zYeglpy9r%Jyb%otMH}7uS0zQZ-Q`?S7MO`QnO%*QwZ_f7uTY^g_trBvY^w1ArLU)K z!dv5(`PRN2#zFmOqaGn`SgSgNH*@NFyoW4hy2qrTb-;gFnEvn3!pDyRjVQ)E9eRV0 z69?Ijv{9CfOqx?u8i9Sw5n{1W@kq*e=!%pP^NpM+;kjnNB9Of@@hus(Py}=1{{LC& z{_lVIKcD*?)GdW#aHrwJJ+dPFLQ+kmJ(%M~CKiVG;4yBm+tu#f{9(n3c48Do{L4pQ zB8?k8mqntmkF|SU4c-5LpAhArT+Qgm6!Cm?I^o$=7fy{oF_G-}US9uH@SO;MnxKM@ zZ72uCSEi6x5$Y&S(x#L8?`NR@*KZWbVe2m%-IR|hy&8W@PF~;x_g9!*tbUAk%5%D9 z=V%B~Rqg=hAygU}TXGlM5A)OiWs3ao6yN_f)NW{C8KMjsAkPPvHyt|KLb|f{le=6E zZOiQ+rrcD_gt^ZX=V;wA)|UR|PWJ!$kkB7^WD$?|LTZQuOoc0oU0dCynayJU+nE0U zzStXrfXQD+u(}5Sda(Y#Km9*XKZb+#Zdh)2E8&`8O}$c7E9bhwg@r~myf)0|xu)9; zjL`j;9KI3`SdIwf;3@fugJ{5!0UVdK(F)zZYkKByEBn8n!YB%n(P2v7E9#Jj-0dDy zc{RU^erMTQkWLJ56AEYb|Mi~x%hLZn5H(p=0ukV07WD6;C-0(1q2OxxUxIv#H`d&r zgUPt=Kjf0Sp$2%8s!M#*>TlQ7WBNyF2i;Z1YSN`$amBAD8Y$xEhv}mIU&E(74^;0k znyn>`a!KP?2Uiv|QXT(#u<9(d%`aD=KQ$m1^Dn&(yS@6|S$3~>{5P8OZ+got4jRs@ z%DrHzThuQ*IFPsko%VibFLo7LZ(~z83krGwJZKB?2pXShYM&;mWE7P`D~9dMtl>ivY((D6+w|Lc9~*Rm;S z?Iv_)DizjyT`B5+u7x7&&^)8>cPL%!D3MbCjFjiN?xIwK5eSqDW1>;y|K9YO}&Su$+Q@AaKIfzR20k26w6W*>a+s6LXzRuy+Ue^?2bulwu{skjcs5BPuh`pSSP z*X>_HgdrrPyFr?vrMo31MTDUh=@1yY8wTlaK{|v%y1ON$JEcWRxbN(9_W$0q&%O7P zUlf^npLacLtzX@Nu_2TP&VTXi=`w!v(VOljLj>F}B!=hE zWvG^)8@$}6Hr6|CNR97*Cl%&ue3PQDgQ#I2Xt*|LAtu(nFkN?xFv zQpln#R%V(*w^OZJfv}XSMXz69-Y!+lE_VOU5%xOsUhO%nkC^W-gRCC_2@lJaQooVB za4Ig8SUPu|N!Pnzf#NJN_2YkkJO5pt=^SWqtLPG|F22^jQ{hu4_9sM{$qX<~hUd+h z|Mebvfp$&HTqAzEw45bY9*)3pd`<#&rS*)3ma|UFt$+47aY;lec9(G60Gpxn@5VtG zX1$th9e4{7D~atRVxftJcgUb-&zOP|^w>6S;OE0Pm0;4~;)j#vR9y;#9 z8*+4wP4d##-@jYK@ztM1nyDQnzR%)t>`U$a7Tr$7wM*3Fwy^Mn>mOBV)kTs9XN^kw zoFZ*=Pvdl^`d>>AY)<4O173fD+5xl%;a|<5LjmM--}&Yoj))I-1#sYlWNd>Pxprnm zg%pLm40tNvJ@st}Z+We+n=pM%PkkK*j`&d|@dgRfz-Dy&&yW58tIgVv{afai(vypN zY8NrTwJdf*-x-z8{6CE+{x3heO4fuc$t!s)*J?=+*u^LOU;fJ9ONRfxx~m9L5C=Wo zb7y?}1%BH>>ub1l;{UIZ|MRDpMYbEW|5odG@qNDO7&qbn?KShyfB4sz5`*b4W_UDm zV<&nGcrzCh5+^S}^=i`C|MATJ`}h7oKanfm9KhZFlzE=PakMSEQ-;K)7m$3>bd+T9 zU$5nVSGoWFY@|$p(XPYHnL>{2KT8%Hs4OZk zxogj3vfJoPkKH!QNYhBkuHu^)e4`h4kj#_HxYMAQ9L}fzfx)5qE>XI#VWVFAueD#h zl!7E$qaC%-@LrwXJ2HQ~*8^hkfm00HVOfLbPW0+#IF_^A_NHz@A+-yN?vKR%vMt-f z7&b$;fpYznys5+D)4u754y6jl-q9w5*B1p4;F%n%qg-0U-47CHbw>8NI9!CeDC4*f zI_PUB>n&Eed+?b2JYL;u%lMazXb-f=wp$EtwF2n|#fwIej~<*9sK05{FwEVOaH!h|NRiONswgh=y$qli)whu z6tKkLuW2DCh`Sz?y{LKfsX{on*qDkf2vdD$X4n#wpz9lAa=eL_5)TL5Jwv_JmV<Cp2JCJCwTq5Sd_>rw-o$JR~JBiGtJvc!+>xaIY(~30?I7NO1YZr4M8@~FXkE{4tgp+TQ z%-y_MaD+U!gDZje6?u`<1k0PlSl)rVaDbi|wo&>j80e6JYmdUCbkW}QhB$NjDyh2P zMG5XZ^pv>)3VvgnLSMzuYZ>kx?Dr*UK?^kNxQI#JS8gp|*bN$viy7{x2GO}{Mgnu} z8$@bG2^|1mzOe&F%YvIyzjjsrWNY^Kf_IV(?}e%*!YmZ={1NeW9)PR7XXb&?4r2bT z2X;=^f*c&l{}jx%TSHOj0_BDJD89trZOcY{1ccvCfCBR#@X7to5A1^~@AoSajsdOk z8O|-IZ5isDxV{8;K&ZKs;XL}%-c$R#s`pL_Eg=r?XlNE%27`x?EY(bJXN=I;_WQTD zKL9rf7DT$3GZ6hU)iwQu*r3sp>wsEM!Rn*JSUlGEWnnLILw&K!19a^-V5*xQZefZHYhmczZmJ>J{IMo}MoA z`4r-a`Qk_W-V32Hgwor9n3*U%>w2+$WPs9xM&|F`u$a!d_i(@Ra0?{ujYmZK{p|(t zfIx|`jl?tsm<%&-epLSXA+ofmW`2J-c7G`Gu=8*?>-%R`m`UkR*%i>c4d%*(+WDNP z`_3nOr+Pt~sw|4#a=5;;95=HiE?`MpAcD~DAMSuK24F)K*rZCBCH1W>76eix%>mDtxpg;uKgNWvxFZd_fap99qzw*Ipz3a;(oqehIG65fGQSPTbZg_^YG%AGKAVE)K~}ejC8YRI-5WTg&x`{XGz{ zu==7$@fJg`G2r|B!qSo3%2o8r)swd;PwPQ8}Tsc_)7h-ZB%1Xb;M^GqWc zXbFvpG{ord5MK)5AF~N$!6kdaq?>gvrd^aAvLq?UAt)yXa|R9NO6dX?Q_p@K01&?@ z+D}`&`oFrtxd%`|F`<>)E^tz&5ZRBvfJhATv0{9ILN`E3i3n_In;)?gI9J1)YslPL zbazruP48cfQg!$f`fx&hqSL{C8fkoyH_)m1yX5}9E$o+rFP+J&OT=yk5oGW#&iTvh zrF{TS^15~Am@Uuygk}U+9ZC?0}0?=?r);k#Y zJOGm3-GkPngY|+YSzCv>{_APspQ8HlM$c*(x;%h~Q`|icOcuA%yh)G$L`OXT2Zb^t z1Vke6_Fp0N6<-Z`Q_W!o^NEUn_w^pqZW>LCe-N zJ{Oa6Nix8Oca--kN&?&i_Qm({0d%g+KS;i(Q%4Yi!VKu`k>*O3?==K;#Q?10_xAda z0K{w#AWT(V_OnMIl)dy6Wr^qT#r0IWNk+ML)>XpFk8+S2MYKn7(q4DarV3Pj0Wy?Mr{n7p zynZz*nbkQMA|Q#XoD-5r2_A5LtYq3P-3EkQ-2w@{7U?0G-)(J-?5?ige-xA8az?tG zX=tslH-;aOan=4O2nn>^Br556w^_&6RAz@*>AYfsGS)#=Kbhl2KlcWlO4K!+5mb2* z_ap@D1bb{Z0W$j027*wB<{1E&pRT+qtvt&)1;{v_HD{u9Utiq;;hNFGzPCU)FEG)w zMt~C)V3aN*a*+n8K3NnE-h^)B)oFiiVGhNRQ-5mO-N;ks`w2uR*-W~`HYqo7L+tq6 zP?iDjhzq7E#MIy7{*jHb2xIZK@a;{Z{Z)v)1ElR`lBO#E1R|dByJpLH1AOFz_(82W zrj#ayC1fCku1XALa;}>X6``VDE$qjO^w=-&)6Mu7U8|hGZkBaD{O|@!J;2rL?qo4x~E*?jb9VMGDx@=a?I3 zqX|)rwDqPK$;CYbEW8vv$T$B0&#G|D;KkO0VI`Z|+=l_Gx1q&=v2m>O1}WxkYS@?3 zLm|r0vF2ap_Bxrqzbfx_pt_Dh@|@T_l0Dz?^c1`N zTv6pcRTyN)oOX;FO-SM*7pHz?cB6-*^$~9^;jj@t+edNU&GF!GfI*m_L9j=A&>*@i9;=Z_=h~xp-8pm z)5atHvu>9xrZ+$!vF~j3;drjhZEZ{|<3j?_YEW#KzgEx4s92`5PA0d&$;E4Xv+LoIu; zF6v3G&_5vK8;xqQFl;Mq=pNma&Khp5s3;|=hgh=UH1wMAgQSVg30~~SZp6vwh)n~N zc<|9VMpbsFo0lu9InDpFm+?N#(Cs!FDdFE856_%yrf66rhu!zb-i5J676D}lDBP1k z7Q$je-g?cpbd*%L!;EphQ3+!2Di}I8VIi1+rHV+%dv@1k8Lc8vLyXxGl?T2l z6J+!$3PUfnZ~^*UVaeoQ!z#dK*n1?C0@9hk=n4hS1Q5KNK+DqV-gxBcxd-NHqCL@j6*8t)_e{m|g)(gZ`7<@Us1!+oj`})W zA3(y_*u5=DFxF>ss499#%k-stFR;CiZiHzg)#hZXE&z1^m+dDjkvv;>1XhoYWp!=5 zBvkQGE_ygxoB6l}ZE8b>8j{rU<;~A`zX5nBfv|}9mID2nt?i8MUG6(`iW9q2M&6T+ z;c0Nj8V*=M1?V`dsBwh~mz^djb^|Z$WX_*e$SnUEeTWp2&8a@DL_Gt6rj?LT(mf0~D zLLp7Itnj&lWZ1A)$KpeR$B?h9@q6^c9*3XK@+9<_NV`mcd^nfF{&b@Nv@9k0n>D{0i7<%~q{R)vRPQS%YIF<@>t;U5{f7tM`65 zzpCR^SRo_@vK2w^HXW%hMrMY=;{HOMx-!WkSDPa7QvkkwY}$)w9<8{%iS8v`v2elh-nfi9BN-?LetV+USL|tbXf~VQ`!@&TDH(agSDmz}Hbo%qZ?yMQ9_UK_~2}kYO zJxTg{<|7NPghk7^moku6M#yV+yC#9NU4ps_&RXeB^qh_1rcMhxa-`2`)>T13`0~TA))9)&-dYFnF>kwG zU=L2XDLk^U$7lI2gV_zSMSpugE#2Owr09*=V(0|JuMlRYv{c4+IfV9^3x_DS1{CMj_?6Zk@l$HQf&N>9m)SK` zWKrsSo=OM~%bN<%mdpA|SH7k}Pm$pv7=>=0J4=7J4+w_HTJiXWK)EkXYweHuRUKa8 z{}^2t2L?zIG{eLEKvP)sKWg5+Y&%FG&|DoOi)mab;%WF(_fy`%MAalKl$B>qn=qBA zjHh44$Tl5ip5T2&_LBW`F7e>Jz>Wnwb}1H#hM=C9qo{-Br^U%_KgPE9@>i_&xBC7* zzRy%Zb<@HPs9?@S%#|nH>(DZQ%4vT|v*9J5MSl+jrcP*gQ67KnLm0YRWjLY|K6W)g zj8V`NxijF#qrw%{U@t$F$_4MDcPbiloS`_L_7JxLnDwhX$(|jvS3#o5uewL85a-6+ zoOD0Z+g#JJ2wQCQipgj(h7%X+7s5=He}bqu=az!sQt4~j4u@J*c}%Ia&Q@2oJF-GV zbA<+nj5*b1vW|2#q~P+KW_MGlH?RM-z&oH{<#axHz4Ajg;rh8ZJFS9&5ck-p5-P$S zL?Kx6cKKS=xo9xiF_`yx_80h#J1OMGSKdJ%_ZxM~MIl$qVa?F}ueSTyI+wE;$jlnz zSn!D*u-Wbv{uGFvgYtP#EoMVEtVjoBrvR#} z&1~qvvS1sKq}BN6r~Pqj99_53;L;0xBg>r(;~}fWvucT2Q>I3!0Cwo9bxD9o4r^m! z)8n)pvz4X&EiPX;sWh({FaRel-*Ii{N^FSQ+@?}*Ib>B*d#E}6IM);WO z( z?hj@RS7QM(&{PKqbqSJr*e)iwXX4|p{nx=0PM(m8s0B<>a%nNrMD}^w9>4Yc0fvm4cliswJciMR(+(SpSOL>I{ zY+~Urio~&xcrUZAhppsEvPp^`d9P^d!X_yGJlG*aVdI-`df2IdH86`lh^4bP zijfx$5i)?i1BUvVa>$U1o)0FPM?VHK=J846yHySDb&0A>dpZFNq+74H-(J?8YgLjd zgNO-9GdlqH>3`l%tLC&-Jy!Trud|e1*f_j*IEu7>drJ5gEN%r78^DTvd5y?W9W}WU zXn;pM7t22~?Kne70#FAmAHNJ{{tXOb!IXqos3%lOFF&WE#p)|p+r6u}Mp)Xo%hwKP zfZ;lm%_}T6AMNGSb#3Vy&6$=CQsl1?lUOR`tsIEF8lRhTKqg7y;K;r0Te3JKy4U8l z;KTOHCOTbxTD?b^B2UHh bK_yWz^(CdGOTXu+ zYU;irW4T=#J6+D(dMV_-Xw^{L?FhdZLGMN(E~;P_ci|6}xLb@=@`%k`s(F5JQC_6L zlfE+mKmlx>PYe|8vpJW_M<{*(JB!z>%la4{Xg=#2q#i-&oLsNqV)IvPMe?(h<6`gc z3LxKStX9ibw|k2Tklqd>Zv%Um?uIeHe9=1R$tg*eTwi>ui1ueqi?G z@mQ;Z$$@WPNkrjdGYR0Ri9c#i%ZEPIqx+oad8g8sac&sT;`oNQx)2|Ew40b{Y2NrDz`({6s`gdY4ZZv3Hy9#dZ;!+Y^X-tjZ7 zH%?Y+=}F)e(_#rSPZw*H6 zz__~H^L$wH3dZZ@o0Vh_>}hEE`N1yK%|~JG`8eCo*EOb8wowz^=V0eOv&F_KvYoPD z5X(&kc0m=!yeBC116vE5kY0kFaJ(>8f^e2ua^srupe2~q^KcpVqOw}Ds554Hh%P&e zCGXBqVHpsIvxO~z+QMQV!uY;LGfE0hYfYNwrJn8Wr_rZCj{-4X-Xy`cH&}g?mh^)1 zID(V1UH#d_{|VLrJ8ggt;U+>lZ>b+yb=#3w;%^tyL7I9FeNn?G3YBG0uA$kh?+S+#?o0iDyqyRAnXwx&pNJ9#71q(pSip1><5pjWSAp4ZCtE@ zI^FzRX~D6#<5hB460i0X|rxg2|S*&>;fdN@R=h8v*#xhG8P9TZM!CU^i_IIGv()fXoW$t{B0Kpf`p)NXoG zv`t7v@ZvP?aKwOYrlYeroD1h|k`A8=e8v;Jq}=;+E;slj4@OC#+C$CbNab%v$7#pF zgbIe@zvb-+o_Fx1%PINXNPKmsl_URI)3m6*AT_szKjpDnfEF1^tEJNNwQJ1S*u*S}C)i^HLCyiqd(-+M zZ?fzI{UhTJk)=mgC_gW_g9=PiNXS-tjhs=8b8BX21ks(GZx9<;zqIGV&RyiM;RC5x z4%SiOI2|u+V;WD@vS)J?<>x$$)3c?hhEzQYSAG9fKFFo~0Zh#$6GU8gah0O|rGggB zL1)G`d{Snk0Srj{(iHNzG}MH4H2q=xqB3a5OAGf0B&mp1^3||bHTA3-M5*%Um;3=T zl?l>Pt&5pl;t_62xa~LYm>T{3Fd{9?t#vPv$w5c<1n_rAL!IZmv&jw;sej&Y#+E82 zzb5lVT;~+LN*IFPl~M1e(C}g@t)2jlZy$ynV7E@M#`mvN@jnb1?{GifBPu|AL*7Al zM0P+YN+YD*cP)cD$wt3`RWm_|Ex37 zCVEqB;8|6usG3m&LF)@)tu!Qy_MLV2CZ-SQL;2=3}ea6bIG0o0y5#FzfA_C zamUAoCM);rM)+x6CfxdII^6@?amwF6l^B2!Biy$7n zghC@~`r3;xV}d=O?u{xw;52z#CLS4E3J+n2F8VsEzJni|+*lhjHBMgh?k0g79tEui zy_osfS!RT9RCK6XGT~~~7k;kR>};>hL(vqr$VWJ+E=W0LXZfV#*#P^q1l8z%!3;Z4 z<4x zmcKzx#i{koqp?e;)l1@#CzGIi$fv&Yr`Gs}MejW{X7cf;QpUGUQ-p}5$`$4Qn%lqp z;81YD1=>7mgW<^K{rm@U)P=ISIeh9h`B2jm$ksR${PF&JXU8R&-oh5DV5J*dCe%kG zbonl_(Dq2id6q6t#o5xR*^K>6jkX7cA z6K)8g2_^j2c}5{4c=2KD+oreN-P6%nyCaz) z;kDOYUo$|2T0c@usW5`_Nhgt%Z}msmk$tdkP1PLC9TDeSkN%kSRIB83zIX4+P5Fv) zh@Z*kpyA@Tnfi(+*+Sfo5oXk(O$8`#lt{ZSft#7xOoery zJUuQi@ceWlIhcmjZ1%XtltraIH%ue1quIk@Mbm7G*~`i_#m}fh_~O}5?bcGWJwe0L zjsBN~6kvD0s1Ak^_aQ=2qi!qYEId(bZt{)ZYu&VS^ajyjyc6!=7!2)8&60SAB5jP8 zEOOV{Jj8~SSqMtOY`H_jSeaMKIzue@*CGmtw9;i{f3QpB$+qXb`>x>ttS4F(qp6#F_Hm z3|EKdDRE5M(Qgl39MxL>1jgLosjwdWY)}h8kIFvEl(6Gl7KX%~f`@d@wC$YA6X1_) zX2&erf@l6 z%6eLGx){(>5)6XSGe)#v!jJS+*CdYBXNwQvxNw|QQMfV31Bz)x2I&8TtH5)&6R^*%|JLB{R}#&4B!#!5#n`cM@7qP?8fW-h^G(eeg9#Z;r1t1RG*LMugmt*@H=tM{O6(|!)+1fma&jqR-LdvWS@hKD zZa0&OYyvGQ&sw7QmC!zv&*n!kbMd*df^-}-m_6$B-$<(n{JV+S1{HKos~o(V(n}ai z?z9|G&=o#UQC>-p@w{2)F1~%4tk%m~GPNxq?`IOwpIW^vW_Tk%m+NWAmJ%VbM`o&o z(kdB}ux29v#sD1_1{YDh!C;ADeC3&k7rDJ8jNxr) z?BU3;RFIRp$QkL?E(BNmDw#jgAS=7U~lK0@BglJUuU# zvE^HiEQoc9lFBIjvu2;JSWE4Cd7V&DJ3axZ17Qrkud~MxXgyCSt=mjC;l@iunHB@e zG?J6o&9~A*D?9>lH$kxs9xRg7b0C#H+xmO8^8VMwsup4*`aMCJ*ps=pwECB*sqQ1A zKmv?+oe^t_w2Ld$?sCtg#YME`4$M(WFujR^jD%ao30N*vjWN=h<&Kilx~HA?C+ zJZP17t!Pv+%lM5jP#v-~dB5n_e;IQ*3`7J`r*5H*v=oGDdUX^eM}#rxTFa8Cbr^f>+hv9I(s?2%@N{N+53q$oI)36!f;pkXS_z= zA?UFZPgIa(D9!erIl03VFB?_P7h#T0A9yTHheL2lRxsf}F)bhBcfSIX!g zU>u`y9jDS-`&8OOS2OO#Ww=MBM;h-G{1(diGn7G%gEmyG^#$4rFFSWF_S3f(?i?+{ zp1H!vV5F;D*D$VXW0eqo=h>oQ3yWrA5mYCj%}#-IlLfIHMg9I9<;4QYK=(ChV&}mN zD`6-$4l8UjaCrmseo1cbBVY&{?(ckP|JznnM;SLO}K2@^!Z#tq(B%EELZKWQl(ce|6!v@Zl8 z%(>M9a^1+;BSZS%bt|fCW@$Ghot)2xYsyZY2Po2lKX>f|)_PqyRRd>CW|w`ua?92! z+)7}i4u)i*CKfwXB(ti2Kz%~bL1kFE-j#iNYS4y5aI8mENwug23GIT=0e!v4r*?wp zo*3K$F~PVf=Hcg2W(?F6{XtwsiT3#>gNad5j{NlKRQ8!iX4e9!V3ud+etv4qwB zE>re@f=^FRIDK31FVLu_bBbdkba^KXXZ(x!Ui<=Cc-SS@VO&XBjl|rgwgyG4{0%3Z zO=-W~b!cjAqkuV^DysxsBV8ak%BInsM?z}QDsNvIoE+`I4uMnB&?SA7hYWurlmp|K zc|;@;oa6^^k-ne5PS~dwb^-a;K{TSF-t9j)NcRCt&OokiTc%Tpz<#SWZhCmC;rkbt z zL~W2azj%4h*1FDS8$W5rT?1h@+$2Zvs0~1e2hPk<94?Ga4>cxt89SnV?|aJDR%_{#rcdW*ZxKO7!zc9kOXbP> z15%G%dVS$$Tda*`VKz8+8EXZMADtsAHm05AK4S_&3=ebovp{Bad^kue$%IgRy^2lq z)DcI?YWH5V&?j9|jv3V)AEAJ`L=Y%cO?5da_~XE$@Ccj06Cz0WgS3eh{9XK;ezUmE zRU*qf3V%ZBqvmc&L523hG7~}%e0lAWX76xyV~Wak(;%61Q3HBjR=oHBu!I4Gv7`sR zEm1)|tzKtY`tPi3~SJYOat*2yD{#Jp0vS?rb|vG1QOGBRxJhFKV4@*Ap8dFzO^ z#ICrX5BN|e?QPpUPf9kk-#ElB(xcg4Zg7b#rE)RFkr*L0Y!4&SLv&Ve2oK?wM3qoai{(`P(MofNv{RHoY1kQ zmr`K0V61Hd$_D%)LUSdUBR*{uJ<)X9`9PxiThY;=xOBlh#U^r3-^d$tIiCosrDi66 zaX^X4;@8esWrJSwN$;kJ!^Y-^6$+nObZF8H&*M@C7Rb9)BD*~pF$FJ6%m^QMr%jO3 zAf4Z`d9wo3Xtx^H`uH=Rw9X7)YgT@Px^-G?@LJf93Ew_8Ktg&=?a^U(iGMrO3pmuP zkC8-SY}`B*@%BC?Lsl5MtEHSF@F&F~N5I{`L5F=yEqi1ax}Nfs3Z86y_ZP%h?XYfj zE+S;B=?U@6jkzF~;H&u?kLKqG#RRmLp%wX9Wmi6Z?ofr0msiLe2&k0OI!A*Rw!w{k z^KqygitCU)#E6|0$nffo3uc`Vr++~*+6ZKPuyfytzqlfvHsh$q9g^M5uMq$HnDoTh|BxXeC}1tAb`rnR%aS#_8hx1WdJLbuC}S zZ-PZI0?0|ICJ@{wPc-)VPYUr|Hi+am*)sW})7ZFRzEQR7AR@A9bQVl7rnmsO<73iU zDrWTQK*yqZ4M#uE!J*R=S0NYivB4zC7IxsyiKjoywMU;PJ95tRJQHC%0xOCetSe2D ziCbQ+D^gk55g5fC{02W zrmn1xiIG`4N(uvqg`H*;4wyXZv;BAPL6fsGBN53sMIH(&r+00A(~4C}%(zY7P&2u3 zkl}2W_H8bww2hG$HMEUklc^iqu!5H=>vnO66;m4ueA7n&R2c?mHxM9AkI`P&k5LXf zC1g#C(sLrk`xY3MAcEbfdBYSSlWXRH{f>$wXul13zJ=0rWoDNw+uL+S&qt@J>CS?3 zY1KHHgu!`hIrMQuth(_w4;A(?P7{;dBed#XC(<%O1H~M)MISV#jh?=#OT|hB?3|*M zy!~KxsO(wKWUQJMhF1Kg0`oML_E-cvCq{fDpeVT4$8mVu$Mk|tB{>C>e%ZRh+biw> z46Q&@ozSO1W@l-~JOD?&VA z?!qX{MQknViITTM3d)JAIYf2fWa$TD|87#$lW&;)oQDB%cJo zDnpv%;>WC4fyC>NgkxQco{ot|QjhK(Ms-jvodU}%S-d(Gz6sj}Ar|^h{z=8z@$Qcz zD}`TF&tqO0>k%BDW3RcTH=IK>q==%Td0Nts>mrOii8h;}dm7>B5amzl@zqAOkQ5MD zkDe=al}h5?H?86O3iPXTt}RahcZvh7*6BsQ5MhUT%p++B5o&7g;FlQm=!Me(74+li zDwAZ{PaUz`wt${6tnekQLV7cNK$;m&!1fnl@V~oKYz$=5Wh?#n7wH`zAoRvKDZ5dr zE2lB;pFh!FWN@<&Vg1mOtN1lEotcCu;we8$8oe>qIrdp(zlNRJnaSBLCa0SiV)0-A^(p&B+}^vlrB3 z{JRavmc}#C;j!>cUsw!@vb!tQYhk4`ywU%X3k%av!SQE=3(Np*^21m;Eswya;x?z{ z?8a2qB@hB#Gp8y@g``^8_uWCYeH)v^!(elwna(fen`9l-g9Cer-;vm|&H*>61Vt93 z#*3jy>phgVHkc(|m!OA*EWWI_7!{eb<$W4zoJ!f#&p#Gc?%h7%&<+JTu&QX0lY{f) zPfGeD=tg^p-yS{?zBvQIJ3+V2G=>Y>vR5v=%U$Lx$j(>Fv8TI59>qD0Qja>YEh(se zU-_wFzAf#}nIr#rpHdfQb9q@?IIeGEaQ3yK<%62jCF^q35yfGK=JU!qJlr6e?(c|8 z=n#yoa1?XPTEIq{49hT9O^!D<6OP&QoII7&JbG_@PPV|uvVKs7vSRuMEM90UT|g57 zlIufWmlte`Cw$8_xVDStz040=$F=SK{xVANAO*BpR6~H-9Fi_^MlS)usbC||cw&X*EB0nTGjOzFor!rohOxs`K8q79y^4JT*)G((wFZzCG&E1H4-I`rLN{@ z11O=Vm;@^fW!-pvjv_EB$AXc#<<|+q;(0<#cJb?7SsHAZERq~4h?-A`KKaif6%CWE z7R846unCHv$qwn*N7Re{DP=g8p^>{n@~gb$A@f4{%_mzhyrNGjFn$WzGL9GjBq_B5 z{K*Z*WFuI$W5{pboWwd#Hk;KCB(`UU9Ngu)5DXZKleLC?kJHaM?zkkrBqBPuS9ksA zOoZ^>6Wjijh<)PO2b699Sd(;7b2gc^W5}x zTs}-|yv$Kji^h^k?5xHCn^G+1`Ia!>ZhfUU2PqPB6G-WzT{@WVg!T>~Q;#!yG0?}J ztRJfwpFKOn=MLr6{0p-i!5dFjL&J=?h!X;3%Y6d~lSjcmEpm$`R>d88(g&f-iet+T*N}?WLh$WUa`6C4Ya3TDb7gl}j}92z!y=4t z#9$^Mj;otAU&`HcHEPi%wDAkpGI$)a+Ns-X?_+-y6kMJlLO%2zaY0kzeQrs8*l+A#$0cwvDQU=^AIePoUg5%y#7`~MP z_u%o4XF6#wehRSu_uOUs#QL~J6XF?1TVyq10rHG z5^;r(pFOnW-1(j7eAT26mS+o+PS|Wm0MqvfQe4*qwU-sLm4W8c!|mm)nAmL<);dr8OboZiWKYncZ=h(z1^R~CK?%}CVT;}H@swP*{69VQWlzBOy5AAG(+6& z?T2DZe`X$jPapqE9XiND*WBGpeUV(Wy{KKRmU**uTe_d+{N-q2xwpx9IPMXP!uNu6sI?@*St;`r86G1IS*UEIjG!$ z(W(#n+plj>KU^U`;cGj%+?g#zM(F%=QpIu;^KjcnvoD~-Pbk4eHjoyz~0QDSUzi(4n}R!>_HVWhx(@u_vj=7>b53C>TFfnx|4k@1;KRnM-cf>0ouNh4ozzU(o0k6`puu*MsZ z24-D4pG{#077DVzBn>X;?>~xOAhSz&Zn3@fxR_9=WglQ}Pz_A|b0*JpAdqq)J%y2_ z!S{5#BcT`-`^sTd?6;8=*imX!lKgE}HoHFz>GqY8`ULO;c(jni|Mu zXmKR)?m7qc+_+YE`Z>CGe=2hthJjXs{puGMii|qPe*l0d0=K3t7(JJ6&|+~y5y?3( zF!Jh)Jnj>S4m}Gt=YAeYQ+UxOikMHqvc388$NM&bu%tw?g=k%6^COJ37-&%_gH+{E z(_V#MO_J^<`TKE-eV7Pl>bZ^JC5o|IN>CG4;*f*s5uM20ik$s~qLLr0AQ6VxV&>f+W07kKpCzd&{n2~`v-!Fl0n!3iVMX98;T^1ZRlw5)A~CnNk0iR8O&Ux zB0B{NM-;H$www8NFwLS0w@*(5p#v_H{LrY$;m&Y9O}E!~9lhM7z?#j@%{S6?t=ab% z#U9$>I-RwaBhpc?BiCOAIWf3WrvBpLwQLUz7h6nKpz>hoqP3$G7wi-z^e|>HPscpe z#A96_l_HV8jQiCl{JcM8&DOp5Pw#PO{_JN9Qlt%VxnypsS*H{S!J3I#Typa!<8bT? z<6otEVn7Z@dYv%_4}7O5cnrGDdt(94IAi3ldHPN&=5!E9j;J5)Gog=K zz(*{1U#=Rh#ah^mwV$Hwi>_7aEgD#52Ja|jHcJLNQYdF ztd7gsQ>gLfWW36RYu#x!pEEqx|Ki+<&b(>>ap{tFdFPNDhVm-N@G6Ex)$^kkg;Dnc zi34dQf@o&#!Faa+fVY}MLlJ`GW@@e=DYUu1Xm*&BDl|@#0$OD>z`qBWJ;nE2%VkfY`b%MMcb7bNTs_6daA}U`bUOEsNiAYI?JV${k?Ow|J&!n6Hx-X z$G9C^6_$g8B`LThGDK}cKGWQLq((S(J}y^5>Jooku7nx4dRoVTD7=1z)s;ok5HwZ~ z$3ssBd3STCgiwK`T}p0HT8RQUOjRYgDQ;`VemT^+U+8kS0>xB?@pO%AoRV|6mO^jv z7@lMpb2zqs(g@J75QkYhVdtd^#* z>@N(%tY-Q_3OUIE6pKEp9xQp*ComiM7aD0EN%zM=koUls8pr)~27O0-XYQgZbTtUqOP z)i{una9&g*WDef5_{CwbUJ5Ek@%*#kpCl14KPmmC}_bOPGb-vU6K+*u#LvWR3x( zf6TjteS>#H)`afrvClPaA5mGM@P-*~-Z8;~Jp6dKcy2g##AkjrIUJMz)aAX|94{Uc zC!OjWBThE7Vk3td3bsA}nWeBAg*8x8O0T+^#XG{powrqWIK)dg zkJcDnzDwircYK*?)_@lfNQKa^pg5kf9Ju3263Ts+-mV@@8fPDhUqW@tEjc)>9b=Mx z`S~?rNwnuNQko15cN%#FDf%7p zQo5Nu#b0=kd@o-<&vg(nCx=fOk- zLjAR@Mlfn)v(9t>0b~nt)IKc}HPVSV1^qM|A!^WSZR=*Iqlh<(i>a#gqb-V85d~>Q zpUQ;3Vxio*X*u&rO+Kw16TihhG-v#BgmHv&H3-q|QV*{ba~iCabjPe?aKuLBVc0%O z4F!>+J4D)@yDZm(0uPKNXl7Kqeg)gR3VFq%P1Z63`5+6oh4nsM^1;>k}vdk0(Jj$3a-SVw*< zc#vJ+m00gwu|W1W&J0EuH3~bjkx}mZRJh3RIvrVsB|hB*U&b>@6livb3eTcC&c1O! zb;Mp)?S2vP){i4Td^#_JoiLfZGP<@bV?_94YGe-2kh%N9hlW%d+egsm=w%NnSEY#`*_UA4QSICP{JDhRtSI#!qaWIXaWG}UB#E8rlr}6X2_Ib!B zkihy)Ui>f0-ZCtXZ_6GI-asI@6C8rOySp{+?(RW@ySux)6D+v9ySoKuTtLXxvy~xSeOe@uec}Rz{5LlU!?)Xrpn2YC zj%@bSt7dsg@-#6q;MXFzmt5X8;Y`!)!S0P=0<@PoM;X@Gjf4lX8IVnn#EZi=`1oKf zWpA}tR{Us+eFJ+BX}IE4DIds^L-YX?kJ@YbSm+FU{+T)np;}xt zTga7WA-`QSb=-S4naw8N0@jIXVE6AQl3f($ksT6`ae2a2Z->79rNcwqvlb+opX?j? z<0#KJP*Co|lXj#OH==K@{;3>WP9tv@StT1Nl5HJ;nwfzY0DTZCzpEUlS2J`Jj~gHB zO@PU0NJ!ejZq_a(KN8HScf7xk^K@yHu-W#(!dgT8*gyq zlWGzdn?&@k%iwW%Lgtg5m(KHkdwQnCr0!tnmyKxS>_ZkkZ8n+x(h={`I*}T7j|-}5 zmYDcCZ=9F%3>Nkp=Hq{}qwO#;eUd41-160uc9bg%S})U+^wNZ+qxL-+Q5Mk@U-J{R zp?fDP_Y>zJL%DP*@|vL&QUCFEzNK+4kK2=b-0YK9u-Ew0hF zW}a_A1cvu3;3RS#kx!2LpNRlDY|q5Cc?M(L0?37psq~|9e{liC*{afR3{z_0oz^X& zJDr-`LM)0*$10;cViRp=x$7>c(aBe{9KqIXOiY5@PX-er$dJI1pjaP=8y88Y8nEab zVoC!zE+sgf)~h&mU9D^qpTao;KZN(t8#N3~%#_3c-0n+tIhO`^>G#R&*N{q{iL&qr zN(8#@GUXE=rD|$SB5c^u1hpb<2jdKBIhxd^%o=Jt`5crMF!x-0NdPJ54OajnXj2IT zSa+<8*(jwG-}dO^@rXEG8L~6dxPt~#!<+YpAV$snt3T3q{i)+o zO_P&IsU>Xa;)0q62U4N7Gq9i3qGkB$YA+DdIUFmn#P~#cRXYq)IhMwlgH)zmSO(pJ znKbx=8)XQo=)Os=PJPMGncv^R5|aw8oCK>L(FsU&1Vmr%oJQkP6cY@}VSyFu^fFoT zMcOq`KRBXO0|!VkCCyWJsgs%|{v+gy@g;9e|7lo7-X=BTKZUkr)|LL#5iUpDL^7cG zY*(P7(paIw--FI~v8*CU0EMy4ak^gw3k*F^cgA>xZ6x$^if~vzAk{K4JR;f0n4|wu z8-S>WNNa%nWj|3Hu8qoEh16!BmhH=y&EuPQ8ompmGezl0-gRIQv4JsjpvJ`0+>6i35(Ie+x*+Uw^gRXfaz&W&lynZ&t$=;UShG0^*?D1ar{EPA>u+axq8 zP7~253qG~R@=0npQ0{eh4~nd8?(gQf?17ZaTnM2|@TIq1{KfcsC-lT{o5Wxcig9P( zgrkTZBV;}(D;c_6p(N^q1|MqV9%LxQH$RrU!B&rJKl4&0j!3quyA2oA@yLw%EaCct77kC>y zV~5g2t;r8GHrWv$Le9|g(6H}f*C;ZjD&@7+7aYog<%x&gc|eN*O7}9C_iDNCWnu)` zMSh78lE8(cUBfnLi^VchNzU!uSYmGzP8UC(I5jNifAt**88(on8RG$->~86ic7XYaIRwFO`Gq#L)QCC!(Yv(E6EUmgYxJo58-6 zZ{a@Z-*}ne-cYjthsWURA3y1;x!r=Q+igE}NjghU+XB#Myve|xknm=6(TdMf1axKKB;)dO z>oZ-SJE+1O2&5uCfp{yz$WMj1T*?JL8fb!8(Q+yx14&X?d@7*J8phhX5wXZqvTYV7C~|KOU8394a>TYuujHeb}|^)Tlg ztV07V!}3IJfE}9G$k^9lvvd+{&at;rMB`t5?&oKng-Uh9jgZtJL&itdgGU(q!E4=aTzhJou-QW|a@XhsWr zM%eFcPx(os)=~q1_xDsIeA{Hxb1oSSX64H(oWTSOm5RbOgq$Rn4ff~LMuqtC?x#W} z32RUhVetATPVza8llFOCumOno7rIh1$|D~VyCvI}YPM4E=8T3bLV+hl3zRs8QeFfK zwMaiO<07TYMW5?O9q~Gn37bI$d=4&dGk8DY5Rum*xE4UC>m^8X?S(!}yTWI5KnFEy zaBMZ|Rj5LdlG#kmX>W;Yvl&Qp#C|sCuXLk2gK4+b&vSEg*Bqp2EyRpHx69*NXt(HH`)~IrbnU2-1iwh2SnP%F=!tfPB$#AK3s8x>~;b0 zcZq}i|Gr|4;)MKRSrQQi^Aro-Ha9>y!tGvn(;vCFS%ga#6w3YbIySlsBC7|t0|pP* zo!h}r&%3A~6-{7LixcnwW!Nli(rLYe@b4jHI`Z)u1A#PuT8zfS$Pu@e#pL7ic4o>H z{G$#yeG_sTpUvzv#pQx5=}Y({`nVP9MH%TU&E2|}U`-=EyIhMh$okfOeK8vj3A{O=+B%Wt*ML`XZE6PM{h16mB=BJQ_- zo38DhQ0#2!82|FWe}0Gm{ZUuq#Gq&JqjZ1y4B*KI)W)U+Og+Y4C1$+GPprQ!|6_e?U5KY`TgDEUCRqk)5QBV(ZUR;p!(-!<5FT{ur5+(Vrry2g1H!3u`9jS9=7P>z+b4xyh>6mO7O^?gg3EHs;>^qfg zHkUH~4|DzB#+M<34_IG$Vnr9iT2{i`nKb3jC^tT(xd&N{ zpBwH+StZ3mieu!0zPRZs9X|3-T74Sz#;}9u-}j=qf*^VVlT^BtyteF6Wb$YKdvO0) z+vP?OnL8xz&-P0`Aa634)7&aX6U<-l0fAA#DsHWk z^i<;Ft?oTr;V=JwX8fnY7~i)&o}^IN9;L8s?HfbE|LVmY9GLfawJq^mMVi4H3zw+g z)RO>#r=pUnCV(`!UljtFXuAZf`W(KXjK&*vZ>QGUJDbbJf_LTR{jTu!$d^Fy&d(6p zTp9Vr|2g<@c1;qmYIMMHc{u_liDZnMrbC1p|A8DI!(KylB5I>tWspVCK*&p?yd-el`yfcbdu*HElhwz&u_Q+YD*H<_opeN4gwtH;VG)&m3B zKj+b(=^N4c)3@qgX`{@y|iCOGm> z?~d4yy#MmNzYOGmzbHcqx-kp*y`cX8XIDWZ1??GMiLgEXzZ?0lx&70RR3PX^?jn0} zlmER}|8m1$Q^uqF=Tm$w#B}#R{`|jRWGDj2wAiw7mE~YW4DjfID}}2n zKZuhkTsV(rRc4^C1u=cwsMJ0-EwU9G)dRr*X7z~9x@la(gDPQ5bj};$=XGeDTa6;n zD2ABYCo0zt2aoL=62rfUjR!`cS-3t^H_Diqj>>8+;ly$jcdfFiJcN+TjFxXyf@hiF zf0TG|M%kco(NfkqPL9gyX%sfm8j!0l#q*yyj0!eOU77XHiEs*yJNbGp-}MClm!V7Arp30I5pbNMBx zjJc>Dgz2*8i{>MV#~9YUCAz+?`oMVfbLTw9urWWXic#NstN97BD2tK<#}?o=T_a!o zWP{E)oXu;Ha=Aw9^61BpEGebMppxG6k{KI{wN@R$nprC4^yLLw^a$kzHov=DL3gIX zYUK)J{-VDJXMcS?%aLkVR6IC}7CgWgZp#9;Tj^q_{KW+DK5G#wuM;@RzCMxE_Lqvr z>%t+zl%NDZ5F_nIP%?73Bh1dF9I6~NY`3gk0LvhixOkV6NFXNiR&{)pR!!v1PGVDy zZ+bz_y`UdT3MNr>qP+5M(gX5r7!M2rGFXHBxV(-o-aZlOrHvnF88o9xKf0V0AuLTH zUOa_L-vH6Q5~)f7kRH6pMX^9z>jxL)O-aIRxwm2RQ@M0bZDvENN^s@cFef6-TamE6F zv7Z*GHDceMb~~r-#Nm@+!ZTpwW5tmt{e6zhh+{XN9&L(XQ+VC&0;A}f*-h%}MR-uN z`naC6jJ(5ueDqvM0HmUZAnp%}nvGGq9aGGpGk_dffp%v4Ilp0XX)0T8f zZ|SPoNfh@dLh0=2DSSQyEIZZ|MXLQB*~N>6aEFLRP_&r`=2CLbvJACIYtuPi8Eouw zwZd28YNnQDnf`}X7u%(@9}xLQ*aQ@%g~<2d2u)y4TAW$@R}09bz6LdnBbjPt1oRQiZ=CQ4A@ido3+Sf-J|v zPvAvS7GByzOkF(fbAaRt)V3-WIzUdVD)~P6M0~s~^u*Y6`G%iFV8{-+25pcUCA*QY z&aE_|=|_^8giG?`1(WLx{h1i6!CBpNh-bkZ724Q3>^d4`8JF{GWWYM#0|hOcU(s*T zb*VeUqA!eyzZH0s3z27?fe2nvP+z*Uuj?u2oKr<`nj(X(%3j1mmi6kOg<<~jkH)OXCy6%NMyuvlU&ui?r(SX$@ro}aslEiWnKUQY z&PCRv4Aq%9@OwSm*$SZOcvwQX%|@o7no zH!+nUiCR>fO5MzmH!x0-#3QwE@li8f;cO?f8g7nE+_NKIS(hR>L2($%6dEV-y$F64 zervTdmM5|oD?tvQ5X14zNsl2xI9Gv=bwYK{w<)Q9Va=Ld3O#Hsrnnm(qcL(tW|s+j zw`R@4C3PrKKmB*;x7kc&r}zU`tyR0x9Z{QqNfcO<0T{Si?(W3zwN>MOwWC9v@u|Pa*Nyl)~vSKSL_zmkD$86r`cM6u$=^_7pN@a zqfoOu?XmI&?xQDR3|W*I1fw}RGI=>v@u{OBuwr<@uXcg|n1p%w@1Fd4CIB?gmFf^YrKbMOwF)Z{DO$$o)*(8-(IrJQM>@1)dtp{N^9N@z0G320Fa}kIjTB4K&PEh4PkfZLK{P_u zGL4rEBU42rmkg?HeI~G}HW3|fT8V&3tH^=kxL`v_kZIK%O0rPjsgeQU(r`>39LhFe zCgyHff9|Sp>}4=~|CsH7lR_uoz7a^y5mU5Z%!yoQdb?BcJuTIgTN}>^lmn8ngR~ai zXykfAxS8Aa-5v_Iaprm$llg>|4V9yp4q2S^ZZJNGtp9E>M4T8&Wua`;UklE=1REGd z#4wu={6*_lPHv5NR?vDbHrGDuw`Y}0Pd>Lui=x3Cbl^Hcdh#Z#>tINU$$Y}#UHPdP zwPkqst4xYS>iw!ke_bkzknt4mbp17z|WE|Ai=p_OEg0&3>tyqB*S z>I7360e=^oLD>gVNh+wdK;t$`V-ns?FGuW?>q?jOETLBII?Nro4&)^om9Lc13G+b9 zZeO(YA})v=cC2$&_6=+Mcv$md7oCkyuoMvZ2DjHMukO4`2=zaNqok?~$`i>`*eZr_8%^67Atd#1&EAq7Zal%1ux-R_=#G#8*Qcz5q%In6I+J)0 zPgER`saeY>6~%LCz}epq@s38JQV}O2CMNZ%G1azKlfpgFB(Hq>qR=0ufTJXu^JeoQ zZ1C#>QewQXWe5hC9$TN{LVRFy#}mT>oLt_!eK9=DpG2rU3;5g5T>#4F+smP^(E@m=@%3lfgS{=!D1APE(~& zU8okQQ=ul(U{uh;L#tU=hiHxlwuRqlqYLj%4i${5tolUO885@>yo25-#~` zEjT*8aak{lLFU&e;`U#4SeO*#Bwr&^TBz_b`ZTrYTyLxz)B!E}$RQ1!;ycJCwoU0x znzUBz)|3+PHN>nYG~^a7V%EDowgHE2P1uC8(8N5Wcjy>GgE|^?Y^hdBmixq-#XB_6 zrYcT;=FEbI6YJ&)t)vcinz`&yRdppZ7SkNNd$~GvOz&37@JVtFqHjq*!a13&d#8g% zf-ma_M=^@I*%&PtAFKCTY}Cl^1I#;YOq0gvPCX^vMq%DOZmLlLgSlGw4z(PW7zws?KX5*`bz9pVtH;Z8RXSU*n zJa_0WGDI%=oQUoa5+$W#UcKi^6HQj4$Q1y1qyQVoYLYU;tQhQBFfFu&8_92yZJgl? zd`dK~&uP+vi?(?hl4i!M;yx&*kH@+gis8WSQRrIBJBcvMrCPFS*NK*Bqymd1}oWV z%49JcD6;F(UR>A{`8c$W_RtJxdX*?eIo znug4~%=S#khR_~3I2?a!n_z|!!@~ZeY)Bg)x%`1+Qg*EgGn>;8Ls;F2ye5YsVlGRZ zrQ5P)FlEg1>#B@E+6Tvkr1Jq`5LZ3%J)qgjTH5F_o;FXymtxN)1*f<3kbiZb1OA_) zR(*JJLfI$}2l|PgeF6{}bYv)@#f_u}&!=Ab9;h^yJ*K1fY$t>KK%hMmhAY}mtYiK9 zg&cB!tY8=w`h}_aVo0P^o(PrEhVbJ>6ZG9Jv;@e=?VN3+@lkR~a+WGKarl)Z;FpR{Eib1w5|Mkd zkWNOZLT#>?=*nkH!lnd7lw2{c>n$$5>THJAk8$`EDrfyof$cyh?WpTv79ojSK5 zz(_fflrHKK80BUJ;9wA__$n2Wh*nHH1FADZqSK(1}lzRgtqoENB0Rp(3;6uT-q&@*(;w| zKNlY@kJwiy4)_c{Y(>! z^Gsa9{2&9rN6g-A?03p|IIT}pd(Ut+>2iA|_V|KK)Up2Q=FKg2@&3ATy33Vg6kxT{ zejnkLF#0|kuTHJ-)s24qkT9-1Hqaf2;-wX6iB}BwHKH+jUim~}P69QKbZ$*rLQ|$q z3s0EH$|2|zUW9W=?}(*VQ97v-8ipdDN-Saq@^a1){hZZh8KKm9`e^N4mZnvLNfC$b z(&AF_;j)RWtZdi_L#u!)<}m2ngJIZWUYZ3ufCm8e>%41eHK)eR$*qYdHy1=G+^^1A zIiOAw;9igGSy#|-Xh!5<)4|C5@Ruq8(qAd6nP%x&s8^&8iG!?lTYylJ)e}BXdlHYp z`ZZI`j)wtw!)DfSlhvvW&6J44;^I=m7F(cS{HBh97eCF7O-p{ghN@LXWL?};(DESd zLzR_nJRD{ZUvG~~5g4t#nPTgOY46X8af8}0nqp>z!qo2TUY)w$^%(s7S)MTN$SsK! zY;=X|}O`K;MK-z%eWB)IVS||8Qt%@xcy756PvQTV zbQunH@D4?BVw^{b1|TXw8%Yibevq6g_5UPZava!akp11cJH;n;`sxcc3<2Z~l|=ez z+DBbx*lxdVWs|t0GI!w=i9@0?aXWyd<0w&It&;rlzF}CsaW)Q=xGYH$)&2s;$AA-? z6lgih=r}ZG3@?BR!o@1m6^_YAJjmQQ%sk+;ve$|)6?&lL3g|rGSyLO@J&n|6S{=N~ z)!{TA$uj!Xu}SWW+CUM8B;cq(u;fd~G*|+0TmSSE?-!^L!(SWv}w374hA)vNtBnjhbL!Ya}@X!u{fkhg~_2xvON_=OSgNne_J=WEch z0lyV1rid{8iJd8r-MtXtZTz}T13%=cX7aXv^(Oj0zc~D;2OqM({G^g1dr<><@gd?6!TaQJHc1lBR!sXA6p7U7- zjZLedc@=})Bta`3AxcAkRA|<9&QxSvFZGGxo@mxWW!ZE%9IPQ5gL@n*N|V}hE>LRl zA+A!oS&x}cfLu)^H%xPuFd{`&9VW8qfLITIFf8qtbk_^rnZ{T_l;YwH6PRHZ9kOj3 zvT5m)o{c->Y1$wYzSLs9?f0Yy(Zrf0)}c<@q3zN1J7zNH$QA3bXPgMPD7#~nw?XB; zx)GSDhz)Y9L>nK`W>)OqwY26Z%Cq%4I06ajRgFio6*^T{1H7AI#y5R|t?_j~mXPv0 z1?H=-n#A=@q-4}gH?1-_QQHQDPA!}i)!}4|yq=;(RV&X;W(TY@lg3rR-|o_|vNerx z9oKlOx|0qxSBc3WGQ-DFBTV%@8ylQ^PfVScLdvwr!RP2y%ST}JZ&B~fU|-W@zg7qI zDdF8{9VV>^X^DzkU9$zVdCIkbIr9885u`g~R*Iccg3s3wnInA{81;f^T1{%9e;se3 zGgl3MSazKh*L5=95b~t6o>7Xvj{TeybhI6o|@qe395_$i-7YsJ~# za$u+m-z)>HboY2b6%}D9)l?-tx)7fRGLq4RmZZwRxC~ff`*y5RRaCmKnVn?H(PSrE zx*)VD%jyc3OYYxKT=`}M4h{EV`Q*ukky4mAPcXi`v6(a|xq`fuE~8nYqhxxVQ3uaH z^!7oMyg_!HV=%GV*r)tU$-1MgAly4sAaje>+QjCs9w#+~aeSJ_A72N@hGpHLejk2P zl0#eXosQv}HbpP$gPqKMX0}2Bm!wg##M8*OmWDzWS>CiY>cU*@JJ{P^LoV?8(UKX3 zKGp%_Cz4K{K8quzxuX5RU_fq6pPiIeQ(FLE^SojW59QhjUvz4(YF?Z|8v6FBxd9G$ zuCc8NW$S~{7DjOd#%&W)2mS+hkOzx4FT`HXzeG^bkau2GYTvVYufv3|SVqrB*JUWR z&88|&kjUdxd|6`4XaHU^Jf|LXUNcC#S^Ma-sHoWZR24@i;r3KPp$@}5fR$n?3fU&> zl<=!i$&!AN0zaAz|L72+foqRj109uc_jtuO-+J@d2}{IAEhupr_9X3(NCf1jU!1FO zcI*}Ku+^9>+DVI5OT{?Kt@3%iK$wPw!c52S*z-lF+ofs4$88eDpE`d~eZV%LB!yRD z5)?%1 z%2){3SYbQSKY$Wklu|8=L9aNYk;<%d(~m;_k%WNO0GB5Q3hl-;>Da>$>V)E;dcXe~ zHk;dV9Dy{u+37NL{|*X!87$+bKC%;r&;rD0#!ly#330}1tw=80Znm)&YNZ(&^6NHZv5gJP z>>C!PDEGnOY6P{M#~vd1BBsuy^L?w4P2S&?OBF={$Y4Si>wI708n|N^gbbntI3I)4kLDC8DCIZ^3ZCe)6=x}~pY^}ewCc+)5&-iSyzLMEquc-H`kkVu4>i1aYp*JN8 ztYByg#8mM(p+sPN&_&V94KX7i!{V=e5gG!o#xBcZTwK+->`oEB%=U1KXkIl0($pc< zL}8Mb#^wdtP+Ij>IF!aQ57Mnc`Quoxn-^W_(py{9laKIcS-K_GML*P6qvFyq@$xxR z4~9*6{qiM-n`Rp9iHlQBQGBRYELRrm*^$7&?)xDhD~Fnafu(@44uRH=apI9n$_v8) z)S@Qmj(09kJ!Kr^8||Jh9ZTu0yYg%*zFB;dTOpLamOe2?YLTGwF9Sq1xu(fl9Lv`( z!&~!u$#mf+EcT9O>HAKb3NLIkWkhsDLno=al)#6maVR|#jb4-2LAgg~n||%D{U9g3 z>-Z9W&q{^8q;9b>)fZr*yu89XxxV&i+wPyBnX^y!J_up|?K%KpE-^#@>~%2~ovBw@ zPZ>qKtWQ2~c`K#OB+!zOHgv*a#`~+D`(F<4o2?<&A$)Ep*!6s#4|tL{U6yn>hCm2! zv(M5XO3>e4L-=m{%6@c3XgiV^>@N_Xc&`3%%nst(b^LlcYgsYPcz@jb4N@B;r!+)~ zC6#aDdFWNvp;oJY8=&`b08#1v`t{iL8w4X3t$soSvjk?R5GkL6V0Pc{7q!0KW^7gJ zcE0>UK1~gSkhuMC&$lLF`xKo5$0AcD{O=Z_LGYN_j-!-clo|9v$lN7e&kNQQfhmxN z-1+;%+9eeI*Pia~59MFw#6R*?B{P~oJx_i6H3tIeZhBsNU-iI%P^oJ=ZX5T1D2FIZ z#XVUfVh@IukjN>^fBty=NdThq%{2TC!Z(M0WEqMlldf(%Ew8Bg@e+({+b!gn1EQ$$ z|I8+_R(L@SzXquH0YTBRvFqrk{=jm%->&*p?t{Y#$&HLtX*!OwJ*&U(hD+161|D|& ze!pB&5%bjs6D@19S^~xi(IilnxK;yh(C#246HC!!W-k3Ya3zuD>{<^n?Ee&ee1KDl zz@S|Pft_`K-K6|^ZD)DPc+nsUXavQoVwcNRD!!|=hPIvci^LZz(id^W9tC^t$BFkV zQ%lT<(WyFwbR4qQD6age_|+dplHgZ3BYx6JC~wGrc)je(A-D$tX|LP2-lHes>`p<2 z81CbYL+7A&Uvv^I@A3$xo@nLMCKMI_6smR;nNrgMV3|ryR&FY<^6Y*;&bnsd$6+H} zk4)IrybV($rfpTPm^r`~!8|G0nO2|R5)Aviovqom9}I3lx0!W5C)zyC+fY}iosqm% zk0w#udR}FyZPc}~7&Di+dYZO4b^l!JJa6S5x5mWjh*B)8qBJGozuXN8ZxE}8v+eHr zx!hsBfh+I1L$l`2X|+^72ESqw!Hucoa_b#a>HT$q;mr2v6T&&uw?XO{k4dNI3Fwt? zI59fy_wA<7(t2IaDtViwi1k!1VvDN{n?8GzQ&}^_wVboGs!Pe?N59S|aX+i`217Ig zFPeu;T^2j9has{%d~5B!ZHU^9SY8Lo+&1rCJyoMv#1^9v63^BqGM+Oj=(*l&gCG^$ zE|B`kl(oHUld{}5P5L0<)lbIfeO;tI$B%y;eJ`Qc*I`o1Ftn&t|Nbz@&F^vsAkuPD zY{3aQnC$VqbRP9to9MO4S)Uw%ipH()V56=sGuC_Usj1OmRk*BNtK)k~Rzl1a?Q8g% z?s%F41y%IM(sBvGe5cd2gkLeZ`Gvqk!T}y^Vc`j2P z-(G%-i`i4$s+b-IZv1S2%y~}ab=xs4-oR*GbiCO`Ym)qW6PNy|8DdkN`ZRY@jp*4d z_h+|BIf3e;Mm)shsrLcNng4od>8i4T(csbNxtkW|S-w&qgE;ylnC6kSq_MkfjZ|Bk z-CQASz;?dqJli%bE>E-ILRBtSj9za$@1f$%kXcJ&)6QlkYhhl<;hD-C9#&AdXPeyH z87_xBF1@2p?e7SmNk+z2Udj|f8BV-!VYsK`3I|zEFAuhKuP7F?Inx@0IkH06Kjn{}5pkPXnpN-ydb!Q9jxl<{)u`Yj)glKx zTy)Az7iDsq#9U?V?N{0}vIRKiwQZqKDH2U(Xe3{n*EBab(FLLQ+?cX(65P!j#IRNa zl@qXsAMuwha9%%sE$*|-3*WpSh(`O+{GmCr&XAWR&o`nXjS8I33dg{rJrTNNrMMDJ z#zh@6&R4hQZsx&W7%BWo4%7v|e z;q9E|V>(balz5NEG-51`g)<-xkkNrwb!oWN^S;FVb{7%C1#uma7!P`%{x*9#) z1qlu8AHGg{?R*X)xY~b)^jzso!@OtMl3@&;mLqhNP1gCkmu{!+G9(ksJ*NX#S$r0#kN&Y$fr z8JF1d-qQ6>E2y*^@%?aQ7de$R3osm$qpTSkO@l3vh{h}OiWFN8lH;TRaW#+pk)8DU zb(`~THX*x^YB(tMtHOSM1lEC_L2HLuz3MO_?$I88x%Ef_=siLuhKz{nB| zAfg;+ZE~GS4`)NDi6Yt%0 zSp2R;F!6klfpv!0hRYJDRsMF6APpq=!+0iVf>l2bM}Hs+r5BKRbRJ6fTC##;+ZO_n z`yAxeeiSt(D&d-b!+QTL6oAI8`+lkh?__?Ln~WQ&bq8t?5lrpp2#6m)jBM5yyFc)& zx1VS568z7OTe*W-8RWvY9HLSv_fU}ZDsD%AF7+^Fc`1IDMEs&<6W|eC8kA=TRrGBU zb}7A6ov0jZo*+4o%3~!Y(`u31oeIL>_fyk2PonwN5PUag3p-G>AY{NtB7_{aTy3Of z?utNsJ$>IF^!$vyoV~5TXjxn8wszR}RYGHyYxHzk?%}e_XM6O_w)Os}N#6nXmekcc6b)1qnx8IOxmK_7{gJdb{ehS$%Oo%U1& zGmsEwtDA*)d!3_qE-%l#HnW@cfL`{45Cis4)8F2g!z)`aGZz)yBnNrTRRq--(hKO+ zxR2EuE^}r-oR)RrH=S0A8n|g-e$r`W={&A+t0`Y6e2Pwjv8tMBU!~S{nRPk~+ST_H za58Jme7_1FXsI6bwnnY>FuORfV|C+oN!h?Sk6gQk>FRhthp*Aq$zeBT)X9tusAx-i z{fyW7-h)fu`gX&t`}+v~sII#yw8CMfekKEx-aVH^rm;z>bX%me>i!CQh<`&nEXXn7 z(?C)~h3(4)+2bT#vleq04g2Cp=a3LCE4}(H*E?nm0oe!>*s6L6BGvb!D61bY68<&%+CY1N6?|^nlWEQOhr4@41BaQ5<*gg6Ar~zqe7jYBQ-Jefh3AqVZQND_mA*cUqk zXUUz{i4Z&6fUezDTs!SWEZT*&13$&vFjcwC*I|%v&>P~#Evq`eY~?D0-fC>oCLWXS zbV2UsF^exr$UuuUypc86``J^!8|n2OGlJm#Slrz+QHfV*Je@T{t%S2kO(@c>0u+73 zoF&eGnoxUV3AG6Nu(f-!ys9ds=N#SHrQ10EBJLWWyJq5$7u30ou@xBh?Rr8Ldd#-3 z`Nqt-4mtP-2C~0@|GQ=+cyYuAJsbjGgXLpvXVd^@UOE8TRN6`m{kDE5q^Q%nZ|YPd zn0&3K??;e6y|9W5jvXWcw}iB{%{P5;GfL>8l#ivfP@7l{HV~EbkU!Z6Q=LaY7*KfO z`C%s3g8lG*?Rmo?`z7u+$JgCmFTIX;b6IFgi?GmQG>-S+(i5JKX?sRYoiOc4t9`{qKh>NE ztxA_fOJo9r01`i0-Y-UCf+JjR?n}-9r@~Amn@7^_X+%lBpVw)3y?Xv2Jn*!^mNVJJ z-UH0hPr)>j$=H|)mi1t&BdmF!dBO;r;IpQ+jf(I!fDYC1ng{yYLP@vAo!8?S^yqM9 zB#Sn1(?;MGY`_t+Rd03g_jU4OCI$6Vj-WRYSMt_4)GR+bGY0n(TKFB^_qytUB$V9` z;QS3j{9QZa*!3rz7qzsklWsDL9tXyyn$-)?4%a4(G8op>D(!TbydECE)sbeb9V?qQ z4jsnOtTnZJWE_tAJxCNV)YfwxRYp{I?#5Z0q}k|xv1|$QGhJ$4DZA>`bhjYkK6*`| zqnyO8p3u=MjY>Q_Gbg04n6YlhaOE7wdWBsPq_!}7VWx`ymyL-HGPSjH2fr*HG~51HO#>Zzpb!}$u8G~KU9klN@s0Wq9y zLMIT#)E}lKldLa_>~bR$do)dS@)+=3dJpv}aYB7L35e#TD@YMn%+ld1B(2_X0~Gfh z2Noq+2`4pn^{wCsF|^=4JW` z0R}!h@OrsGi!n9!I8~=uk!-flK2JN!@9&r22#}P^z#Gc}i1AeUY*IulA!cDnL$rYk z4o}mT(X)SeDZ@v*kCtP%0(P+^!y|j&&;v;*JSqkT_sE@e0l1Z3igLEI_LSdjJw zJFk6|U*vI!y2LDMR}RLitMH-2+Yy~HCMuDh@hQzxHrJjS9=)##zJ0xwPjOCBcK!OD zPG4eO%iK5R>uA4+v;GNTHluUWA6JuZrqxx zy4J*;GZnN>AW4VY?99A>sv+By>2aTEYHwD%)YRrJsEd>>R!cf_!#K*mPRik1K1a;h zYU6ZcSTo3Ny%n(9LHg;r7*i{M=<9-TjvF-ctBhmRcL+3at*bN7Z9ft)t8uq( zlqop2w=ZTx?&F`FoR!X4Df#j38X|xjF-4l>U{+nTKgcc#p9!XYCE;*#rD}1-q_%r< zEXWXjir!ZGm9*~J&mqYa!F8Z|aj308I3Ozv8n^vpb9UnfkFHi85m}B45ByPlN0sM^^Om?#14nqC%Pm0%4_u$k;SHrKG*+amZiwOz9HEI{v|wCj z{8^MrlE>+plzLU(2_wssXitFsP`EO?R|;abdpE&hn;twmIEl-+kZTfp%wbLM-#pKF zu;J5L`Z&C7i ztYz~$>_wlv((E=`nHL!s*g^Y!*rG(UoM(N4?|Fe>9YoD@WVW%D83%dZlO0N^s75@u z$buU%jNIE%XacZ8Bc3v)qg>N11=k(a`>`g~H;MssV5U;Nm>xy$34W{Sac<43v8H9y zBGCOqF^Bv&+YcawT}4tjq^icZQFvWftc>pB3+Z*{Ud?`13iF92B)~Tk%N-R+-w^%w zAm)S#jF}SsKDMp*1nCtQ)a=pmAh$Ud&dT%(~Jpk!E{XN zwGp*KGQj7*?B^Es7wkZgH)BJQE5}7Frs#5Lap;wl~t{|2LoKs^p#+Y|{oAox^+qPJxU4)3mnYL??4rdCz z%kcR64LVSxwJvZY(79D<+TNB2RF&2yK^EXQJId8rdkov2X?UK0?KOD4Xo>KS^T!^C zTB?433+lq7QzodiyA7-Pb?nO=Z2*I=Wfs-6Ub?Mq`0i< z7zDGLfg`zURRa}H3%&S++a5mOS#;fGLrA4SwsM3p}@h@Q?pM|JP?zjo=Djnz$K4f$d2 z^J5dcNh$HN&*H?Hq2J!Oe)GbCg>Pj1ez5zE5%M9zJSEhg63L*w&uM88D3ORtLcPVXt?t=Vv++1_%K(#!mVJ2Idmu9VLl{qfp5L z{y=8_lz^cuL@6M5iMwbZS)a$SCv7}os}A>ONQp-gpr`jpbcTBny)RSdZ}F|h`jtWa zU~!@Xk8VSh+XFy?NOrDBDN;6H7LlBcbF;PRvu~?O?7Jo>n0irYd~_IB@v7#r^s(=s zQgQe}B#z+~_3HTg1X2;=tI_|Dtat2^gpHat%jmLg+tp>O%eHOXwr$&Xb=m5&ZQGiB z-Z|&Y%$h%tD_4HVd+!|)S75-FxYJCA@tehc<#j>Ec1dNfY5cqnA`zFLeT05K+Ox5M z?uv)Nz`MO}Vu9uXX3z~|ESEbO4Xxm)-WGY2f(v-i5VgOr<)AP0aD{A>wOWZjr#rsi z_GOO}W~Uqcz1M8B8!5N9wLxoUJ3pSwzZfPPQGV{2KbPCJ6>JIeS`e!^p_&|9mTVh6 z50mV;JsD;it=W+@tIf@xT#W?T9d8XFdoFivt`_h#M$&e_-##}Oy53%jS_)Ef_SO7u z7Z&t9l^aylIl6pjcK%v=q%x`aOouUOtLYmgy(uJF&2^7(tZP+6Q}0(0c)Z<-XWu@8 z90Tjzw`Mv0)n=w}>QX(%=_BBJ5vOSk*i+2Wt==kD9#;J-l{;VV8b;_nQ09yI)KBd> zllr~mF$-FaP((bfWPhh*1j9f>{~~m0v1N)^Ae$Bk+k2Lbk)3K{)pS{v#(NLAHeRX<&JIkBGiC3d%8fNs zY5(Uk6VYD}@j44~FD{nIe8N9vh&kmwZ=V7pDqjaqUk$3gj2Nj*-99x<)!Y;%m z=L9(b^=COHsEIOGAR(d;baCgI;8AnLL3}(SDYRxVAks}bc}IpfHiElY@5;fh5|D{+ z3&WJW^jls;ZX+v&%(GuQJ%v)xeZDC}MMKmyGNEQNq!A)aCCd^?jdG!}B_TY(8P?d< zGC`uJ$NfEg=yT?ZsX=!r%b&-wCJETU=s3-et|7HQVqZ) zxKO$(p%%**i@K>z)q?A4qa(K*Bou~ZU(4J>UjRdYO4azX1fhb6usPey10shTNdqRo zwMGH4Mn+Z(1KQJ2C0bIosqYv~6ind1=x}J9Q&0jLz{BTKBpcTN3+y zJx|yy&WuRg44CN+Amh2HbdCkQOD56|2=rH4RS#oBUlH&0{_SppMJd%aQ)qOP<10`Y z=pugsYM%>NOjv*ZM}Tw0wKAqK&F*e zRr08&(cu#HC3T}|*i5669N%|+*-ZAd6zu9uT|suv^JRhTS>LSROW{!q9)YF{zk6XW zA$O8YZKmjM(?F-K>o;AI8iUS4%~ZM?0}l`Lz)ci`5ykB^7`Cl}O;vIp3?T@ZH+OWi zy2t-F5fB4ae?VrXF30<%oCR5yK}vZE&?;W1Bi50@jR(dD&j3@C@$1ytYhmSp;}F$a zePlkv6c&pqVvz}n%(K|>BmegSfB02K(Z5YQEES$3Dmfxk@7KMshei9(yHPtRb6EYQ zYWDk*?OC5|ClPO=e}9G&UTNh1of=!28~=;Ik6b;>>##yb39~6jL*BKjyUplArZe@R zVTGvoc={jC;46wOhXFue)TDd{G%dtEdEwPT)FU!)>BOSez9v5 z-wP3m|LbugDx(BQ|dhYNUba*F32EQ)<4PW9PMqEqZ=4E@c@!^G9l!G z0i^`STkJ8p#8O_Ak8p=i!hqsTSf4?xKk1M$Y=J4g&A4A}J*#~aJ3Ybd3YNQdt-4XP zwcqz?X=Lg&{Lq%u^W*E)yX(QN^SBe|y*Gbh-xt$z<>c-@zZ%)z-&|(gC5f(;y^l8? zimlPZxolbihp(vm*ow0xiD!q?gi>lzB2=}l-G+x2Q-WCU(MH44I53b9Cd=`f{~Nar zfk>VF&oL;iDT~__-}kfd&ga}iR*J_3*AmW+W|w|lfrpK=DT~8+YSa1ZsXDMNyz2i)hq~YC^*mOP*t}4hInwp_Y<%V>Rxp zwJVi}Mowp86|K%BU3B9*tBr3)t&Q=&J`cBS2)~==`vS3!N-hhmAIJyef1vLd*N?ML zs(R@7=r?uEYFA97R586J$VWZb1G7}7rZui+F0030f-PcoIJDN2Rc`y(#Z6m7_3C{} zZ`rp^v(8Y%-}aO13{tSxr;l4FmxS1h(x=U{_~SRwROPrS~lCEXD`Qdt@+n1so7>W(uH=kp#a z7uT)zg9B)4+*tP}r00PP-D{dv7z=2<{1v20Mg<>XW%k9{VFOi(`>i(4-sh z#!=&Nny?#{5|b7(ou#~H);XkPY0 zvNGYwO(cxg^7`qIDmI~oi(e(CWLg^&-v=8Qf1i^NDrG|TqIB+QnpE<;Ys9P@RSyp8 zs+nb)0*etz96%l5#BhRI<(w^=$J6wsM5xv*GF5MDLTW?&r-kUnOOz{J++`veJ%ffP z@_{mqcD1PL34X&9L&$wohO~I6fefqb9u2mf5zqZ=&To|@lTL&(xX*w*mc_Q?P67V4 zn5oeN0#h=DC^^mkcH~;KOqrnx<6j2avMaFIy&b3RE=0*@&%^P3{0hxR;~T8rT6*Xjsl;dS5e>^JKXUW z3du!%C6@GiJ>ATo?zX-eV@S#b3yH(PqT}fZ()$H%XfvJzI zkMr<_-g}|n=LFgy$51f-y%dt(0Co!h;NieorWCwh0dTh8#T#EPd@TKyu-?!6N>dcGXDYqUgqz1(Y#pkl z0&^m*A$iK&su;b42%jBn%~R>3$t=}4vb{e!S?+|m2a$U~)+{?wbjVP_9O$&B<*?^m zbs*T5$3vDj`YW=2=vhjoj`vO)UZsLWHQ)l~a|X$G3%nG?OPTE9Q$d|V4e|_D-ybXh z1wK9x9=My_&S#9igKslt0j{T6oCl#G6!G34W5uBu={d)L?e|aC{ zs1im>1DW(vT^oVaJ`jBM!fok$7n8op2Q0s>K_H=N4 z>)-Ua%5cvqDNEqt$Z)fX|47nEJXQJaYoY>@wlabKxQ3OQ2R+1p+j-a2-RoH_Z5M^S zet1v5d*i-ZIPd3o7)Dk1BA3m;pL|8S7Mf*{=xEpUllW8P>emY=5~E$4jA*o`%qE3! zhtzGeAaroTHDm*=9MBiYc*3$-wY(8tCkfv8mAP>Et*msQ%V=o)*1acc|9bImSSh|GrDfPC#MmJT1NORvi5580G=ouqoAdAd89JBZ`a3t z?dBjqsBq1VE3g~0E`UznmozJMhQ0&V{0GK~D^=z0>i}|ntz+k7P2I!P#rjAu@3HX3 z_lr7Js6nX|7{tAX+^TH?OUk&jibZQ*+9@1ibpVqXvUMyl)yiUVSSM8VrMBGWmx9WQXZA}u4APRGY6Pg_lGDy3#h(BXXd^}=88 zdDZDy)^icaIlFH25tQ5fwcu6>r!mDEQyn~l3DJqx%CE(po#S>;iDUcMWjr!P_-W#L zV~X?Z)$E3wC2RVE2TWVTX3bjJa;X6VkO4tJYpU{Xdlqk_?{Yhv4_mH2%UkFOzJgXjB1GM=vGV6lONBuB{%vS7qM8+>ji{$6aCm8CgY;`o~4xi_23t4 zuN_AdGJ?+8#s@!#pQqlBFCUsU!VU#GjJ7P60}r{ir;^zXTj0Ootv7g-y1m9%YQAq9 zVGyqEW^_e}pmJ}cE8CstGXwhUBUUDM9B=<@q)M<_ZP?_tTfLZD8FQZpTw6s}g?eGD zVC`H-nse)}m}i<=+q6sRfWG}0IxiAwe3@2tz2>PjVk*R!^&99O0;%)N^u?5_>L^Xr z=%;->7kJxtUUuhAXfT7_wtV~fx~^V7dE30c2Q1yJtNSI?Y8Z(y>5ch2ybkBJ&>3_) z9p5Tl`nv3|vng!&UxtF*cD%|N|BXSYGe~tOGF+=lT2-yBoqQRpxUj=${k)TX+#iKS z`8A!Jt!%lz2v?~CH1qC)Bm~cy0xxt3Eop~v7el8q!Z_C zb*JUacO}8t&5-ijOhwm)`%7^(`o9kaDN?Dt(V!+>9Zt`h z96!50=y=^;Hsm)YM3T)ZybU(H5VWT9)wIVYAYu-!iB)Xabh-mZGcDceTdm9=*LBz3 zSKB5p40@bRsv92tn}g452didN**ewgyW7jLP5plo-nMR3TYJ5qd``As^XfR>^NoIt zuL;|TJ10}nCv!9itP%cnQdbw!|LzaU>QvWItJuDkcbAOpEY>5B#g>_egpZI=`#?aoMp=zkDRm@}J<%xiyLUGBf0pKG;yBN13F`(j%X82m+)Hp4w2;} zE1w#>jL?x*hdPUs1ic+ zwgt?JNtSH2#Yh%n5+I6(@$aVl0seyJ`E)QDxIyI zF~lw)N00+Tj@%mSxIsdtL5eNtzK;=E}%~s6ObRLwJ9fu-LN_ zYrm<#I~p22EdJbu@sA2T*YZCUpMqCOR4L3fofUarUDZor?c_R-OF?SEn|-H5VH`$G z`cP`9_e8LZ;eYHZsLta<7?9x!Cb8kgd!#Dsf(+|kLekX%cOvP1`iEYs)Z*xT>i^3%>qVoCH3Z0 zj+UW4mTDw8FFsFOPvMyjBJ6`WPpixk}T$3Wd#L$=8F zUg~~rcH8V=s{m)TskHO0AKb2Hv%b|G2nXSUSFFuOYh&4NwcA+mIrc{T)ZtV@7j_#( z7iqy|UP^Omi~r**R#}ISbxUOKnch!HZ8A}h$Ma}$aE8ZNfV19xslHzK$!UB@Rn6ze z%}LGL&Q|WJpT*vM4^xxPrLCu}_PeAFJRMRx1Ob#RK zP#9RYbG)0i>=r+dpGK3C+1uJVvC9$_N;qSmsk9&C+t=6BN?VA@QpdZ4kr-7_KG2*j zW!{vvLHBH9o&cO7rl1zok870LO_4St5XzpV~*o!3Y|?G!RSao>#q${;jhnv-?}J`x>;#HkKAT+@_Yq-8aI^$a(>S_BhnbuYTUy!;f0P`au1C*G7R+U z&f^X!v%H2PXrYr}J0W(ADo*~sZaPIuFspyeqPvbxSXv*!)6A2BU0qTa6`!(73JIg2n`$Q6^1+1vhFOwYHWtD)xcxa3L7HXRW+ z*)=cr`hHHIq*dRv&K)>4!_)U~uC4YMia*dqrQx!7zrRPtRxl}Zu_iKKZj*j$I@2&v zYiHY-Nf#|CBlNc3eQUsQO)0L>hmu)7f5zm(^s*^7>z&o{1T6Kr9^^4&^4Fa2KZ+i zKZ;H(a7*L&+#p4h<+U(pKnejw;O~yG9{tg^ovp2@z91D^fUeliWciTJu1Q`9IBXG7 z83P@u!BgG4a3Nn9a|!6&lDX6Pxu605XM*or$SryW9;5>KwBtSq*a=hoggr@4_i`0X z?l!;Qz<&zSS3rQ;LcFshs1TKQ^~8Y5872wUj$I54nG;}?vUqal9{Bg$pysnJg$TUz{ZsMTiPKLzTpdo zLI$G+G4+EzYPN7zV&kB8JYo25tq%bO{QyutQ}U`SJ<(OXd1NAUqZ9KSTVy{3(NR3R zkyp}Sb}+B9vCUYwN2my@Oj(j7gnFJm1pE^7GQRaPc@y`#Z9*_fh{#uX2| zN5q4X{GRKYl7bob?ct$h_ws zJ{Hu3=BL_z#T@Ano{*-}llu9Z8~66sx+R6ARrc*U%ZnV*LNHYLi%yzw_^y2quIWEo zL8OzS;k5Elhb^dsAx)82!Q}q-2UHYnrtE~g@CBFgf0G3lDPz%(Frf?0_oblkvf?~$ zk$m`bJ|N-=oOjw(*?6IhpY+IM35H^^`sjg^a^tz_4WV%Cv+b%ns{8RR2Dx z8c!bMUF0cBSr-yr0<`WQ<(=p@nbG#>+Q|kp^tL=~$8VK|&j(&R%M=9A6*SO~Ajul6 z&+CMJIkMU9=P8UZbz0w#P@;6TH@ex^dP!Gwj@%I7*CDZT#&IpVm#v6l?79Bv>dekc z-YT208h_%H&F-Y*uTr!%kzZ2Yaar^4psd%7^SRAvw!@W8TZ4$kQ>~e*ec66P{^o7Z7e@DGGc`xzo4KvtNGSKY!;3aO ziW_l2;d$w~J{D1l$~l)t?ddV#W;!Rk?%F$?GL9;9?KI7jq;Sm1+p0szvO(1gxXNh5 z_Nj&kijtik9lVXYu~`pof?FVxcG&rR-W$*T)yrEDJ;T`3xz+xau+n8rfIf;*B1V7Y zT6*ghfU(mj?P*#8@OQgXY}t;g;NA!gQY;S}W>ST<5_1XLv-7snwugbqaV#H#bKYqg zRG|y=cKykDZ^H29c(|H&<@EBc*zjF9$fJTmdzX;ktw{gZxmDUSu9DY#bVjk_a>+=e zJt*6uqhdShU*EH8b>gfnP8eE{o@Yq>DcHsJBOceioNd`u371ki=JN8e{vL`~UUJ7X zA7A@5>rvJFwOdH<5NZuE)GBZk-8#?8WnDz4bC;sm(?c)S4py~!P-Z9Pf#*xSmR+5G zcfFp5T_L1xg0i!6!E&D2Wme5X)A?B#jG=n$3eU7r)i_ON0Dsh+CE?^4%Lbe_X;a#{ zYc|W%+StRUa*|aspI^qlqj=Ja-jO3#XdI^TwmI^N;roty+-v=0o=7r3jvlMff_7^( zt~KEWv(vnw%>TV`;r6$=m~AU_7mYpQOWNh(_k#Tm49@G%Ov z<)*LkKy2w=^BC65YWeSbLdnUe<#ra`FO@9Chi3d)=_3@4^js= zEhvboCK`tSjBMDU%{?qh6zMybfYZOjz~j;V4&STf?|7ws_jO})2m_{8p>h;UhV_JI zo{C%Vtz()+%4^L|m$fDI^~9UPUc@VA@XI$K_1<^r%M1_->S1 z1>Yj|M@kAt?ho;z6Ykcpqh^T$v5z)2|2wbht^&5qPGS{-nUXIm*>`a7C|ja=CD_dql9{G(BI= zILaX$7TMx{Xr6$8mhFV@(kw9$%(28p>PM1AP9ZVi9CHA zbr6Z&wo&4wlHp=+VEhKtyB`m@pwB8@m~{Ou;%v}^`9>AuEM!EB##D&dMq4BR}(4}vcw#w$W3#rCfn zkwe=SEBpzBjMM>GD?Bef2~Y1kO0vtkU6hGH*|!iGcY%RIn#uXT4ynjG>9&USbnec2 ztk;+yFY@=1`m|xb-e`7T+5h|b<5EDTOnerDGNgp30Cgd`-#qgR8IHUc4u|<>Tvd^} zV2}IxR%b1Nj+IIwyw?EJi|ESaWho;6TXqL9KJ2D%bx>r&DC#SQk^n7lyy5`-SYCQ$ zVt+QqpeB;L7zBIwvmtb3)sO^cOeB(x5&jwuJ?1pyK}4?+Yn}wR&0)prD%e{$a*J)g zjHpkwwE{;Zh9TG>K73Y2&6LdD11`CO>byBT5|t7rx9U0tt!<{gJu#2Cq!~F#HuC-~ z!cPcXCDFwk%;XK9xwg(PIQp1;#Sj@nYFzYAUwmHruYS*F!t(2bCjN)a$L>1Zw?t>)!@XCycI; zgztj~n!jEH>bfYOYW2`x_upn(+4r(lv~`LUxLk*u6+x?-tFoyrm033)O=0alx~6-O zIQ*-gV+KAZ0R+t6B44So(Z9B}8+3h4fvA7Ss`f*4cbaQpQ1f#~m~F{$YrVOioFeXY zINS9X!eJflf_U27u$?4gIuA5H7>I4phieJA3i*RUL**-zD_oXV3y;9_za#^b6!r~z z2kzW|piQ>5Xu6)^+|->xQ3L==cn+$_qwC}05-&qkmf?H3OR`TX*!M}ov@He1tLq*c(Hb!8L8Zq+cPV9oZ{- zefvh$nbpN$xSn_C{zupS<=z5vEdC9zXcj%4%i9jd9Mu8D)@W_{8l2{LEBDvEui<>U zfm+`0YHG_zH#<+c84nsG&WrUd4BBmn{r10#;^4I#O1aeU?|_tzw(Qs8T+5A(g(VDi zCs$|h!=!i|rn2K!#waZ9-XV5whXGMZ=qK0MFi0HY*n%vl?~+knaDdBhK^Rv$sFx|< z4vK^FV{DPzEm8X0gz-Y^*H2p zlUztztlxV!Ljy!NT@vaP|zl&bQBr+k6hhsGJGYs ze(0}SR&?unzq!0gB9{Ov84zP;5xPx6uE#Of8Cf1qYRI{QTTq$5f)E9P=P6MQG6$fd znX33#h~|qlzDu30S{i7?Jpxei=PqebZlb?rJ_u~b4+S00Jrvwgvjb z>G6pBd)u9)?eRQS%T~|^I@(tp|5U|E!tKh&HtqC5TB#b^wpEL6DJ+Y zZnYkGLYQ@NN-B+h3A0c7(=-Os;aAJQi6V9}K`%hG)ZEC5>(7O@i?B9Uk4 zphpeEJ7$#N_v({-tkD?vK zQQg*+0Da^ftBY(&!H=g)Os&!%wX8^^sx6{7dJ!j~k49j6t)E(wv1cU>iIejDQdG8N zf}5)%&V6#SUEZUFXqB{QgZhl`4p80i1BaKCi6_~S3dFFXl8`YzFvr7#u^x*bo@0zu zrbe+wXPCfjp0D{Zotn}w-1`uQc%+k~axRvzJBvH@JnNR; zNNj)w1&b~uI&P1EfQBZC9@5r4=!A{Vp9~Vmz9irF<0_V>k;xWqfie<@lP}MgCBlua zLNirWULvX;agJeA+#FX`EYCQNu2-9YO1PuxtG`S_&v4pulux$8Zea_fdn}yN*V0RTUDWO7 ztRub1Xs;zIF{`i4wJ72$%MjF9owXe4680cetc)F~FN>Ucvo2+F+dtob;T_~4R-TEJ zyH{1#Rrzkkbr_$W_m(JVp(r6rM(>h=Ivn3VmNHvO2EYGSCcCiiq{iIEJZQ zSjlE-AxNEivp$>(ObHOy9H4PlAGs8JT4e89h5_;Cga&BfvFzbaN{-;I7Zhi{YdSaJX`;Gn1~rLinA%LeNXu0sG)_Ii!4XzRzj3zFsqdus-RXRNo7c z+Ap=TIfH5b zeOo9TSs(ti3*&w?AEw9rJmO`_zhK^?dGtcJ&a?Ll8p|ZlR7N~ADR@gl-hR&q$(cg< zdG!?6+e62O3VK>(xz#m;qk8p%iGRg$fY3atYD%z`Y3C+so7ZLj15KdQpk6~HR2?K;2HVOkFPhywSQRN(4plaC z@%AoovThXz`vse4V_U0~fvZkKWE>Lw(+7{$8ph%ru$0@ZnsI*qfnnoZdGC6bQNysq z@m7xf%EK)aZW(*9^=8XhR)alblbrN+H~yj8aqE>G(B}k8eoQ{cu5Y%RR-Y+ujNvIQ zVrYEp)Q8E$>O8RNpa`TpnzCSt`#W0c|9uPuoTe5;fWI^*!rwqd$z|u?%<_C(`vbvI z2Wq|mF+WlPKhtgXWJOQQr8r7QPmB3K22y%VVf0go`xc z@|h0cljZve&6)r!4ya9*|$-W>({q5+^Jv?MUjNdU-*{l}G&+VO}wd-T)u+w+V z|G3Dw;xu!#7u<$J;z9a_a728Wh zcCO(^l;Tk}=i0CZ23Zt8sr4L&(d2=ioC@gUf9?o=`ftF2187Mn0*C^Vgti8I6TNPs zLf|4@oyW1Gg>TJcE2^rLkeXRXXDbtcMo8|n8D9?we!>;CX@Z?yCM=MEwz6mBB|rq+ z#P~sM3VzAK#2ZT_>u<<(u?hz7Zu!aJU&V%P{LgrC%cS7{N0NHVp%fqP47wbqL#NmG z%cgB>q?_CUAKaZ;n~iZH1NM)ZI(|jF@i;Q?w1$pLG0?3%48o@$nl2uX!1)mfDZv7x zQ8VBzP71IN2?2=T5L>yxN(n)!WR0h4V8u`=U$JQzv3}>V_bmMhT@uwoqJYJo6_k zhEic1;bvw;DiU8`qqR+$+a&&Axx+Yf;jS?8aJk^FnXd5FQRS>OE)oeoI>Cxx9W!>LW#SK@MRJsOCAlV3e2olEdo~BD&A9b}m4u2sC0M{Knq&9v8p>Nx zO}Y=7GaZ$a#Z$UEf!#SrQl#J^IS3spLJ@4mBBT~8fDajBiL7JvnqYz$g8@?QZDlZ& zc91aV;slr_ucbIuyI$!HR34_9W2{;7MrN?AOPlDJ(%XZ1PvhatW~gtT+3L35?84$` zl#W(ajM8losTrUl^6I_fWv0qSbpjsMgAWuUXMiN2duio5CX{5H;E?o;eqCarp8A-) zM9)Ytuo?ls20}(|Voy{do-b!Wi)nlASyGUC&&T z97NrLf3ZQa-Emwxn(AB6RkNvF>d~87nCJU&iSyT6QKcg2u3R6TVsPM2iH3{26yNWd z4I+OtcOu#0TO;Wd(yaWj&aHhOSo7#6Wra__P6pFJM&mp@DAXX}{GWm+LoBQEx^fww zAMM{TBpS(k>WJp0Sw;62UH5np-)vk+VL z3SXazJ{wm=U%QzEeMEo4>qHBluGJojm+BEiY* z`5=FcypIkEgE&Yuqccq`2|HwdDnvC2*xAF4krDExH9>hD*}}E7?+w&orx;ZQC*rw% z%x2$DPVu4<3YW)SPYqpmEy*98ralXeF=QY z3xh?xaqv>}AVQUV4b^cwnB1L0(xU!#bkgQ>c$GcGi2I~TgnH+A6to0kepAY_@ zx7mhe(Th^A0g}u<^i-`a;4Lg`%O^VaeB(nD+IPZvNo~(U8KX0br4BNHBz20-CR-Wr zUW_v^e!}~*8I+14Q!G{<_L+0z4avsnu_XsrodvNGgun-doI7u3O9kn_xVH9(lPn#? zNvh-8zg7aDj_nA@+=i)7y=5OGQ=E;BO;fc`37NMzbwo@I!;+`-froli@t!ED7KB~1 z_}3TFHxpmgor%na86dOR?r`9rP&Y~F4%q&nGAHSLAksCBJ$nU;6j1B$P~Ep|?)_+9 z^3m~*SLR4SLdjWWV}falzX8-E7XVo4K`VZR<4LK;YGo_`bC6=Iwv=-Pkcxr}a=h%o zqQ$~<)<^;woiT630--yX1(_vbH1|i?F^9$a&)pzi2&5EErdbE=Ie#`B%=3^y6Vb${ z2Vewc#7R$4DV5^7EGl#jo^r|-2`J8}$Qk2)KZI*GnawVqXo3<_XIxNUc-n^QVe*h0>yb@7edCT=-oMj0(R_ls8w9?@nw%OGD zykrsN^vutyN6_l#V_{VNnzy(O=TJEOASQ#L8Q9S+;3q>}aA*iN)jmymC`fRb3eEO> z9XCU?E$f-F^FDJfZfqh#;rcEg~a9z4$`GYs6G2THhio zD%FvJnsm4=e_LY$jm6zQ33d>cUA0@?8>AUcg%o+AanVXL_?I%h<$bcSDk^Gf;mm;@ z7yajld_E8G%)bUJ3j9TUE+>r&P_?*xr%ZGq&3yL$`0s%0{b$|)SKa3TXD5D~piB-G z{NG9;&w`dzv1F1<@|FK80oaYnsKBcNa`~JOO{*xp*^S=X_c>mjj_dwjQWz!3L@IJE!3P10*(WkEL^Wb0N$(40^ zoHmYrD(4l>vO+)ahKSZ5^q~#=yr4|rNk~+S$IH2WlIUR>RAHBT1Fo?kiu)Y{NdJuO zehrB!vo#xKTT=8~E(#+A+d^6JAcSG2ifBJ~-v{$v<#*;yE9-=2v?x_|EdiU`LR2RhvDOP1Rju0wT!=O2s!U+$r4@wzkehSe1W}DP#>KhuJNP_7xtm76uG(j@4 z2V^Liqd+G2+^%Lqwv;2((ch>D)5v_2>g&k(2^p=z%1P>SnVm$RTTjs0gU4-TuXO6_ zvf2Fh2>$vRn50vhB7=FvCTSqcg4)LZ-X0=eX+`OvHG5M^&6qg4#j$atN+JPikkM4(uZ zRcGxSjyfFlWRlmViJW_Jnh98do;%|JA{7)hwc3+e47p?bex18Fnu~B`aJ+Ai5m3=ke|GK5a>$D z8t38@%wEMFyPd!5m}xw><-hy^H4n=nN>cpuIwdTc*aAP$`;+>DuUj|1X zYHr99h>Cb<;n)0qeWyHFIVY+2amlPc=15bA9muPSMn$5`sJ5&1d;h`YdRWLXyJfT3 zW47m0ZDoVzv6?ZG5imZJ_wt1U5nnihJ3$2-N0Z2LcARcYk zB*?65WVOV{T@rYWHYj{9APIf#F!Ei$!sEo9?f zidtcg|L@I&h^mj)Xyi;K5Nl$f97FiHVx*7fYagR`OmkWqm)&~xX+^K@gdE7T)lo{+ zR#3Dsp2o!iT&~^i9>AmPJS~j+3xHeff@GF?3g&CqomU4R{!!l3G=C9(=7#{7U?}7T zvdRr}g9U>lY^o%~hhwcTnG@VQM71f1&GrU~16sRpxgzY7oscw%pi#`hkZ;t2ZOV1x z|0TxPi-M2IikfcG=#t1XXi?43if&-0K$=q}oJ{aLG)3#j4?4IGOkty!k~^uH_}&do zZpmhxnIhopI_)V0QHy|4e@BS|YI=qU7d!D2F6K+eqP#u`x5B@N)S-VC=BdEOQ)8h1 zO;z;JEYhXn=Cw(-wwSawx74l9KhBhn$SKqK@2ibv9W|LmUE(wr#|PCT-0n-gsN0&*s*oUY94u!NAKY96HqoD^(i|OC!$Ch zI!)+?A3%eep_q=S(r7K&;rkk_rIPb2WD7FT*w)^i3lh*lU-z*`Lc*_gz&xTQsqYnt z_OE9Z;&SVY%!e~uM=^fdzuoTR7Ec^)0&GIkGs7IpYkziTR~~DsYLgnqI_wb@rBDNU zPNs{Jl)VSNN^?%f+H-Sc!8+!GE>?dOhceN?J4rEfPRo*c@T_lU=Ikh@;&9c0%>*M1 z4DzSc_+|oS7}BUDOBQ0+WOuy=wR=P}28-fh2c-Ldd$eM~)|&Q;L&m*ICnsQ5&X)z( zbLRszgIPfqyz7M>EFD`jt@VGxZII6gFsVI{>sd)+Pd)bQ21b^<%CA(sqt2Q2qen4N z55nh`hg(5}#Aya~7^;1+TZTu!4Bb3&gr>nGaLVMG`O5edn*dIY=|bqvYH_HtDEUwVoygE+#*j)ZpI7jr`ogku z2~=uRQ31(G7YF_9z3Y82zF`;qZ%{o_Y`xLeT^)B`-rvmKeMYUgmrjaU8@sEMk7>Vo zdfZ#A(oiAa77(yMLV_d{Ii(NpX7=BpPxeAK(%O!@suRzVPd1y#4HMzbj7o>S*wG*G zPml)cx2VOEriu7g{-k1O`ve7`&P8VvCTl3Yrg(f&zD?ToL`mNL{BP6S|I&1R@{|Oc z?ieMaQT}4{lGSD%79;NEa1DbaXn=U!x+E3C$mOUUPVNsahBAe) zj(#l9{n*EKR~#rAElBo_!Q>z(j};8l3h=)^<31puh|a}*?6P!_JdR(d7{41;OK$eZ z(t_=onU%>XK#W^i>4uII#w-$8CeSzkFS6c&Ig>B!`i&;W#I`lDZQHh;iEZ1qGqG*s zj&0lK$^U)oJyqvCRs8|Fs=N2@eXX^Ai<%SJt(>GuUbG*5l$gYVNcwz;Jl2IYo>H`& zz-vE$SdQ(6e=I3MB17DTx^Pjx#^%BIX$A(fl=0#2{h7AybT2Rxk^YFmQc%&{L;TOL zHgk6oLp+uuu1>AY2V#(b_yJ@RG8*QNp^#+)NH|dx*PulWqd^zFP5bq% zCXS8)D&L#KmgmgI@3cn8yO(q{cNQqIF*k)R5oR7yLXZs%O=GSdbMYcjHdF>QeBZyv zP_>uc*ahj41KFC`tBVPxo7R(%cjPFWa;?#6=JmdgZMve-`gN_Eu}$DA}oV3wEY z+T7>;11qYzo%dE8gYmj>64mzB3TT?_L!vLn_(5ZclHl!CVf6ssRwVV*8{qAVs~RHS zv^<7d(N7Bm{!mO;iCuXGN%?{{|MWmR)fy%OI8GtgfkT3XbDGvW>a*Mm!z39|!m0do zZJNz0kXS%{g>mL1e(Z_1ClLZ}&6^3E=ixtBrgGtK6yHHgdTpOnoVf@%zQDsGD*IR-sfBw&Z>~6|}iTS0#v?4hD0C z%!LVbsM7<36`xpzXjlK0@O9aHOUW%s7)j_{x?bsJw|`C2DewH#MtS*j&74nS&UzOU z@RnN(NDC@#JRZzampVpU#5ER(lN}cdFih0S(b}?%yiDd%CkN)OFQ4{OI9; zam`SvPA&srlW#JULzx;BA9w;4fDOYz>(0To9WAS!+!UMgIX8}xL4#pjddQk>R9n=chhL3RP0vN%Xm=v=qVyn3P~v4Q|Yyd9A&jli>7Qimpqz

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+ Operation + + Result prior to Spark 3.0 + + Result starting Spark 3.0 +
+ CAST('infinity' AS DOUBLE)
+ CAST('+infinity' AS DOUBLE)
+ CAST('inf' AS DOUBLE)
+ CAST('+inf' AS DOUBLE)
+
+ NULL + + Double.PositiveInfinity +
+ CAST('-infinity' AS DOUBLE)
+ CAST('-inf' AS DOUBLE)
+
+ NULL + + Double.NegativeInfinity +
+ CAST('infinity' AS FLOAT)
+ CAST('+infinity' AS FLOAT)
+ CAST('inf' AS FLOAT)
+ CAST('+inf' AS FLOAT)
+
+ NULL + + Float.PositiveInfinity +
+ CAST('-infinity' AS FLOAT)
+ CAST('-inf' AS FLOAT)
+
+ NULL + + Float.NegativeInfinity +
+ CAST('nan' AS DOUBLE) + + NULL + + Double.NaN +
+ CAST('nan' AS FLOAT) + + NULL + + Float.NaN +
+ ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was 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 a0cb5da078438..32e2707948919 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.math.{BigDecimal => JavaBigDecimal} import java.time.ZoneId +import java.util.Locale import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkException @@ -193,6 +194,22 @@ object Cast { } def resolvableNullability(from: Boolean, to: Boolean): Boolean = !from || to + + /** + * We process literals such as 'Infinity', 'Inf', '-Infinity' and 'NaN' etc in case + * insensitive manner to be compatible with other database systems such as PostgreSQL and DB2. + */ + def processFloatingPointSpecialLiterals(v: String, isFloat: Boolean): Any = { + v.trim.toLowerCase(Locale.ROOT) match { + case "inf" | "+inf" | "infinity" | "+infinity" => + if (isFloat) Float.PositiveInfinity else Double.PositiveInfinity + case "-inf" | "-infinity" => + if (isFloat) Float.NegativeInfinity else Double.NegativeInfinity + case "nan" => + if (isFloat) Float.NaN else Double.NaN + case _ => null + } + } } /** @@ -563,8 +580,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // DoubleConverter private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => try s.toString.toDouble catch { - case _: NumberFormatException => null + buildCast[UTF8String](_, s => { + val doubleStr = s.toString + try doubleStr.toDouble catch { + case _: NumberFormatException => + Cast.processFloatingPointSpecialLiterals(doubleStr, false) + } }) case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) @@ -579,8 +600,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // FloatConverter private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => try s.toString.toFloat catch { - case _: NumberFormatException => null + buildCast[UTF8String](_, s => { + val floatStr = s.toString + try floatStr.toFloat catch { + case _: NumberFormatException => + Cast.processFloatingPointSpecialLiterals(floatStr, true) + } }) case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) @@ -718,9 +743,9 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case ByteType => castToByteCode(from, ctx) case ShortType => castToShortCode(from, ctx) case IntegerType => castToIntCode(from, ctx) - case FloatType => castToFloatCode(from) + case FloatType => castToFloatCode(from, ctx) case LongType => castToLongCode(from, ctx) - case DoubleType => castToDoubleCode(from) + case DoubleType => castToDoubleCode(from, ctx) case array: ArrayType => castArrayCode(from.asInstanceOf[ArrayType].elementType, array.elementType, ctx) @@ -1260,48 +1285,66 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String (c, evPrim, evNull) => code"$evPrim = (long) $c;" } - private[this] def castToFloatCode(from: DataType): CastFunction = from match { - case StringType => - (c, evPrim, evNull) => - code""" + private[this] def castToFloatCode(from: DataType, ctx: CodegenContext): CastFunction = { + from match { + case StringType => + val floatStr = ctx.freshVariable("floatStr", StringType) + (c, evPrim, evNull) => + code""" + final String $floatStr = $c.toString(); try { - $evPrim = Float.valueOf($c.toString()); + $evPrim = Float.valueOf($floatStr); } catch (java.lang.NumberFormatException e) { - $evNull = true; + final Float f = (Float) Cast.processFloatingPointSpecialLiterals($floatStr, true); + if (f == null) { + $evNull = true; + } else { + $evPrim = f.floatValue(); + } } """ - case BooleanType => - (c, evPrim, evNull) => code"$evPrim = $c ? 1.0f : 0.0f;" - case DateType => - (c, evPrim, evNull) => code"$evNull = true;" - case TimestampType => - (c, evPrim, evNull) => code"$evPrim = (float) (${timestampToDoubleCode(c)});" - case DecimalType() => - (c, evPrim, evNull) => code"$evPrim = $c.toFloat();" - case x: NumericType => - (c, evPrim, evNull) => code"$evPrim = (float) $c;" + case BooleanType => + (c, evPrim, evNull) => code"$evPrim = $c ? 1.0f : 0.0f;" + case DateType => + (c, evPrim, evNull) => code"$evNull = true;" + case TimestampType => + (c, evPrim, evNull) => code"$evPrim = (float) (${timestampToDoubleCode(c)});" + case DecimalType() => + (c, evPrim, evNull) => code"$evPrim = $c.toFloat();" + case x: NumericType => + (c, evPrim, evNull) => code"$evPrim = (float) $c;" + } } - private[this] def castToDoubleCode(from: DataType): CastFunction = from match { - case StringType => - (c, evPrim, evNull) => - code""" + private[this] def castToDoubleCode(from: DataType, ctx: CodegenContext): CastFunction = { + from match { + case StringType => + val doubleStr = ctx.freshVariable("doubleStr", StringType) + (c, evPrim, evNull) => + code""" + final String $doubleStr = $c.toString(); try { - $evPrim = Double.valueOf($c.toString()); + $evPrim = Double.valueOf($doubleStr); } catch (java.lang.NumberFormatException e) { - $evNull = true; + final Double d = (Double) Cast.processFloatingPointSpecialLiterals($doubleStr, false); + if (d == null) { + $evNull = true; + } else { + $evPrim = d.doubleValue(); + } } """ - case BooleanType => - (c, evPrim, evNull) => code"$evPrim = $c ? 1.0d : 0.0d;" - case DateType => - (c, evPrim, evNull) => code"$evNull = true;" - case TimestampType => - (c, evPrim, evNull) => code"$evPrim = ${timestampToDoubleCode(c)};" - case DecimalType() => - (c, evPrim, evNull) => code"$evPrim = $c.toDouble();" - case x: NumericType => - (c, evPrim, evNull) => code"$evPrim = (double) $c;" + case BooleanType => + (c, evPrim, evNull) => code"$evPrim = $c ? 1.0d : 0.0d;" + case DateType => + (c, evPrim, evNull) => code"$evNull = true;" + case TimestampType => + (c, evPrim, evNull) => code"$evPrim = ${timestampToDoubleCode(c)};" + case DecimalType() => + (c, evPrim, evNull) => code"$evPrim = $c.toDouble();" + case x: NumericType => + (c, evPrim, evNull) => code"$evPrim = (double) $c;" + } } private[this] def castArrayCode( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 69adb8e922cc3..1f9fa22d30e13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -1049,4 +1049,30 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { Cast(Literal(134.12), DecimalType(3, 2)), "cannot be represented") } } + + test("Process Infinity, -Infinity, NaN in case insensitive manner") { + Seq("inf", "+inf", "infinity", "+infiNity", " infinity ").foreach { value => + checkEvaluation(cast(value, FloatType), Float.PositiveInfinity) + } + Seq("-infinity", "-infiniTy", " -infinity ", " -inf ").foreach { value => + checkEvaluation(cast(value, FloatType), Float.NegativeInfinity) + } + Seq("inf", "+inf", "infinity", "+infiNity", " infinity ").foreach { value => + checkEvaluation(cast(value, DoubleType), Double.PositiveInfinity) + } + Seq("-infinity", "-infiniTy", " -infinity ", " -inf ").foreach { value => + checkEvaluation(cast(value, DoubleType), Double.NegativeInfinity) + } + Seq("nan", "nAn", " nan ").foreach { value => + checkEvaluation(cast(value, FloatType), Float.NaN) + } + Seq("nan", "nAn", " nan ").foreach { value => + checkEvaluation(cast(value, DoubleType), Double.NaN) + } + + // Invalid literals when casted to double and float results in null. + Seq(DoubleType, FloatType).foreach { dataType => + checkEvaluation(cast("badvalue", dataType), null) + } + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql index 801a16cf41f54..5d54be9341148 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql @@ -59,16 +59,14 @@ select avg(CAST(null AS DOUBLE)) from range(1,4); select sum(CAST('NaN' AS DOUBLE)) from range(1,4); select avg(CAST('NaN' AS DOUBLE)) from range(1,4); --- [SPARK-27768] verify correct results for infinite inputs SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('Infinity' AS DOUBLE))) v(x); +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('Infinity'), ('1')) v(x); +FROM (VALUES ('infinity'), ('1')) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('Infinity'), ('Infinity')) v(x); +FROM (VALUES ('infinity'), ('infinity')) v(x); SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('-Infinity'), ('Infinity')) v(x); - +FROM (VALUES ('-infinity'), ('infinity')) v(x); -- test accuracy with a large input offset SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql index 3dad5cd56ba02..058467695a608 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql @@ -38,7 +38,6 @@ INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20'); -- special inputs SELECT float('NaN'); --- [SPARK-28060] Float type can not accept some special inputs SELECT float('nan'); SELECT float(' NAN '); SELECT float('infinity'); @@ -49,7 +48,6 @@ SELECT float('N A N'); SELECT float('NaN x'); SELECT float(' INFINITY x'); --- [SPARK-28060] Float type can not accept some special inputs SELECT float('Infinity') + 100.0; SELECT float('Infinity') / float('Infinity'); SELECT float('nan') / float('nan'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql index 6f8e3b596e60e..957dabdebab4e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql @@ -37,7 +37,6 @@ SELECT double('-10e-400'); -- special inputs SELECT double('NaN'); --- [SPARK-28060] Double type can not accept some special inputs SELECT double('nan'); SELECT double(' NAN '); SELECT double('infinity'); @@ -49,7 +48,6 @@ SELECT double('NaN x'); SELECT double(' INFINITY x'); SELECT double('Infinity') + 100.0; --- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner SELECT double('Infinity') / double('Infinity'); SELECT double('NaN') / double('NaN'); -- [SPARK-28315] Decimal can not accept NaN as input @@ -190,7 +188,7 @@ SELECT tanh(double('1')); SELECT asinh(double('1')); SELECT acosh(double('2')); SELECT atanh(double('0.5')); --- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner + -- test Inf/NaN cases for hyperbolic functions SELECT sinh(double('Infinity')); SELECT sinh(double('-Infinity')); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out index 51ca1d558691c..29bafb42f579e 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out @@ -236,7 +236,7 @@ NaN -- !query 29 SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('Infinity' AS DOUBLE))) v(x) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x) -- !query 29 schema struct -- !query 29 output @@ -245,7 +245,7 @@ Infinity NaN -- !query 30 SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('Infinity'), ('1')) v(x) +FROM (VALUES ('infinity'), ('1')) v(x) -- !query 30 schema struct -- !query 30 output @@ -254,7 +254,7 @@ Infinity NaN -- !query 31 SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('Infinity'), ('Infinity')) v(x) +FROM (VALUES ('infinity'), ('infinity')) v(x) -- !query 31 schema struct -- !query 31 output @@ -263,7 +263,7 @@ Infinity NaN -- !query 32 SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) -FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +FROM (VALUES ('-infinity'), ('infinity')) v(x) -- !query 32 schema struct -- !query 32 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out index 86d88007d8892..6e47cff91a7d5 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out @@ -63,7 +63,7 @@ SELECT float('nan') -- !query 7 schema struct -- !query 7 output -NULL +NaN -- !query 8 @@ -71,7 +71,7 @@ SELECT float(' NAN ') -- !query 8 schema struct -- !query 8 output -NULL +NaN -- !query 9 @@ -79,7 +79,7 @@ SELECT float('infinity') -- !query 9 schema struct -- !query 9 output -NULL +Infinity -- !query 10 @@ -87,7 +87,7 @@ SELECT float(' -INFINiTY ') -- !query 10 schema struct -- !query 10 output -NULL +-Infinity -- !query 11 @@ -135,7 +135,7 @@ SELECT float('nan') / float('nan') -- !query 16 schema struct<(CAST(CAST(nan AS FLOAT) AS DOUBLE) / CAST(CAST(nan AS FLOAT) AS DOUBLE)):double> -- !query 16 output -NULL +NaN -- !query 17 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out index eb9e8aa6361a1..b4ea3c1ad1cab 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -95,7 +95,7 @@ SELECT double('nan') -- !query 11 schema struct -- !query 11 output -NULL +NaN -- !query 12 @@ -103,7 +103,7 @@ SELECT double(' NAN ') -- !query 12 schema struct -- !query 12 output -NULL +NaN -- !query 13 @@ -111,7 +111,7 @@ SELECT double('infinity') -- !query 13 schema struct -- !query 13 output -NULL +Infinity -- !query 14 @@ -119,7 +119,7 @@ SELECT double(' -INFINiTY ') -- !query 14 schema struct -- !query 14 output -NULL +-Infinity -- !query 15 From 0343854f54b48b206ca434accec99355011560c2 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Aug 2019 17:29:54 -0700 Subject: [PATCH 062/149] [SPARK-28487][K8S] More responsive dynamic allocation with K8S This change implements a few changes to the k8s pod allocator so that it behaves a little better when dynamic allocation is on. (i) Allow the application to ramp up immediately when there's a change in the target number of executors. Without this change, scaling would only trigger when a change happened in the state of the cluster, e.g. an executor going down, or when the periodical snapshot was taken (default every 30s). (ii) Get rid of pending pod requests, both acknowledged (i.e. Spark knows that a pod is pending resource allocation) and unacknowledged (i.e. Spark has requested the pod but the API server hasn't created it yet), when they're not needed anymore. This avoids starting those executors to just remove them after the idle timeout, wasting resources in the meantime. (iii) Re-work some of the code to avoid unnecessary logging. While not bad without dynamic allocation, the existing logging was very chatty when dynamic allocation was on. With the changes, all the useful information is still there, but only when interesting changes happen. (iv) Gracefully shut down executors when they become idle. Just deleting the pod causes a lot of ugly logs to show up, so it's better to ask pods to exit nicely. That also allows Spark to respect the "don't delete pods" option when dynamic allocation is on. Tested on a small k8s cluster running different TPC-DS workloads. Closes #25236 from vanzin/SPARK-28487. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../scheduler/dynalloc/ExecutorMonitor.scala | 14 +- .../org/apache/spark/deploy/k8s/Config.scala | 6 + .../cluster/k8s/ExecutorPodsAllocator.scala | 204 ++++++++++++------ .../k8s/ExecutorPodsLifecycleManager.scala | 10 +- .../k8s/ExecutorPodsSnapshotsStore.scala | 2 + .../k8s/ExecutorPodsSnapshotsStoreImpl.scala | 22 +- .../k8s/KubernetesClusterManager.scala | 6 +- .../KubernetesClusterSchedulerBackend.scala | 79 ++++--- ...erministicExecutorPodsSnapshotsStore.scala | 2 +- .../k8s/ExecutorPodsAllocatorSuite.scala | 64 ++++-- ...bernetesClusterSchedulerBackendSuite.scala | 56 +++-- 11 files changed, 323 insertions(+), 142 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index d0337b6e34962..2a5b91f364b2c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -193,8 +193,10 @@ private[spark] class ExecutorMonitor( } } - logDebug(s"Activated executors $activatedExecs due to shuffle data needed by new job" + - s"${event.jobId}.") + if (activatedExecs.nonEmpty) { + logDebug(s"Activated executors $activatedExecs due to shuffle data needed by new job" + + s"${event.jobId}.") + } if (needTimeoutUpdate) { nextTimeout.set(Long.MinValue) @@ -243,8 +245,10 @@ private[spark] class ExecutorMonitor( } } - logDebug(s"Executors $deactivatedExecs do not have active shuffle data after job " + - s"${event.jobId} finished.") + if (deactivatedExecs.nonEmpty) { + logDebug(s"Executors $deactivatedExecs do not have active shuffle data after job " + + s"${event.jobId} finished.") + } } jobToStageIDs.remove(event.jobId).foreach { stages => @@ -506,6 +510,8 @@ private[spark] class ExecutorMonitor( } } + def nonEmpty: Boolean = ids != null && ids.nonEmpty + override def toString(): String = { ids.mkString(",") + (if (excess > 0) s" (and $excess more)" else "") } 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 1af8dfbe4385a..54afe92e81567 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 @@ -330,6 +330,12 @@ private[spark] object Config extends Logging { .booleanConf .createWithDefault(true) + val KUBERNETES_DYN_ALLOC_KILL_GRACE_PERIOD = + ConfigBuilder("spark.kubernetes.dynamicAllocation.deleteGracePeriod") + .doc("How long to wait for executors to shut down gracefully before a forceful kill.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("5s") + val KUBERNETES_SUBMIT_GRACE_PERIOD = ConfigBuilder("spark.kubernetes.appKillPodDeletionGracePeriod") .doc("Time to wait for graceful deletion of Spark pods when spark-submit" + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index da3edfeca9b1f..2201bf91d3905 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicLong} import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.client.KubernetesClient @@ -66,15 +66,28 @@ private[spark] class ExecutorPodsAllocator( // snapshot yet. Mapped to the timestamp when they were created. private val newlyCreatedExecutors = mutable.Map.empty[Long, Long] + private val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(conf) + + private val hasPendingPods = new AtomicBoolean() + + private var lastSnapshot = ExecutorPodsSnapshot(Nil) + def start(applicationId: String): Unit = { snapshotsStore.addSubscriber(podAllocationDelay) { onNewSnapshots(applicationId, _) } } - def setTotalExpectedExecutors(total: Int): Unit = totalExpectedExecutors.set(total) + def setTotalExpectedExecutors(total: Int): Unit = { + totalExpectedExecutors.set(total) + if (!hasPendingPods.get()) { + snapshotsStore.notifySubscribers() + } + } - private def onNewSnapshots(applicationId: String, snapshots: Seq[ExecutorPodsSnapshot]): Unit = { + private def onNewSnapshots( + applicationId: String, + snapshots: Seq[ExecutorPodsSnapshot]): Unit = synchronized { newlyCreatedExecutors --= snapshots.flatMap(_.executorPods.keys) // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, @@ -82,81 +95,138 @@ private[spark] class ExecutorPodsAllocator( // handled the creation request), or the API server created the pod but we missed // both the creation and deletion events. In either case, delete the missing pod // if possible, and mark such a pod to be rescheduled below. - newlyCreatedExecutors.foreach { case (execId, timeCreated) => - val currentTime = clock.getTimeMillis() + val currentTime = clock.getTimeMillis() + val timedOut = newlyCreatedExecutors.flatMap { case (execId, timeCreated) => if (currentTime - timeCreated > podCreationTimeout) { - logWarning(s"Executor with id $execId was not detected in the Kubernetes" + - s" cluster after $podCreationTimeout milliseconds despite the fact that a" + - " previous allocation attempt tried to create it. The executor may have been" + - " deleted but the application missed the deletion event.") - - if (shouldDeleteExecutors) { - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId) - .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) - .delete() - } - } - newlyCreatedExecutors -= execId + Some(execId) } else { logDebug(s"Executor with id $execId was not found in the Kubernetes cluster since it" + s" was created ${currentTime - timeCreated} milliseconds ago.") + None } } - if (snapshots.nonEmpty) { - // Only need to examine the cluster as of the latest snapshot, the "current" state, to see if - // we need to allocate more executors or not. - val latestSnapshot = snapshots.last - val currentRunningExecutors = latestSnapshot.executorPods.values.count { - case PodRunning(_) => true - case _ => false + if (timedOut.nonEmpty) { + logWarning(s"Executors with ids ${timedOut.mkString(",")} were not detected in the" + + s" Kubernetes cluster after $podCreationTimeout ms despite the fact that a previous" + + " allocation attempt tried to create them. The executors may have been deleted but the" + + " application missed the deletion event.") + + newlyCreatedExecutors --= timedOut + if (shouldDeleteExecutors) { + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withLabelIn(SPARK_EXECUTOR_ID_LABEL, timedOut.toSeq.map(_.toString): _*) + .delete() + } } - val currentPendingExecutors = latestSnapshot.executorPods.values.count { - case PodPending(_) => true + } + + if (snapshots.nonEmpty) { + lastSnapshot = snapshots.last + } + + val currentRunningCount = lastSnapshot.executorPods.values.count { + case PodRunning(_) => true + case _ => false + } + + val currentPendingExecutors = lastSnapshot.executorPods + .filter { + case (_, PodPending(_)) => true case _ => false } - val currentTotalExpectedExecutors = totalExpectedExecutors.get - logDebug(s"Currently have $currentRunningExecutors running executors and" + - s" $currentPendingExecutors pending executors. $newlyCreatedExecutors executors" + - s" have been requested but are pending appearance in the cluster.") - if (newlyCreatedExecutors.isEmpty - && currentPendingExecutors == 0 - && currentRunningExecutors < currentTotalExpectedExecutors) { - val numExecutorsToAllocate = math.min( - currentTotalExpectedExecutors - currentRunningExecutors, podAllocationSize) - logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") - for ( _ <- 0 until numExecutorsToAllocate) { - val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() - val executorConf = KubernetesConf.createExecutorConf( - conf, - newExecutorId.toString, - applicationId, - driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr, - kubernetesClient) - val podWithAttachedContainer = new PodBuilder(executorPod.pod) - .editOrNewSpec() - .addToContainers(executorPod.container) - .endSpec() - .build() - kubernetesClient.pods().create(podWithAttachedContainer) - newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() - logDebug(s"Requested executor with id $newExecutorId from Kubernetes.") + .map { case (id, _) => id } + + if (snapshots.nonEmpty) { + logDebug(s"Pod allocation status: $currentRunningCount running, " + + s"${currentPendingExecutors.size} pending, " + + s"${newlyCreatedExecutors.size} unacknowledged.") + } + + val currentTotalExpectedExecutors = totalExpectedExecutors.get + + // This variable is used later to print some debug logs. It's updated when cleaning up + // excess pod requests, since currentPendingExecutors is immutable. + var knownPendingCount = currentPendingExecutors.size + + // It's possible that we have outstanding pods that are outdated when dynamic allocation + // decides to downscale the application. So check if we can release any pending pods early + // instead of waiting for them to time out. Drop them first from the unacknowledged list, + // then from the pending. + // + // TODO: with dynamic allocation off, handle edge cases if we end up with more running + // executors than expected. + val knownPodCount = currentRunningCount + currentPendingExecutors.size + + newlyCreatedExecutors.size + if (knownPodCount > currentTotalExpectedExecutors) { + val excess = knownPodCount - currentTotalExpectedExecutors + val knownPendingToDelete = currentPendingExecutors.take(excess - newlyCreatedExecutors.size) + val toDelete = newlyCreatedExecutors.keys.take(excess).toList ++ knownPendingToDelete + + if (toDelete.nonEmpty) { + logInfo(s"Deleting ${toDelete.size} excess pod requests (${toDelete.mkString(",")}).") + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withField("status.phase", "Pending") + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withLabelIn(SPARK_EXECUTOR_ID_LABEL, toDelete.sorted.map(_.toString): _*) + .delete() + newlyCreatedExecutors --= toDelete + knownPendingCount -= knownPendingToDelete.size } - } else if (currentRunningExecutors >= currentTotalExpectedExecutors) { - // TODO handle edge cases if we end up with more running executors than expected. - logDebug("Current number of running executors is equal to the number of requested" + - " executors. Not scaling up further.") - } else if (newlyCreatedExecutors.nonEmpty || currentPendingExecutors != 0) { - logDebug(s"Still waiting for ${newlyCreatedExecutors.size + currentPendingExecutors}" + - s" executors to begin running before requesting for more executors. # of executors in" + - s" pending status in the cluster: $currentPendingExecutors. # of executors that we have" + - s" created but we have not observed as being present in the cluster yet:" + - s" ${newlyCreatedExecutors.size}.") + } + } + + if (newlyCreatedExecutors.isEmpty + && currentPendingExecutors.isEmpty + && currentRunningCount < currentTotalExpectedExecutors) { + val numExecutorsToAllocate = math.min( + currentTotalExpectedExecutors - currentRunningCount, podAllocationSize) + logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") + for ( _ <- 0 until numExecutorsToAllocate) { + val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() + val executorConf = KubernetesConf.createExecutorConf( + conf, + newExecutorId.toString, + applicationId, + driverPod) + val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr, + kubernetesClient) + val podWithAttachedContainer = new PodBuilder(executorPod.pod) + .editOrNewSpec() + .addToContainers(executorPod.container) + .endSpec() + .build() + kubernetesClient.pods().create(podWithAttachedContainer) + newlyCreatedExecutors(newExecutorId) = clock.getTimeMillis() + logDebug(s"Requested executor with id $newExecutorId from Kubernetes.") + } + } + + // Update the flag that helps the setTotalExpectedExecutors() callback avoid triggering this + // update method when not needed. + hasPendingPods.set(knownPendingCount + newlyCreatedExecutors.size > 0) + + // The code below just prints debug messages, which are only useful when there's a change + // in the snapshot state. Since the messages are a little spammy, avoid them when we know + // there are no useful updates. + if (!log.isDebugEnabled || snapshots.isEmpty) { + return + } + + if (currentRunningCount >= currentTotalExpectedExecutors && !dynamicAllocationEnabled) { + logDebug("Current number of running executors is equal to the number of requested" + + " executors. Not scaling up further.") + } else { + val outstanding = knownPendingCount + newlyCreatedExecutors.size + if (outstanding > 0) { + logDebug(s"Still waiting for $outstanding executors before requesting more.") } } } 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 95e1ba8362a02..d6b75824d05d8 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 @@ -68,9 +68,13 @@ private[spark] class ExecutorPodsLifecycleManager( s" pod name ${state.pod.getMetadata.getName}") onFinalNonDeletedState(failed, execId, schedulerBackend, execIdsRemovedInThisRound) case succeeded@PodSucceeded(_) => - logDebug(s"Snapshot reported succeeded executor with id $execId," + - s" pod name ${state.pod.getMetadata.getName}. Note that succeeded executors are" + - s" unusual unless Spark specifically informed the executor to exit.") + if (schedulerBackend.isExecutorActive(execId.toString)) { + logInfo(s"Snapshot reported succeeded executor with id $execId, " + + "even though the application has not requested for it to be removed.") + } else { + logDebug(s"Snapshot reported succeeded executor with id $execId," + + s" pod name ${state.pod.getMetadata.getName}.") + } onFinalNonDeletedState(succeeded, execId, schedulerBackend, execIdsRemovedInThisRound) case _ => } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala index dd264332cf9e8..d50ea85d3757b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStore.scala @@ -26,6 +26,8 @@ private[spark] trait ExecutorPodsSnapshotsStore { def stop(): Unit + def notifySubscribers(): Unit + def updatePod(updatedPod: Pod): Unit def replaceSnapshot(newSnapshot: Seq[Pod]): Unit diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 010d93fbf8470..8aa20bfbc82db 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -52,8 +52,8 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul private val SNAPSHOT_LOCK = new Object() - private val subscribers = mutable.Buffer.empty[SnapshotsSubscriber] - private val pollingTasks = mutable.Buffer.empty[Future[_]] + private val subscribers = new CopyOnWriteArrayList[SnapshotsSubscriber]() + private val pollingTasks = new CopyOnWriteArrayList[Future[_]] @GuardedBy("SNAPSHOT_LOCK") private var currentSnapshot = ExecutorPodsSnapshot() @@ -66,16 +66,24 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul SNAPSHOT_LOCK.synchronized { newSubscriber.snapshotsBuffer.add(currentSnapshot) } - subscribers += newSubscriber - pollingTasks += subscribersExecutor.scheduleWithFixedDelay( + subscribers.add(newSubscriber) + pollingTasks.add(subscribersExecutor.scheduleWithFixedDelay( () => callSubscriber(newSubscriber), 0L, processBatchIntervalMillis, - TimeUnit.MILLISECONDS) + TimeUnit.MILLISECONDS)) + } + + override def notifySubscribers(): Unit = SNAPSHOT_LOCK.synchronized { + subscribers.asScala.foreach { s => + subscribersExecutor.submit(new Runnable() { + override def run(): Unit = callSubscriber(s) + }) + } } override def stop(): Unit = { - pollingTasks.foreach(_.cancel(true)) + pollingTasks.asScala.foreach(_.cancel(false)) ThreadUtils.shutdown(subscribersExecutor) } @@ -90,7 +98,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul } private def addCurrentSnapshotToSubscribers(): Unit = { - subscribers.foreach { subscriber => + subscribers.asScala.foreach { subscriber => subscriber.snapshotsBuffer.add(currentSnapshot) } } 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 31ca06b721c5d..6fff9dd4f9443 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 @@ -77,8 +77,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) } - val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( - "kubernetes-executor-requests") + val schedulerExecutorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor( + "kubernetes-executor-maintenance") val subscribersExecutor = ThreadUtils .newDaemonThreadPoolScheduledExecutor( @@ -114,7 +114,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit scheduler.asInstanceOf[TaskSchedulerImpl], sc, kubernetesClient, - requestExecutorsService, + schedulerExecutorService, snapshotsStore, executorPodsAllocator, executorPodsLifecycleEventHandler, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 4a91a2f5f10b3..e221a926daca8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.scheduler.cluster.k8s -import java.util.concurrent.ExecutorService +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.Future import io.fabric8.kubernetes.client.KubernetesClient @@ -27,8 +27,8 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO -import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} -import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} +import org.apache.spark.rpc.RpcAddress +import org.apache.spark.scheduler.{ExecutorKilled, ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} import org.apache.spark.util.{ThreadUtils, Utils} @@ -36,7 +36,7 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, kubernetesClient: KubernetesClient, - requestExecutorsService: ExecutorService, + executorService: ScheduledExecutorService, snapshotsStore: ExecutorPodsSnapshotsStore, podAllocator: ExecutorPodsAllocator, lifecycleEventHandler: ExecutorPodsLifecycleManager, @@ -44,9 +44,6 @@ private[spark] class KubernetesClusterSchedulerBackend( pollEvents: ExecutorPodsPollingSnapshotSource) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - private implicit val requestExecutorContext = - ExecutionContext.fromExecutorService(requestExecutorsService) - protected override val minRegisteredRatio = if (conf.get(SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO).isEmpty) { 0.8 @@ -60,7 +57,9 @@ private[spark] class KubernetesClusterSchedulerBackend( // Allow removeExecutor to be accessible by ExecutorPodsLifecycleEventHandler private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { - removeExecutor(executorId, reason) + if (isExecutorActive(executorId)) { + removeExecutor(executorId, reason) + } } /** @@ -76,9 +75,7 @@ private[spark] class KubernetesClusterSchedulerBackend( override def start(): Unit = { super.start() - if (!Utils.isDynamicAllocationEnabled(conf)) { - podAllocator.setTotalExpectedExecutors(initialExecutors) - } + podAllocator.setTotalExpectedExecutors(initialExecutors) lifecycleEventHandler.start(this) podAllocator.start(applicationId()) watchEvents.start(applicationId()) @@ -111,7 +108,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } Utils.tryLogNonFatalError { - ThreadUtils.shutdown(requestExecutorsService) + ThreadUtils.shutdown(executorService) } Utils.tryLogNonFatalError { @@ -119,11 +116,9 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { - // TODO when we support dynamic allocation, the pod allocator should be told to process the - // current snapshot in order to decrease/increase the number of executors accordingly. + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { podAllocator.setTotalExpectedExecutors(requestedTotal) - true + Future.successful(true) } override def sufficientResourcesRegistered(): Boolean = { @@ -134,14 +129,48 @@ private[spark] class KubernetesClusterSchedulerBackend( super.getExecutorIds() } - override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - kubernetesClient - .pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId()) - .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) - .delete() - // Don't do anything else - let event handling from the Kubernetes API do the Spark changes + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + executorIds.foreach { id => + removeExecutor(id, ExecutorKilled) + } + + // Give some time for the executors to shut themselves down, then forcefully kill any + // remaining ones. This intentionally ignores the configuration about whether pods + // should be deleted; only executors that shut down gracefully (and are then collected + // by the ExecutorPodsLifecycleManager) will respect that configuration. + val killTask = new Runnable() { + override def run(): Unit = Utils.tryLogNonFatalError { + val running = kubernetesClient + .pods() + .withField("status.phase", "Running") + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) + + if (!running.list().getItems().isEmpty()) { + logInfo(s"Forcefully deleting ${running.list().getItems().size()} pods " + + s"(out of ${executorIds.size}) that are still running after graceful shutdown period.") + running.delete() + } + } + } + executorService.schedule(killTask, conf.get(KUBERNETES_DYN_ALLOC_KILL_GRACE_PERIOD), + TimeUnit.MILLISECONDS) + + // Return an immediate success, since we can't confirm or deny that executors have been + // actually shut down without waiting too long and blocking the allocation thread, which + // waits on this future to complete, blocking further allocations / deallocations. + // + // This relies a lot on the guarantees of Spark's RPC system, that a message will be + // delivered to the destination unless there's an issue with the connection, in which + // case the executor will shut itself down (and the driver, separately, will just declare + // it as "lost"). Coupled with the allocation manager keeping track of which executors are + // pending release, returning "true" here means that eventually all the requested executors + // will be removed. + // + // The cleanup timer above is just an optimization to make sure that stuck executors don't + // stick around in the k8s server. Normally it should never delete any pods at all. + Future.successful(true) } override def createDriverEndpoint(): DriverEndpoint = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index f7721e6fd6388..1b6dfe5443377 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -34,7 +34,7 @@ class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore override def stop(): Unit = {} - def notifySubscribers(): Unit = { + override def notifySubscribers(): Unit = { subscribers.foreach(_(snapshotsBuffer)) snapshotsBuffer.clear() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 5862f647ccb90..4475d5db6f03a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -92,8 +92,6 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - snapshotsStore.replaceSnapshot(Seq.empty[Pod]) - snapshotsStore.notifySubscribers() for (nextId <- 1 to podAllocationSize) { verify(podOperations).create(podWithAttachedContainerForId(nextId)) } @@ -103,8 +101,6 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize + 1) - snapshotsStore.replaceSnapshot(Seq.empty[Pod]) - snapshotsStore.notifySubscribers() for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } @@ -121,8 +117,6 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("When a current batch reaches error states immediately, re-request" + " them on the next batch.") { podsAllocatorUnderTest.setTotalExpectedExecutors(podAllocationSize) - snapshotsStore.replaceSnapshot(Seq.empty[Pod]) - snapshotsStore.notifySubscribers() for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } @@ -134,25 +128,69 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("When an executor is requested but the API does not report it in a reasonable time, retry" + " requesting that executor.") { - podsAllocatorUnderTest.setTotalExpectedExecutors(1) - snapshotsStore.replaceSnapshot(Seq.empty[Pod]) - snapshotsStore.notifySubscribers() - snapshotsStore.replaceSnapshot(Seq.empty[Pod]) - waitForExecutorPodsClock.setTime(podCreationTimeout + 1) when(podOperations .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) .thenReturn(podOperations) when(podOperations - withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) .thenReturn(podOperations) when(podOperations - .withLabel(SPARK_EXECUTOR_ID_LABEL, "1")) + .withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1")) .thenReturn(labeledPods) + podsAllocatorUnderTest.setTotalExpectedExecutors(1) + verify(podOperations).create(podWithAttachedContainerForId(1)) + waitForExecutorPodsClock.setTime(podCreationTimeout + 1) snapshotsStore.notifySubscribers() verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) } + test("SPARK-28487: scale up and down on target executor count changes") { + when(podOperations + .withField("status.phase", "Pending")) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any())) + .thenReturn(podOperations) + + // Target 1 executor, make sure it's requested, even with an empty initial snapshot. + podsAllocatorUnderTest.setTotalExpectedExecutors(1) + verify(podOperations).create(podWithAttachedContainerForId(1)) + + // Mark executor as running, verify that subsequent allocation cycle is a no-op. + snapshotsStore.updatePod(runningExecutor(1)) + snapshotsStore.notifySubscribers() + verify(podOperations, times(1)).create(any()) + verify(podOperations, never()).delete() + + // Request 3 more executors, make sure all are requested. + podsAllocatorUnderTest.setTotalExpectedExecutors(4) + snapshotsStore.notifySubscribers() + verify(podOperations).create(podWithAttachedContainerForId(2)) + verify(podOperations).create(podWithAttachedContainerForId(3)) + verify(podOperations).create(podWithAttachedContainerForId(4)) + + // Mark 2 as running, 3 as pending. Allocation cycle should do nothing. + snapshotsStore.updatePod(runningExecutor(2)) + snapshotsStore.updatePod(pendingExecutor(3)) + snapshotsStore.notifySubscribers() + verify(podOperations, times(4)).create(any()) + verify(podOperations, never()).delete() + + // Scale down to 1. Pending executors (both acknowledged and not) should be deleted. + podsAllocatorUnderTest.setTotalExpectedExecutors(1) + snapshotsStore.notifySubscribers() + verify(podOperations, times(4)).create(any()) + verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "3", "4") + verify(podOperations).delete() + } + private def executorPodAnswer(): Answer[SparkPod] = (invocation: InvocationOnMock) => { val k8sConf: KubernetesExecutorConf = invocation.getArgument(0) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 8ed934d91dd7e..7e1e39c85a183 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -16,14 +16,19 @@ */ package org.apache.spark.scheduler.cluster.k8s +import java.util.Arrays +import java.util.concurrent.TimeUnit + +import io.fabric8.kubernetes.api.model.{Pod, PodList} import io.fabric8.kubernetes.client.KubernetesClient import org.jmock.lib.concurrent.DeterministicScheduler import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.ArgumentMatchers.{eq => mockitoEq} -import org.mockito.Mockito.{never, verify, when} +import org.mockito.ArgumentMatchers.{any, eq => mockitoEq} +import org.mockito.Mockito.{mock, never, spy, verify, when} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} @@ -34,7 +39,7 @@ import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils.TEST_SP class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAndAfter { - private val requestExecutorsService = new DeterministicScheduler() + private val schedulerExecutorService = new DeterministicScheduler() private val sparkConf = new SparkConf(false) .set("spark.executor.instances", "3") .set("spark.app.id", TEST_SPARK_APP_ID) @@ -98,7 +103,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn taskScheduler, sc, kubernetesClient, - requestExecutorsService, + schedulerExecutorService, eventQueue, podAllocator, lifecycleEventHandler, @@ -127,29 +132,42 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn } test("Remove executor") { - schedulerBackendUnderTest.start() - schedulerBackendUnderTest.doRemoveExecutor("1", ExecutorKilled) - verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) + val backend = spy(schedulerBackendUnderTest) + when(backend.isExecutorActive(any())).thenReturn(false) + when(backend.isExecutorActive(mockitoEq("2"))).thenReturn(true) + + backend.start() + backend.doRemoveExecutor("1", ExecutorKilled) + verify(driverEndpointRef, never()).send(RemoveExecutor("1", ExecutorKilled)) + + backend.doRemoveExecutor("2", ExecutorKilled) + verify(driverEndpointRef, never()).send(RemoveExecutor("1", ExecutorKilled)) } test("Kill executors") { schedulerBackendUnderTest.start() - when(podOperations.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)).thenReturn(labeledPods) + when(podOperations.withField(any(), any())).thenReturn(labeledPods) + when(labeledPods.withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)).thenReturn(labeledPods) when(labeledPods.withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)).thenReturn(labeledPods) when(labeledPods.withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2")).thenReturn(labeledPods) + + val podList = mock(classOf[PodList]) + when(labeledPods.list()).thenReturn(podList) + when(podList.getItems()).thenReturn(Arrays.asList[Pod]()) + + schedulerBackendUnderTest.doKillExecutors(Seq("1", "2")) + verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) + verify(driverEndpointRef).send(RemoveExecutor("2", ExecutorKilled)) + verify(labeledPods, never()).delete() + schedulerExecutorService.tick(sparkConf.get(KUBERNETES_DYN_ALLOC_KILL_GRACE_PERIOD) * 2, + TimeUnit.MILLISECONDS) + verify(labeledPods, never()).delete() + + when(podList.getItems()).thenReturn(Arrays.asList(mock(classOf[Pod]))) schedulerBackendUnderTest.doKillExecutors(Seq("1", "2")) verify(labeledPods, never()).delete() - requestExecutorsService.runNextPendingCommand() + schedulerExecutorService.tick(sparkConf.get(KUBERNETES_DYN_ALLOC_KILL_GRACE_PERIOD) * 2, + TimeUnit.MILLISECONDS) verify(labeledPods).delete() } - - test("Request total executors") { - schedulerBackendUnderTest.start() - schedulerBackendUnderTest.doRequestTotalExecutors(5) - verify(podAllocator).setTotalExpectedExecutors(3) - verify(podAllocator, never()).setTotalExpectedExecutors(5) - requestExecutorsService.runNextPendingCommand() - verify(podAllocator).setTotalExpectedExecutors(5) - } - } From 3302042ec493947a41a6c688ef2a8cc7762118e3 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 14 Aug 2019 03:14:07 +0000 Subject: [PATCH 063/149] [SPARK-28719][BUILD] [FOLLOWUP] Add JDK11 for Github Actions ## What changes were proposed in this pull request? Add JDK11 for Github Actions Closes #25444 from dbtsai/jdk11. Authored-by: DB Tsai Signed-off-by: DB Tsai --- .github/workflows/master.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d29b2c3a00f28..fed6b826fd25f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -9,13 +9,17 @@ jobs: build: runs-on: ubuntu-latest + strategy: + matrix: + java: [ '1.8', '11' ] + name: Build Spark with JDK ${{ matrix.java }} steps: - uses: actions/checkout@master - - name: Set up JDK 1.8 + - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: - version: 1.8 + version: ${{ matrix.java }} - name: Build with Maven run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" From 598fcbe5ed353e3e432d5d32f656527806c7c612 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Wed, 14 Aug 2019 14:24:13 +0800 Subject: [PATCH 064/149] [SPARK-28265][SQL] Add renameTable to TableCatalog API ## What changes were proposed in this pull request? This PR adds the `renameTable` call to the `TableCatalog` API, as described in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d). This PR is related to: https://github.com/apache/spark/pull/24246 ## How was this patch tested? Added unit tests and contract tests. Closes #25206 from edgarRd/SPARK-28265-add-rename-table-catalog-api. Authored-by: Edgar Rodriguez Signed-off-by: Wenchen Fan --- .../spark/sql/catalog/v2/TableCatalog.java | 20 +++++ .../sql/catalog/v2/TableCatalogSuite.scala | 47 ++++++++++++ .../sql/catalog/v2/TestTableCatalog.scala | 13 ++++ .../datasources/v2/V2SessionCatalog.scala | 10 +++ .../v2/V2SessionCatalogSuite.scala | 74 ++++++++++++++++++- .../sources/v2/TestInMemoryTableCatalog.scala | 14 ++++ 6 files changed, 177 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java index 681629d2d5405..4775b58edf049 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java @@ -134,4 +134,24 @@ Table alterTable( * @return true if a table was deleted, false if no table exists for the identifier */ boolean dropTable(Identifier ident); + + /** + * Renames a table in the catalog. + *

+ * If the catalog supports views and contains a view for the old identifier and not a table, this + * throws {@link NoSuchTableException}. Additionally, if the new identifier is a table or a view, + * this throws {@link TableAlreadyExistsException}. + *

+ * If the catalog does not support table renames between namespaces, it throws + * {@link UnsupportedOperationException}. + * + * @param oldIdent the table identifier of the existing table to rename + * @param newIdent the new table identifier of the table + * @throws NoSuchTableException If the table to rename doesn't exist or is a view + * @throws TableAlreadyExistsException If the new table name already exists or is a view + * @throws UnsupportedOperationException If the namespaces of old and new identiers do not + * match (optional) + */ + void renameTable(Identifier oldIdent, Identifier newIdent) + throws NoSuchTableException, TableAlreadyExistsException; } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala index 089b4c5ed94f9..e4c1b3c297165 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala @@ -45,6 +45,7 @@ class TableCatalogSuite extends SparkFunSuite { private val testNs = Array("`", ".") private val testIdent = Identifier.of(testNs, "test_table") + private val testIdentNew = Identifier.of(testNs, "test_table_new") test("Catalogs can load the catalog") { val catalog = newCatalog() @@ -656,6 +657,52 @@ class TableCatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) } + test("renameTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + assert(!catalog.tableExists(testIdentNew)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + catalog.renameTable(testIdent, testIdentNew) + + assert(!catalog.tableExists(testIdent)) + assert(catalog.tableExists(testIdentNew)) + } + + test("renameTable: fail if table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.renameTable(testIdent, testIdentNew) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("renameTable: fail if new table name already exists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + assert(!catalog.tableExists(testIdentNew)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdentNew, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + assert(catalog.tableExists(testIdentNew)) + + val exc = intercept[TableAlreadyExistsException] { + catalog.renameTable(testIdent, testIdentNew) + } + + assert(exc.message.contains(testIdentNew.quoted)) + assert(exc.message.contains("already exists")) + } + test("listNamespaces: list namespaces from metadata") { val catalog = newCatalog() catalog.createNamespace(Array("ns1"), Map("property" -> "value").asJava) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala index 6fdd6e30e1ee4..de7c5c9536fdf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala @@ -93,6 +93,19 @@ class TestTableCatalog extends TableCatalog with SupportsNamespaces { override def dropTable(ident: Identifier): Boolean = Option(tables.remove(ident)).isDefined + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + if (tables.containsKey(newIdent)) { + throw new TableAlreadyExistsException(newIdent) + } + + Option(tables.remove(oldIdent)) match { + case Some(table) => + tables.put(newIdent, InMemoryTable(table.name, table.schema, table.properties)) + case _ => + throw new NoSuchTableException(oldIdent) + } + } + private def allNamespaces: Seq[Seq[String]] = { (tables.keySet.asScala.map(_.namespace.toSeq) ++ namespaces.keySet.asScala).toSeq.distinct } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index a3b8f28fc5c39..79ea8756721ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -153,6 +153,16 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { } } + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + if (tableExists(newIdent)) { + throw new TableAlreadyExistsException(newIdent) + } + + // Load table to make sure the table exists + loadTable(oldIdent) + catalog.renameTable(oldIdent.asTableIdentifier, newIdent.asTableIdentifier) + } + implicit class TableIdentifierHelper(ident: Identifier) { def asTableIdentifier: TableIdentifier = { ident.namespace match { 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 3822882cc91cb..4f14ecc28680d 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 @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, TableCatalog, TableChange} import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -45,6 +46,7 @@ class V2SessionCatalogSuite override protected def beforeAll(): Unit = { super.beforeAll() spark.sql("""CREATE DATABASE IF NOT EXISTS db""") + spark.sql("""CREATE DATABASE IF NOT EXISTS db2""") spark.sql("""CREATE DATABASE IF NOT EXISTS ns""") spark.sql("""CREATE DATABASE IF NOT EXISTS ns2""") } @@ -52,6 +54,7 @@ class V2SessionCatalogSuite override protected def afterAll(): Unit = { spark.sql("""DROP TABLE IF EXISTS db.test_table""") spark.sql("""DROP DATABASE IF EXISTS db""") + spark.sql("""DROP DATABASE IF EXISTS db2""") spark.sql("""DROP DATABASE IF EXISTS ns""") spark.sql("""DROP DATABASE IF EXISTS ns2""") super.afterAll() @@ -59,6 +62,7 @@ class V2SessionCatalogSuite after { newCatalog().dropTable(testIdent) + newCatalog().dropTable(testIdentNew) } private def newCatalog(): TableCatalog = { @@ -67,7 +71,9 @@ class V2SessionCatalogSuite newCatalog } - private val testIdent = Identifier.of(Array("db"), "test_table") + private val testNs = Array("db") + private val testIdent = Identifier.of(testNs, "test_table") + private val testIdentNew = Identifier.of(testNs, "test_table_new") test("Catalogs can load the catalog") { val catalog = newCatalog() @@ -680,4 +686,70 @@ class V2SessionCatalogSuite assert(!wasDropped) assert(!catalog.tableExists(testIdent)) } + + test("renameTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + assert(!catalog.tableExists(testIdentNew)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + catalog.renameTable(testIdent, testIdentNew) + + assert(!catalog.tableExists(testIdent)) + assert(catalog.tableExists(testIdentNew)) + } + + test("renameTable: fail if table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.renameTable(testIdent, testIdentNew) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("renameTable: fail if new table name already exists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + assert(!catalog.tableExists(testIdentNew)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.createTable(testIdentNew, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + assert(catalog.tableExists(testIdentNew)) + + val exc = intercept[TableAlreadyExistsException] { + catalog.renameTable(testIdent, testIdentNew) + } + + assert(exc.message.contains(testIdentNew.quoted)) + assert(exc.message.contains("already exists")) + } + + test("renameTable: fail if db does not match for old and new table names") { + val catalog = newCatalog() + val testIdentNewOtherDb = Identifier.of(Array("db2"), "test_table_new") + + assert(!catalog.tableExists(testIdent)) + assert(!catalog.tableExists(testIdentNewOtherDb)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + val exc = intercept[AnalysisException] { + catalog.renameTable(testIdent, testIdentNewOtherDb) + } + + assert(exc.message.contains(testIdent.namespace.quoted)) + assert(exc.message.contains(testIdentNewOtherDb.namespace.quoted)) + assert(exc.message.contains("RENAME TABLE source and destination databases do not match")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 7c51a29bde905..b7151766cf987 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -104,6 +104,20 @@ class TestInMemoryTableCatalog extends TableCatalog { Option(tables.remove(ident)).isDefined } + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { + if (tables.containsKey(newIdent)) { + throw new TableAlreadyExistsException(newIdent) + } + + Option(tables.remove(oldIdent)) match { + case Some(table) => + tables.put(newIdent, + new InMemoryTable(table.name, table.schema, table.partitioning, table.properties)) + case _ => + throw new NoSuchTableException(oldIdent) + } + } + def clearTables(): Unit = { tables.clear() } From 391c7e8f2e5d1079dc3eb5a38b3a0d58eb686cba Mon Sep 17 00:00:00 2001 From: John Zhuge Date: Wed, 14 Aug 2019 19:49:53 +0800 Subject: [PATCH 065/149] [SPARK-27739][SQL] df.persist should save stats from optimized plan ## What changes were proposed in this pull request? CacheManager.cacheQuery saves the stats from the optimized plan to cache. ## How was this patch tested? Existing testss. Closes #24623 from jzhuge/SPARK-27739. Authored-by: John Zhuge Signed-off-by: Wenchen Fan --- .../spark/sql/execution/CacheManager.scala | 11 +++++----- .../execution/columnar/InMemoryRelation.scala | 12 +++++------ .../apache/spark/sql/DatasetCacheSuite.scala | 20 +++++++++++++++++++ 3 files changed, 32 insertions(+), 11 deletions(-) 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 a13e6ada83dd0..10dc74dd8a8ff 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 @@ -79,12 +79,13 @@ class CacheManager extends Logging { logWarning("Asked to cache already cached data.") } else { val sparkSession = query.sparkSession + val qe = sparkSession.sessionState.executePlan(planToCache) val inMemoryRelation = InMemoryRelation( sparkSession.sessionState.conf.useCompression, sparkSession.sessionState.conf.columnBatchSize, storageLevel, - sparkSession.sessionState.executePlan(planToCache).executedPlan, + qe.executedPlan, tableName, - planToCache) + optimizedPlan = qe.optimizedPlan) this.synchronized { if (lookupCachedData(planToCache).nonEmpty) { logWarning("Data has already been cached.") @@ -184,10 +185,10 @@ class CacheManager extends Logging { } needToRecache.map { cd => cd.cachedRepresentation.cacheBuilder.clearCache() - val plan = spark.sessionState.executePlan(cd.plan).executedPlan + val qe = spark.sessionState.executePlan(cd.plan) val newCache = InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = plan), - logicalPlan = cd.plan) + cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = qe.executedPlan), + optimizedPlan = qe.optimizedPlan) val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { if (lookupCachedData(recomputedPlan.plan).nonEmpty) { 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 b77f90d19b62d..de1c14cc688a6 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 @@ -146,17 +146,17 @@ object InMemoryRelation { storageLevel: StorageLevel, child: SparkPlan, tableName: Option[String], - logicalPlan: LogicalPlan): InMemoryRelation = { + optimizedPlan: LogicalPlan): InMemoryRelation = { val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) - val relation = new InMemoryRelation(child.output, cacheBuilder, logicalPlan.outputOrdering) - relation.statsOfPlanToCache = logicalPlan.stats + val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats relation } - def apply(cacheBuilder: CachedRDDBuilder, logicalPlan: LogicalPlan): InMemoryRelation = { + def apply(cacheBuilder: CachedRDDBuilder, optimizedPlan: LogicalPlan): InMemoryRelation = { val relation = new InMemoryRelation( - cacheBuilder.cachedPlan.output, cacheBuilder, logicalPlan.outputOrdering) - relation.statsOfPlanToCache = logicalPlan.stats + cacheBuilder.cachedPlan.output, cacheBuilder, optimizedPlan.outputOrdering) + relation.statsOfPlanToCache = optimizedPlan.stats relation } 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 b828b23025151..d888585e120a4 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 @@ -246,4 +246,24 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits assert(df2LimitInnerPlan.isDefined && df2LimitInnerPlan.get.find(_.isInstanceOf[InMemoryTableScanExec]).isEmpty) } + + test("SPARK-27739 Save stats from optimized plan") { + withTable("a") { + spark.range(4) + .selectExpr("id", "id % 2 AS p") + .write + .partitionBy("p") + .saveAsTable("a") + + val df = sql("SELECT * FROM a WHERE p = 0") + df.cache() + df.count() + df.queryExecution.withCachedData match { + case i: InMemoryRelation => + // Optimized plan has non-default size in bytes + assert(i.statsOfPlanToCache.sizeInBytes !== + df.sparkSession.sessionState.conf.defaultSizeInBytes) + } + } + } } From 2eeb25e52d928459f7d7db5bb779f18bda455ad3 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Wed, 14 Aug 2019 23:38:45 +0800 Subject: [PATCH 066/149] [SPARK-28351][SQL] Support DELETE in DataSource V2 ## What changes were proposed in this pull request? This pr adds DELETE support for V2 datasources. As a first step, this pr only support delete by source filters: ```scala void delete(Filter[] filters); ``` which could not deal with complicated cases like subqueries. Since it's uncomfortable to embed the implementation of DELETE in the current V2 APIs, a new mix-in of datasource is added, which is called `SupportsMaintenance`, similar to `SupportsRead` and `SupportsWrite`. A datasource which can be maintained means we can perform DELETE/UPDATE/MERGE/OPTIMIZE on the datasource, as long as the datasource implements the necessary mix-ins. ## How was this patch tested? new test case. Please review https://spark.apache.org/contributing.html before opening a pull request. Closes #25115 from xianyinxin/SPARK-28351. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../spark/sql/sources/v2/SupportsDelete.java | 43 ++++++++++++++ .../sql/catalyst/analysis/Analyzer.scala | 2 + .../sql/catalyst/analysis/CheckAnalysis.scala | 7 ++- .../sql/catalyst/parser/AstBuilder.scala | 16 ++++- .../plans/logical/basicLogicalOperators.scala | 7 +++ .../logical/sql/DeleteFromStatement.scala | 27 +++++++++ .../v2/DataSourceV2Implicits.scala | 11 +++- .../analysis/AnalysisErrorSuite.scala | 3 +- .../datasources/DataSourceResolution.scala | 17 ++++-- .../datasources/v2/DataSourceV2Strategy.scala | 11 +++- .../datasources/v2/DeleteFromTableExec.scala | 38 ++++++++++++ .../datasources/v2/V2WriteSupportCheck.scala | 9 ++- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 36 +++++++++++ .../sources/v2/TestInMemoryTableCatalog.scala | 59 ++++++++++++------- 15 files changed, 252 insertions(+), 35 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index aa44e677e577b..5c23373af0a0d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -214,6 +214,7 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration + | DELETE FROM multipartIdentifier tableAlias whereClause #deleteFromTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java new file mode 100644 index 0000000000000..8650a0ef1d4ba --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java @@ -0,0 +1,43 @@ +/* + * 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.sources.v2; + +import org.apache.spark.sql.sources.Filter; + +/** + * A mix-in interface for {@link Table} delete support. Data sources can implement this + * interface to provide the ability to delete data from tables that matches filter expressions. + */ +public interface SupportsDelete { + /** + * Delete data from a data source table that matches filter expressions. + *

+ * Rows are deleted from the data source iff all of the filter expressions match. That is, the + * expressions must be interpreted as a set of filters that are ANDed together. + *

+ * Implementations may reject a delete operation if the delete isn't possible without significant + * effort. For example, partitioned data sources may reject deletes that do not filter by + * partition columns because the filter may require rewriting files without deleted records. + * To reject a delete implementations should throw {@link IllegalArgumentException} with a clear + * error message that identifies which expression was rejected. + * + * @param filters filter expressions, used to select rows to delete when all expressions match + * @throws IllegalArgumentException If the delete is rejected due to required effort + */ + void deleteWhere(Filter[] filters); +} 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 5a04d5794a659..7267ad8d37c82 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 @@ -1761,6 +1761,8 @@ class Analyzer( // Only a few unary nodes (Project/Filter/Aggregate) can contain subqueries. case q: UnaryNode if q.childrenResolved => resolveSubQueries(q, q.children) + case d: DeleteFromTable if d.childrenResolved => + resolveSubQueries(d, d.children) } } 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 519c558d12770..bd54c66992dac 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 @@ -585,7 +585,7 @@ trait CheckAnalysis extends PredicateHelper { // Only certain operators are allowed to host subquery expression containing // outer references. plan match { - case _: Filter | _: Aggregate | _: Project => // Ok + case _: Filter | _: Aggregate | _: Project | _: DeleteFromTable => // Ok case other => failAnalysis( "Correlated scalar sub-queries can only be used in a " + s"Filter/Aggregate/Project: $plan") @@ -594,9 +594,10 @@ trait CheckAnalysis extends PredicateHelper { case inSubqueryOrExistsSubquery => plan match { - case _: Filter => // Ok + case _: Filter | _: DeleteFromTable => // Ok case _ => - failAnalysis(s"IN/EXISTS predicate sub-queries can only be used in a Filter: $plan") + failAnalysis(s"IN/EXISTS predicate sub-queries can only be used in" + + s" Filter/DeleteFromTable: $plan") } } 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 6decfdb2c2244..3d1ed69391c98 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -338,6 +338,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) } + override def visitDeleteFromTable( + ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { + + val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) + val tableAlias = if (ctx.tableAlias() != null) { + val ident = ctx.tableAlias().strictIdentifier() + if (ident != null) { Some(ident.getText) } else { None } + } else { + None + } + + DeleteFromStatement(tableId, tableAlias, expression(ctx.whereClause().booleanExpression())) + } + /** * Create a partition specification map. */ 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 d9c370af47fb8..968a561da9c38 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 @@ -567,6 +567,13 @@ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Comm override val output = DescribeTableSchema.describeTableAttributes() } +case class DeleteFromTable( + child: LogicalPlan, + condition: Expression) extends Command { + + override def children: Seq[LogicalPlan] = child :: Nil +} + /** * Drop a table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala new file mode 100644 index 0000000000000..21e24127eee31 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala @@ -0,0 +1,27 @@ +/* + * 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.sql + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +case class DeleteFromStatement( + tableName: Seq[String], + tableAlias: Option[String], + condition: Expression) + extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index eed69cdc8cac6..2d59c42ee8684 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} object DataSourceV2Implicits { implicit class TableHelper(table: Table) { @@ -40,6 +40,15 @@ object DataSourceV2Implicits { } } + def asDeletable: SupportsDelete = { + table match { + case support: SupportsDelete => + support + case _ => + throw new AnalysisException(s"Table does not support deletes: ${table.name}") + } + } + def supports(capability: TableCapability): Boolean = table.capabilities.contains(capability) def supportsAny(capabilities: TableCapability*): Boolean = capabilities.exists(supports) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 4d42f5fb73362..1cc5dd8ce1d54 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -531,7 +531,8 @@ class AnalysisErrorSuite extends AnalysisTest { val plan = Project( Seq(a, Alias(InSubquery(Seq(a), ListQuery(LocalRelation(b))), "c")()), LocalRelation(a)) - assertAnalysisError(plan, "Predicate sub-queries can only be used in a Filter" :: Nil) + assertAnalysisError(plan, "Predicate sub-queries can only be used" + + " in Filter/DeleteFromTable" :: Nil) } test("PredicateSubQuery is used is a nested condition") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index a150a049f33e1..4791fe5fb5251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedAttribute, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} @@ -173,9 +173,18 @@ case class DataSourceResolution( // only top-level adds are supported using AlterTableAddColumnsCommand AlterTableAddColumnsCommand(table, newColumns.map(convertToStructField)) + case DeleteFromStatement(AsTableIdentifier(table), tableAlias, condition) => + throw new AnalysisException( + s"Delete from tables is not supported using the legacy / v1 Spark external catalog" + + s" API. Identifier: $table.") + + case delete: DeleteFromStatement => + val relation = UnresolvedRelation(delete.tableName) + val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation) + DeleteFromTable(aliased, delete.condition) + case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) => UnresolvedCatalogRelation(catalogTable) - } object V1WriteProvider { 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 91fc2e068af70..585fe06ce4ce7 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -222,6 +222,15 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _) => OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil + case DeleteFromTable(r: DataSourceV2Relation, condition) => + // fail if any filter cannot be converted. correctness depends on removing all matching data. + val filters = splitConjunctivePredicates(condition).map { + f => DataSourceStrategy.translateFilter(f).getOrElse( + throw new AnalysisException(s"Exec delete failed:" + + s" cannot translate expression to source filter: $f")) + }.toArray + DeleteFromTableExec(r.table.asDeletable, filters) :: Nil + case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala new file mode 100644 index 0000000000000..a5840571fff23 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.SupportsDelete +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class DeleteFromTableExec( + table: SupportsDelete, + condition: Array[Filter]) extends LeafExecNode { + + override protected def doExecute(): RDD[InternalRow] = { + table.deleteWhere(condition) + sparkContext.emptyRDD + } + + override def output: Seq[Attribute] = Nil +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala index cf77998c122f8..5648d5439ba5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.catalyst.expressions.{Literal, SubqueryExpression} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, DeleteFromTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.BooleanType @@ -51,6 +51,11 @@ object V2WriteSupportCheck extends (LogicalPlan => Unit) { } } + case DeleteFromTable(_, condition) => + if (SubqueryExpression.hasSubquery(condition)) { + failAnalysis(s"Delete by condition with subquery is not supported: $condition") + } + case _ => // OK } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 9ae51d577b562..9b1a23a1f2bbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -1862,6 +1862,42 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } } + test("DeleteFrom: basic") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t WHERE id = 2") + checkAnswer(spark.table(t), Seq( + Row(3, "c", 3))) + } + } + + test("DeleteFrom: alias") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t tbl WHERE tbl.id = 2") + checkAnswer(spark.table(t), Seq( + Row(3, "c", 3))) + } + } + + test("DeleteFrom: fail if has subquery") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + val exc = intercept[AnalysisException] { + sql(s"DELETE FROM $t WHERE id IN (SELECT id FROM $t)") + } + + assert(spark.table(t).count === 3) + assert(exc.getMessage.contains("Delete by condition with subquery is not supported")) + } + } + private def testCreateAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index b7151766cf987..93889ba1da156 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalog.v2.expressions.{IdentityTransform, Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.sources.{And, EqualTo, Filter} +import org.apache.spark.sql.sources.{And, EqualTo, Filter, IsNotNull} import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.types.StructType @@ -131,7 +131,7 @@ class InMemoryTable( val schema: StructType, override val partitioning: Array[Transform], override val properties: util.Map[String, String]) - extends Table with SupportsRead with SupportsWrite { + extends Table with SupportsRead with SupportsWrite with SupportsDelete { partitioning.foreach { t => if (!t.isInstanceOf[IdentityTransform]) { @@ -234,35 +234,50 @@ class InMemoryTable( private class Overwrite(filters: Array[Filter]) extends TestBatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { - val deleteKeys = dataMap.keys.filter { partValues => + dataMap --= deletesKeys(filters) + withData(messages.map(_.asInstanceOf[BufferedRows])) + } + } + + private object TruncateAndAppend extends TestBatchWrite { + override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { + dataMap.clear + withData(messages.map(_.asInstanceOf[BufferedRows])) + } + } + + override def deleteWhere(filters: Array[Filter]): Unit = dataMap.synchronized { + dataMap --= deletesKeys(filters) + } + + private def splitAnd(filter: Filter): Seq[Filter] = { + filter match { + case And(left, right) => splitAnd(left) ++ splitAnd(right) + case _ => filter :: Nil + } + } + + private def deletesKeys(filters: Array[Filter]): Iterable[Seq[Any]] = { + dataMap.synchronized { + dataMap.keys.filter { partValues => filters.flatMap(splitAnd).forall { case EqualTo(attr, value) => - partFieldNames.zipWithIndex.find(_._1 == attr) match { - case Some((_, partIndex)) => - value == partValues(partIndex) - case _ => - throw new IllegalArgumentException(s"Unknown filter attribute: $attr") - } + value == extractValue(attr, partValues) + case IsNotNull(attr) => + null != extractValue(attr, partValues) case f => throw new IllegalArgumentException(s"Unsupported filter type: $f") } } - dataMap --= deleteKeys - withData(messages.map(_.asInstanceOf[BufferedRows])) - } - - private def splitAnd(filter: Filter): Seq[Filter] = { - filter match { - case And(left, right) => splitAnd(left) ++ splitAnd(right) - case _ => filter :: Nil - } } } - private object TruncateAndAppend extends TestBatchWrite { - override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { - dataMap.clear - withData(messages.map(_.asInstanceOf[BufferedRows])) + private def extractValue(attr: String, partValues: Seq[Any]): Any = { + partFieldNames.zipWithIndex.find(_._1 == attr) match { + case Some((_, partIndex)) => + partValues(partIndex) + case _ => + throw new IllegalArgumentException(s"Unknown filter attribute: $attr") } } } From 3a4afce96c6840431ed45280742f9e969be19639 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 14 Aug 2019 08:44:44 -0700 Subject: [PATCH 067/149] [SPARK-28687][SQL] Support `epoch`, `isoyear`, `milliseconds` and `microseconds` at `extract()` ## What changes were proposed in this pull request? In the PR, I propose new expressions `Epoch`, `IsoYear`, `Milliseconds` and `Microseconds`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT): 1. `epoch` - the number of seconds since 1970-01-01 00:00:00 local time in microsecond precision. 2. `isoyear` - the ISO 8601 week-numbering year that the date falls in. Each ISO 8601 week-numbering year begins with the Monday of the week containing the 4th of January. 3. `milliseconds` - the seconds field including fractional parts multiplied by 1,000. 4. `microseconds` - the seconds field including fractional parts multiplied by 1,000,000. Here are examples: ```sql spark-sql> SELECT EXTRACT(EPOCH FROM TIMESTAMP '2019-08-11 19:07:30.123456'); 1565550450.123456 spark-sql> SELECT EXTRACT(ISOYEAR FROM DATE '2006-01-01'); 2005 spark-sql> SELECT EXTRACT(MILLISECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456'); 30123.456 spark-sql> SELECT EXTRACT(MICROSECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456'); 30123456 ``` ## How was this patch tested? Added new tests to `DateExpressionsSuite`, and uncommented existing tests in `extract.sql` and `pgSQL/date.sql`. Closes #25408 from MaxGekk/extract-ext3. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../expressions/datetimeExpressions.scala | 81 ++++++ .../sql/catalyst/parser/AstBuilder.scala | 8 + .../sql/catalyst/util/DateTimeUtils.scala | 36 +++ .../expressions/DateExpressionsSuite.scala | 45 ++- .../resources/sql-tests/inputs/extract.sql | 18 ++ .../resources/sql-tests/inputs/pgSQL/date.sql | 4 +- .../sql-tests/results/extract.sql.out | 220 +++++++++++---- .../sql-tests/results/pgSQL/date.sql.out | 266 ++++++++++-------- 8 files changed, 496 insertions(+), 182 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index a6d6545b81186..1ce493ece18b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -302,6 +302,48 @@ case class Second(child: Expression, timeZoneId: Option[String] = None) } } +case class Milliseconds(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression { + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + // DecimalType is used here to not lose precision while converting microseconds to + // the fractional part of milliseconds. Scale 3 is taken to have all microseconds as + // the fraction. The precision 8 should cover 2 digits for seconds, 3 digits for + // milliseconds and 3 digits for microseconds. + override def dataType: DataType = DecimalType(8, 3) + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected def nullSafeEval(timestamp: Any): Any = { + DateTimeUtils.getMilliseconds(timestamp.asInstanceOf[Long], timeZone) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceObj("timeZone", timeZone) + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getMilliseconds($c, $tz)") + } +} + +case class Microseconds(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression { + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def dataType: DataType = IntegerType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected def nullSafeEval(timestamp: Any): Any = { + DateTimeUtils.getMicroseconds(timestamp.asInstanceOf[Long], timeZone) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceObj("timeZone", timeZone) + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getMicroseconds($c, $tz)") + } +} + @ExpressionDescription( usage = "_FUNC_(date) - Returns the day of year of the date/timestamp.", examples = """ @@ -350,6 +392,22 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu } } +case class IsoYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getIsoYear(date.asInstanceOf[Int]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$dtu.getIsoYear($c)") + } +} + @ExpressionDescription( usage = "_FUNC_(date) - Returns the quarter of the year for date, in the range 1 to 4.", examples = """ @@ -1882,3 +1940,26 @@ case class Decade(child: Expression) extends UnaryExpression with ImplicitCastIn defineCodeGen(ctx, ev, c => s"$dtu.getDecade($c)") } } + +case class Epoch(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression { + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + // DecimalType is used to not lose precision while converting microseconds to + // the fractional part of seconds. Scale 6 is taken to have all microseconds as + // the fraction. The precision 20 should cover whole valid range of years [1, 9999] + // plus negative years that can be used in some cases though are not officially supported. + override def dataType: DataType = DecimalType(20, 6) + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override protected def nullSafeEval(timestamp: Any): Any = { + DateTimeUtils.getEpoch(timestamp.asInstanceOf[Long], zoneId) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + defineCodeGen(ctx, ev, c => s"$dtu.getEpoch($c, $zid)") + } +} 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 3d1ed69391c98..ca89b7c5a062b 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 @@ -1418,6 +1418,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Decade(expression(ctx.source)) case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(expression(ctx.source)) + case "ISOYEAR" => + IsoYear(expression(ctx.source)) case "QUARTER" | "QTR" => Quarter(expression(ctx.source)) case "MONTH" | "MON" | "MONS" | "MONTHS" => @@ -1440,6 +1442,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Minute(expression(ctx.source)) case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(expression(ctx.source)) + case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => + Milliseconds(expression(ctx.source)) + case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => + Microseconds(expression(ctx.source)) + case "EPOCH" => + Epoch(expression(ctx.source)) case other => throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 9e5aa2d8ea6a3..65a9bee5eaedd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -25,6 +25,7 @@ import java.util.concurrent.TimeUnit._ import scala.util.control.NonFatal +import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.UTF8String /** @@ -455,6 +456,23 @@ object DateTimeUtils { (MICROSECONDS.toSeconds(localTimestamp(microsec, timeZone)) % 60).toInt } + /** + * Returns seconds, including fractional parts, multiplied by 1000. The timestamp + * is expressed in microseconds since the epoch. + */ + def getMilliseconds(timestamp: SQLTimestamp, timeZone: TimeZone): Decimal = { + val micros = Decimal(getMicroseconds(timestamp, timeZone)) + (micros / Decimal(MICROS_PER_MILLIS)).toPrecision(8, 3) + } + + /** + * Returns seconds, including fractional parts, multiplied by 1000000. The timestamp + * is expressed in microseconds since the epoch. + */ + def getMicroseconds(timestamp: SQLTimestamp, timeZone: TimeZone): Int = { + Math.floorMod(localTimestamp(timestamp, timeZone), MICROS_PER_SECOND * 60).toInt + } + /** * Returns the 'day in year' value for the given date. The date is expressed in days * since 1.1.1970. @@ -489,6 +507,14 @@ object DateTimeUtils { LocalDate.ofEpochDay(date).getYear } + /** + * Returns the year which conforms to ISO 8601. Each ISO 8601 week-numbering + * year begins with the Monday of the week containing the 4th of January. + */ + def getIsoYear(date: SQLDate): Int = { + daysToLocalDate(date).get(IsoFields.WEEK_BASED_YEAR) + } + /** * Returns the quarter for the given date. The date is expressed in days * since 1.1.1970. @@ -812,4 +838,14 @@ object DateTimeUtils { def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { convertTz(time, getTimeZone(timeZone), TimeZoneGMT) } + + /** + * Returns the number of seconds with fractional part in microsecond precision + * since 1970-01-01 00:00:00 local time. + */ + def getEpoch(timestamp: SQLTimestamp, zoneId: ZoneId): Decimal = { + val offset = zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds + val sinceEpoch = BigDecimal(timestamp) / MICROS_PER_SECOND + offset + new Decimal().set(sinceEpoch, 20, 6) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index e75721d26b6db..ae3549b4aaf5f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{ZoneId, ZoneOffset} +import java.time.{LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -1010,4 +1010,47 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Decade(date.copy(year = Literal(-11))), -2) checkEvaluation(Decade(date.copy(year = Literal(-2019))), -202) } + + test("milliseconds and microseconds") { + outstandingTimezonesIds.foreach { timezone => + var timestamp = MakeTimestamp(Literal(2019), Literal(8), Literal(10), + Literal(0), Literal(0), Literal(Decimal(BigDecimal(10.123456789), 8, 6)), + Some(Literal(timezone))) + + checkEvaluation(Milliseconds(timestamp), Decimal(BigDecimal(10123.457), 8, 3)) + checkEvaluation(Microseconds(timestamp), 10123457) + + timestamp = timestamp.copy(sec = Literal(Decimal(0.0, 8, 6))) + checkEvaluation(Milliseconds(timestamp), Decimal(0, 8, 3)) + checkEvaluation(Microseconds(timestamp), 0) + + timestamp = timestamp.copy(sec = Literal(Decimal(BigDecimal(59.999999), 8, 6))) + checkEvaluation(Milliseconds(timestamp), Decimal(BigDecimal(59999.999), 8, 3)) + checkEvaluation(Microseconds(timestamp), 59999999) + + timestamp = timestamp.copy(sec = Literal(Decimal(BigDecimal(60.0), 8, 6))) + checkEvaluation(Milliseconds(timestamp), Decimal(0, 8, 3)) + checkEvaluation(Microseconds(timestamp), 0) + } + } + + test("epoch") { + val zoneId = ZoneId.systemDefault() + val nanos = 123456000 + val timestamp = Epoch(MakeTimestamp( + Literal(2019), Literal(8), Literal(9), Literal(0), Literal(0), + Literal(Decimal(nanos / DateTimeUtils.NANOS_PER_SECOND.toDouble, 8, 6)), + Some(Literal(zoneId.getId)))) + val instant = LocalDateTime.of(2019, 8, 9, 0, 0, 0, nanos) + .atZone(zoneId).toInstant + val expected = Decimal(BigDecimal(nanos) / DateTimeUtils.NANOS_PER_SECOND + + instant.getEpochSecond + + zoneId.getRules.getOffset(instant).getTotalSeconds) + checkEvaluation(timestamp, expected) + } + + test("ISO 8601 week-numbering year") { + checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(1))), 2005) + checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(2))), 2006) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/extract.sql b/sql/core/src/test/resources/sql-tests/inputs/extract.sql index f42b75623d5a2..2d6cad8838704 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/extract.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/extract.sql @@ -21,6 +21,8 @@ select extract(years from c) from t; select extract(yr from c) from t; select extract(yrs from c) from t; +select extract(isoyear from c) from t; + select extract(quarter from c) from t; select extract(qtr from c) from t; @@ -63,4 +65,20 @@ select extract(sec from c) from t; select extract(seconds from c) from t; select extract(secs from c) from t; +select extract(milliseconds from c) from t; +select extract(msec from c) from t; +select extract(msecs from c) from t; +select extract(millisecon from c) from t; +select extract(mseconds from c) from t; +select extract(ms from c) from t; + +select extract(microseconds from c) from t; +select extract(usec from c) from t; +select extract(usecs from c) from t; +select extract(useconds from c) from t; +select extract(microsecon from c) from t; +select extract(us from c) from t; + +select extract(epoch from c) from t; + select extract(not_supported from c) from t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index 8341dc07e7e88..b9a6b998e52fe 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -229,8 +229,8 @@ SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; -- -- epoch -- --- SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 --- SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01'); -- 0 +SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 +SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01'); -- 0 -- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00'); -- 0 -- -- century diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index a7efe825c98e5..b02dfe054344b 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 50 +-- Number of queries: 64 -- !query 0 @@ -147,15 +147,15 @@ struct -- !query 18 -select extract(quarter from c) from t +select extract(isoyear from c) from t -- !query 18 schema -struct +struct -- !query 18 output -2 +2011 -- !query 19 -select extract(qtr from c) from t +select extract(quarter from c) from t -- !query 19 schema struct -- !query 19 output @@ -163,15 +163,15 @@ struct -- !query 20 -select extract(month from c) from t +select extract(qtr from c) from t -- !query 20 schema -struct +struct -- !query 20 output -5 +2 -- !query 21 -select extract(mon from c) from t +select extract(month from c) from t -- !query 21 schema struct -- !query 21 output @@ -179,7 +179,7 @@ struct -- !query 22 -select extract(mons from c) from t +select extract(mon from c) from t -- !query 22 schema struct -- !query 22 output @@ -187,7 +187,7 @@ struct -- !query 23 -select extract(months from c) from t +select extract(mons from c) from t -- !query 23 schema struct -- !query 23 output @@ -195,15 +195,15 @@ struct -- !query 24 -select extract(week from c) from t +select extract(months from c) from t -- !query 24 schema -struct +struct -- !query 24 output -18 +5 -- !query 25 -select extract(w from c) from t +select extract(week from c) from t -- !query 25 schema struct -- !query 25 output @@ -211,7 +211,7 @@ struct -- !query 26 -select extract(weeks from c) from t +select extract(w from c) from t -- !query 26 schema struct -- !query 26 output @@ -219,15 +219,15 @@ struct -- !query 27 -select extract(day from c) from t +select extract(weeks from c) from t -- !query 27 schema -struct +struct -- !query 27 output -6 +18 -- !query 28 -select extract(d from c) from t +select extract(day from c) from t -- !query 28 schema struct -- !query 28 output @@ -235,7 +235,7 @@ struct -- !query 29 -select extract(days from c) from t +select extract(d from c) from t -- !query 29 schema struct -- !query 29 output @@ -243,47 +243,47 @@ struct -- !query 30 -select extract(dayofweek from c) from t +select extract(days from c) from t -- !query 30 schema -struct +struct -- !query 30 output 6 -- !query 31 -select extract(dow from c) from t +select extract(dayofweek from c) from t -- !query 31 schema -struct<(dayofweek(CAST(c AS DATE)) - 1):int> +struct -- !query 31 output -5 +6 -- !query 32 -select extract(isodow from c) from t +select extract(dow from c) from t -- !query 32 schema -struct<(weekday(CAST(c AS DATE)) + 1):int> +struct<(dayofweek(CAST(c AS DATE)) - 1):int> -- !query 32 output 5 -- !query 33 -select extract(doy from c) from t +select extract(isodow from c) from t -- !query 33 schema -struct +struct<(weekday(CAST(c AS DATE)) + 1):int> -- !query 33 output -126 +5 -- !query 34 -select extract(hour from c) from t +select extract(doy from c) from t -- !query 34 schema -struct +struct -- !query 34 output -7 +126 -- !query 35 -select extract(h from c) from t +select extract(hour from c) from t -- !query 35 schema struct -- !query 35 output @@ -291,7 +291,7 @@ struct -- !query 36 -select extract(hours from c) from t +select extract(h from c) from t -- !query 36 schema struct -- !query 36 output @@ -299,7 +299,7 @@ struct -- !query 37 -select extract(hr from c) from t +select extract(hours from c) from t -- !query 37 schema struct -- !query 37 output @@ -307,7 +307,7 @@ struct -- !query 38 -select extract(hrs from c) from t +select extract(hr from c) from t -- !query 38 schema struct -- !query 38 output @@ -315,15 +315,15 @@ struct -- !query 39 -select extract(minute from c) from t +select extract(hrs from c) from t -- !query 39 schema -struct +struct -- !query 39 output -8 +7 -- !query 40 -select extract(m from c) from t +select extract(minute from c) from t -- !query 40 schema struct -- !query 40 output @@ -331,7 +331,7 @@ struct -- !query 41 -select extract(min from c) from t +select extract(m from c) from t -- !query 41 schema struct -- !query 41 output @@ -339,7 +339,7 @@ struct -- !query 42 -select extract(mins from c) from t +select extract(min from c) from t -- !query 42 schema struct -- !query 42 output @@ -347,7 +347,7 @@ struct -- !query 43 -select extract(minutes from c) from t +select extract(mins from c) from t -- !query 43 schema struct -- !query 43 output @@ -355,15 +355,15 @@ struct -- !query 44 -select extract(second from c) from t +select extract(minutes from c) from t -- !query 44 schema -struct +struct -- !query 44 output -9 +8 -- !query 45 -select extract(s from c) from t +select extract(second from c) from t -- !query 45 schema struct -- !query 45 output @@ -371,7 +371,7 @@ struct -- !query 46 -select extract(sec from c) from t +select extract(s from c) from t -- !query 46 schema struct -- !query 46 output @@ -379,7 +379,7 @@ struct -- !query 47 -select extract(seconds from c) from t +select extract(sec from c) from t -- !query 47 schema struct -- !query 47 output @@ -387,7 +387,7 @@ struct -- !query 48 -select extract(secs from c) from t +select extract(seconds from c) from t -- !query 48 schema struct -- !query 48 output @@ -395,10 +395,122 @@ struct -- !query 49 -select extract(not_supported from c) from t +select extract(secs from c) from t -- !query 49 schema -struct<> +struct -- !query 49 output +9 + + +-- !query 50 +select extract(milliseconds from c) from t +-- !query 50 schema +struct +-- !query 50 output +9123.456 + + +-- !query 51 +select extract(msec from c) from t +-- !query 51 schema +struct +-- !query 51 output +9123.456 + + +-- !query 52 +select extract(msecs from c) from t +-- !query 52 schema +struct +-- !query 52 output +9123.456 + + +-- !query 53 +select extract(millisecon from c) from t +-- !query 53 schema +struct +-- !query 53 output +9123.456 + + +-- !query 54 +select extract(mseconds from c) from t +-- !query 54 schema +struct +-- !query 54 output +9123.456 + + +-- !query 55 +select extract(ms from c) from t +-- !query 55 schema +struct +-- !query 55 output +9123.456 + + +-- !query 56 +select extract(microseconds from c) from t +-- !query 56 schema +struct +-- !query 56 output +9123456 + + +-- !query 57 +select extract(usec from c) from t +-- !query 57 schema +struct +-- !query 57 output +9123456 + + +-- !query 58 +select extract(usecs from c) from t +-- !query 58 schema +struct +-- !query 58 output +9123456 + + +-- !query 59 +select extract(useconds from c) from t +-- !query 59 schema +struct +-- !query 59 output +9123456 + + +-- !query 60 +select extract(microsecon from c) from t +-- !query 60 schema +struct +-- !query 60 output +9123456 + + +-- !query 61 +select extract(us from c) from t +-- !query 61 schema +struct +-- !query 61 output +9123456 + + +-- !query 62 +select extract(epoch from c) from t +-- !query 62 schema +struct +-- !query 62 output +1304665689.123456 + + +-- !query 63 +select extract(not_supported from c) from t +-- !query 63 schema +struct<> +-- !query 63 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'NOT_SUPPORTED' are currently not supported.(line 1, pos 7) diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index f5586c5a4aa3a..cb2be6d1cd22d 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 89 +-- Number of queries: 91 -- !query 0 @@ -502,336 +502,352 @@ struct -- !query 47 -SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 47 schema -struct +struct -- !query 47 output --2 +0 -- !query 48 -SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 48 schema -struct +struct -- !query 48 output --1 +0 -- !query 49 -SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query 49 schema -struct +struct -- !query 49 output --1 +-2 -- !query 50 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query 50 schema -struct +struct -- !query 50 output -1 +-1 -- !query 51 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 51 schema -struct +struct -- !query 51 output -1 +-1 -- !query 52 -SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') -- !query 52 schema -struct +struct -- !query 52 output -19 +1 -- !query 53 -SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') -- !query 53 schema -struct +struct -- !query 53 output -20 +1 -- !query 54 -SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') -- !query 54 schema -struct +struct -- !query 54 output -20 +19 -- !query 55 -SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') -- !query 55 schema -struct +struct -- !query 55 output -21 +20 -- !query 56 -SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') -- !query 56 schema -struct +struct -- !query 56 output -true +20 -- !query 57 -SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') -- !query 57 schema -struct +struct -- !query 57 output --1 +21 -- !query 58 -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True -- !query 58 schema -struct +struct -- !query 58 output -1 +true -- !query 59 -SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 59 schema -struct +struct -- !query 59 output -1 +-1 -- !query 60 -SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') -- !query 60 schema -struct +struct -- !query 60 output -2 +1 -- !query 61 -SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') -- !query 61 schema -struct +struct -- !query 61 output -2 +1 -- !query 62 -SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') -- !query 62 schema -struct +struct -- !query 62 output -3 +2 -- !query 63 -SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') -- !query 63 schema -struct +struct -- !query 63 output -3 +2 -- !query 64 -SELECT EXTRACT(DECADE FROM DATE '1994-12-25') +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') -- !query 64 schema -struct +struct -- !query 64 output -199 +3 -- !query 65 -SELECT EXTRACT(DECADE FROM DATE '0010-01-01') +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) -- !query 65 schema -struct +struct -- !query 65 output -1 +3 -- !query 66 -SELECT EXTRACT(DECADE FROM DATE '0009-12-31') +SELECT EXTRACT(DECADE FROM DATE '1994-12-25') -- !query 66 schema -struct +struct -- !query 66 output -0 +199 -- !query 67 -SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(DECADE FROM DATE '0010-01-01') -- !query 67 schema -struct +struct -- !query 67 output -0 +1 -- !query 68 -SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(DECADE FROM DATE '0009-12-31') -- !query 68 schema -struct +struct -- !query 68 output --1 +0 -- !query 69 -SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) -- !query 69 schema -struct +struct -- !query 69 output --1 +0 -- !query 70 -SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query 70 schema -struct +struct -- !query 70 output --2 +-1 -- !query 71 -SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) -- !query 71 schema -struct +struct -- !query 71 output -true +-1 -- !query 72 -SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) -- !query 72 schema -struct +struct -- !query 72 output -20 +-2 -- !query 73 -SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True -- !query 73 schema -struct +struct -- !query 73 output -1001-01-01 00:07:02 +true -- !query 74 -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 74 schema -struct +struct -- !query 74 output -1001-01-01 00:07:02 +20 -- !query 75 -SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 75 schema -struct +struct -- !query 75 output -1901-01-01 00:00:00 +1001-01-01 00:07:02 -- !query 76 -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query 76 schema -struct +struct -- !query 76 output -1901-01-01 00:00:00 +1001-01-01 00:07:02 -- !query 77 -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 77 schema -struct +struct -- !query 77 output -2001-01-01 00:00:00 +1901-01-01 00:00:00 -- !query 78 -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') -- !query 78 schema -struct +struct -- !query 78 output -0001-01-01 00:07:02 +1901-01-01 00:00:00 -- !query 79 -SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') -- !query 79 schema -struct +struct -- !query 79 output --0099-01-01 00:07:02 +2001-01-01 00:00:00 -- !query 80 -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query 80 schema -struct +struct -- !query 80 output -1990-01-01 00:00:00 +0001-01-01 00:07:02 -- !query 81 -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) -- !query 81 schema -struct +struct -- !query 81 output -0000-01-01 00:07:02 +-0099-01-01 00:07:02 -- !query 82 -SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') -- !query 82 schema -struct +struct -- !query 82 output --0010-01-01 00:07:02 +1990-01-01 00:00:00 -- !query 83 -select make_date(2013, 7, 15) +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') -- !query 83 schema -struct +struct -- !query 83 output -2013-07-15 +0000-01-01 00:07:02 -- !query 84 -select make_date(-44, 3, 15) +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query 84 schema -struct +struct -- !query 84 output --0044-03-15 +-0010-01-01 00:07:02 -- !query 85 -select make_date(2013, 2, 30) +select make_date(2013, 7, 15) -- !query 85 schema -struct +struct -- !query 85 output -NULL +2013-07-15 -- !query 86 -select make_date(2013, 13, 1) +select make_date(-44, 3, 15) -- !query 86 schema -struct +struct -- !query 86 output -NULL +-0044-03-15 -- !query 87 -select make_date(2013, 11, -1) +select make_date(2013, 2, 30) -- !query 87 schema -struct +struct -- !query 87 output NULL -- !query 88 -DROP TABLE DATE_TBL +select make_date(2013, 13, 1) -- !query 88 schema -struct<> +struct -- !query 88 output +NULL + + +-- !query 89 +select make_date(2013, 11, -1) +-- !query 89 schema +struct +-- !query 89 output +NULL + + +-- !query 90 +DROP TABLE DATE_TBL +-- !query 90 schema +struct<> +-- !query 90 output From 3ec24fd128c011356b6f872bf1477134ccdf1f07 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Thu, 15 Aug 2019 10:39:33 +0900 Subject: [PATCH 068/149] [SPARK-28203][CORE][PYTHON] PythonRDD should respect SparkContext's hadoop configuration ## What changes were proposed in this pull request? 1. PythonHadoopUtil.mapToConf generates a Configuration with loadDefaults disabled 2. merging hadoop conf in several places of PythonRDD is consistent. ## How was this patch tested? Added a new test and existed tests Closes #25002 from advancedxy/SPARK-28203. Authored-by: Xianjin YE Signed-off-by: HyukjinKwon --- .../spark/api/python/PythonHadoopUtil.scala | 2 +- .../apache/spark/api/python/PythonRDD.scala | 6 +- .../spark/api/python/PythonRDDSuite.scala | 87 ++++++++++++++++++- 3 files changed, 88 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index a4817b3cf770d..5d0f1dcc88097 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -156,7 +156,7 @@ private[python] object PythonHadoopUtil { * Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]] */ def mapToConf(map: java.util.Map[String, String]): Configuration = { - val conf = new Configuration() + val conf = new Configuration(false) map.asScala.foreach { case (k, v) => conf.set(k, v) } conf } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 080dcca035928..4d76ff76e6752 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -335,7 +335,7 @@ private[spark] object PythonRDD extends Logging { valueConverterClass: String, confAsMap: java.util.HashMap[String, String], batchSize: Int): JavaRDD[Array[Byte]] = { - val conf = PythonHadoopUtil.mapToConf(confAsMap) + val conf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, None, inputFormatClass, keyClass, valueClass, conf) @@ -404,7 +404,7 @@ private[spark] object PythonRDD extends Logging { valueConverterClass: String, confAsMap: java.util.HashMap[String, String], batchSize: Int): JavaRDD[Array[Byte]] = { - val conf = PythonHadoopUtil.mapToConf(confAsMap) + val conf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = hadoopRDDFromClassNames[K, V, F](sc, None, inputFormatClass, keyClass, valueClass, conf) @@ -620,7 +620,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, useNewAPI: Boolean): Unit = { - val conf = PythonHadoopUtil.mapToConf(confAsMap) + val conf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized), keyConverterClass, valueConverterClass, new JavaToWritableConverter) if (useNewAPI) { diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index e2ec50fb1f172..aae5fb002e1e8 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -17,16 +17,42 @@ package org.apache.spark.api.python -import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.io.{ByteArrayOutputStream, DataOutputStream, File} import java.net.{InetAddress, Socket} import java.nio.charset.StandardCharsets +import java.util +import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.concurrent.duration.Duration -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapred.TextInputFormat +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.security.{SocketAuthHelper, SocketAuthServer} +import org.apache.spark.util.Utils + +class PythonRDDSuite extends SparkFunSuite with LocalSparkContext { -class PythonRDDSuite extends SparkFunSuite { + var tempDir: File = _ + + override def beforeAll(): Unit = { + super.beforeAll() + tempDir = Utils.createTempDir() + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterAll() + } + } test("Writing large strings to the worker") { val input: List[String] = List("a"*100000) @@ -65,4 +91,59 @@ class PythonRDDSuite extends SparkFunSuite { throw new Exception("exception within handleConnection") } } + + test("mapToConf should not load defaults") { + val map = Map("key" -> "value") + val conf = PythonHadoopUtil.mapToConf(map.asJava) + assert(conf.size() === map.size) + assert(conf.get("key") === map("key")) + } + + test("SparkContext's hadoop configuration should be respected in PythonRDD") { + // hadoop conf with default configurations + val hadoopConf = new Configuration() + assert(hadoopConf.size() > 0) + val headEntry = hadoopConf.asScala.head + val (firstKey, firstValue) = (headEntry.getKey, headEntry.getValue) + + // passed to spark conf with a different value(prefixed by spark.) + val conf = new SparkConf().setAppName("test").setMaster("local") + conf.set("spark.hadoop." + firstKey, "spark." + firstValue) + + sc = new SparkContext(conf) + val outDir = new File(tempDir, "output").getAbsolutePath + // write output as HadoopRDD's input + sc.makeRDD(1 to 1000, 10).saveAsTextFile(outDir) + + val javaSparkContext = new JavaSparkContext(sc) + val confMap = new util.HashMap[String, String]() + // set input path in job conf + confMap.put(FileInputFormat.INPUT_DIR, outDir) + + val pythonRDD = PythonRDD.hadoopRDD( + javaSparkContext, + classOf[TextInputFormat].getCanonicalName, + classOf[LongWritable].getCanonicalName, + classOf[Text].getCanonicalName, + null, + null, + confMap, + 0 + ) + + @tailrec + def getRootRDD(rdd: RDD[_]): RDD[_] = { + rdd.dependencies match { + case Nil => rdd + case dependency :: _ => getRootRDD(dependency.rdd) + } + } + + // retrieve hadoopRDD as it's a root RDD + val hadoopRDD = getRootRDD(pythonRDD).asInstanceOf[HadoopRDD[_, _]] + val jobConf = hadoopRDD.getConf + // the jobConf passed to HadoopRDD should contain SparkContext's hadoop items rather the default + // configs in client's Configuration + assert(jobConf.get(firstKey) === "spark." + firstValue) + } } From 0526529b31737e5bf4829f8259f3a020f2cc51f1 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 15 Aug 2019 12:29:34 +0800 Subject: [PATCH 069/149] [SPARK-28666] Support saveAsTable for V2 tables through Session Catalog ## What changes were proposed in this pull request? We add support for the V2SessionCatalog for saveAsTable, such that V2 tables can plug in and leverage existing DataFrameWriter.saveAsTable APIs to write and create tables through the session catalog. ## How was this patch tested? Unit tests. A lot of tests broke under hive when things were not working properly under `ResolveTables`, therefore I believe the current set of tests should be sufficient in testing the table resolution and read code paths. Closes #25402 from brkyvz/saveAsV2. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../sql/catalog/v2/utils/CatalogV2Util.scala | 4 +- .../sources/v2/internal/UnresolvedTable.scala | 89 +++++++ .../sql/catalyst/analysis/Analyzer.scala | 11 +- .../apache/spark/sql/DataFrameWriter.scala | 35 ++- .../datasources/DataSourceResolution.scala | 4 - .../datasources/v2/V2SessionCatalog.scala | 65 +---- ...SourceV2DataFrameSessionCatalogSuite.scala | 225 ++++++++++++++++++ .../sql/sources/v2/DataSourceV2SQLSuite.scala | 9 +- 8 files changed, 361 insertions(+), 81 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala index cd9bcc0f44f74..d5079202c8fee 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, NamespaceChange, TableChange} import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType} -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} @@ -219,5 +219,7 @@ object CatalogV2Util { Option(catalog.asTableCatalog.loadTable(ident)) } catch { case _: NoSuchTableException => None + case _: NoSuchDatabaseException => None + case _: NoSuchNamespaceException => None } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala new file mode 100644 index 0000000000000..8813d0ab840d0 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.internal + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.catalog.v2.expressions.{LogicalExpressions, Transform} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.types.StructType + +/** + * An implementation of catalog v2 `Table` to expose v1 table metadata. + */ +case class UnresolvedTable(v1Table: CatalogTable) extends Table { + implicit class IdentifierHelper(identifier: TableIdentifier) { + def quoted: String = { + identifier.database match { + case Some(db) => + Seq(db, identifier.table).map(quote).mkString(".") + case _ => + quote(identifier.table) + + } + } + + private def quote(part: String): String = { + if (part.contains(".") || part.contains("`")) { + s"`${part.replace("`", "``")}`" + } else { + part + } + } + } + + def catalogTable: CatalogTable = v1Table + + lazy val options: Map[String, String] = { + v1Table.storage.locationUri match { + case Some(uri) => + v1Table.storage.properties + ("path" -> uri.toString) + case _ => + v1Table.storage.properties + } + } + + override lazy val properties: util.Map[String, String] = v1Table.properties.asJava + + override lazy val schema: StructType = v1Table.schema + + override lazy val partitioning: Array[Transform] = { + val partitions = new mutable.ArrayBuffer[Transform]() + + v1Table.partitionColumnNames.foreach { col => + partitions += LogicalExpressions.identity(col) + } + + v1Table.bucketSpec.foreach { spec => + partitions += LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*) + } + + partitions.toArray + } + + override def name: String = v1Table.identifier.quoted + + override def capabilities: util.Set[TableCapability] = new util.HashSet[TableCapability]() + + override def toString: String = s"UnresolvedTable($name)" +} 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 7267ad8d37c82..3a72988f8345d 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 @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.sources.v2.internal.UnresolvedTable import org.apache.spark.sql.types._ /** @@ -650,8 +651,14 @@ class Analyzer( if catalog.isTemporaryTable(ident) => u // temporary views take precedence over catalog table names - case u @ UnresolvedRelation(CatalogObjectIdentifier(Some(catalogPlugin), ident)) => - loadTable(catalogPlugin, ident).map(DataSourceV2Relation.create).getOrElse(u) + case u @ UnresolvedRelation(CatalogObjectIdentifier(maybeCatalog, ident)) => + maybeCatalog.orElse(sessionCatalog) + .flatMap(loadTable(_, ident)) + .map { + case unresolved: UnresolvedTable => u + case resolved => DataSourceV2Relation.create(resolved) + } + .getOrElse(u) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index af7ddd756ae89..0b49cf24e6c7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister} import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.sources.v2.internal.UnresolvedTable import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -251,19 +252,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { assertNotBucketed("save") val session = df.sparkSession - val useV1Sources = - session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",") val cls = DataSource.lookupDataSource(source, session.sessionState.conf) - val shouldUseV1Source = cls.newInstance() match { - case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true - case _ => useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT)) - } + val canUseV2 = canUseV2Source(session, cls) && partitioningColumns.isEmpty // In Data Source V2 project, partitioning is still under development. // Here we fallback to V1 if partitioning columns are specified. // TODO(SPARK-26778): use V2 implementations when partitioning feature is supported. - if (!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(cls) && - partitioningColumns.isEmpty) { + if (canUseV2) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, session.sessionState.conf) @@ -493,13 +488,20 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ val session = df.sparkSession + val provider = DataSource.lookupDataSource(source, session.sessionState.conf) + val canUseV2 = canUseV2Source(session, provider) + val sessionCatalogOpt = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case CatalogObjectIdentifier(Some(catalog), ident) => saveAsTable(catalog.asTableCatalog, ident, modeForDSV2) - // TODO(SPARK-28666): This should go through V2SessionCatalog + + case CatalogObjectIdentifier(None, ident) + if canUseV2 && sessionCatalogOpt.isDefined && ident.namespace().length <= 1 => + // We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility + // for now. + saveAsTable(sessionCatalogOpt.get.asTableCatalog, ident, modeForDSV1) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) @@ -525,6 +527,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } val command = (mode, tableOpt) match { + case (_, Some(table: UnresolvedTable)) => + return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption)) + case (SaveMode.Append, Some(table)) => AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan) @@ -830,6 +835,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def modeForDSV2 = mode.getOrElse(SaveMode.Append) + private def canUseV2Source(session: SparkSession, providerClass: Class[_]): Boolean = { + val useV1Sources = + session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",") + val shouldUseV1Source = providerClass.newInstance() match { + case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true + case _ => useV1Sources.contains(providerClass.getCanonicalName.toLowerCase(Locale.ROOT)) + } + !shouldUseV1Source && classOf[TableProvider].isAssignableFrom(providerClass) + } + /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options /////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 4791fe5fb5251..48b504a6545f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -31,11 +31,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} -import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} -import org.apache.spark.sql.util.SchemaUtils case class DataSourceResolution( conf: SQLConf, @@ -183,8 +181,6 @@ case class DataSourceResolution( val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation) DeleteFromTable(aliased, delete.condition) - case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) => - UnresolvedCatalogRelation(catalogTable) } object V1WriteProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 79ea8756721ed..6dcebe29537d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchT import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SessionState -import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.sources.v2.internal.UnresolvedTable import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -70,7 +71,7 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { throw new NoSuchTableException(ident) } - CatalogTableAsV2(catalogTable) + UnresolvedTable(catalogTable) } override def invalidateTable(ident: Identifier): Unit = { @@ -179,66 +180,6 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { override def toString: String = s"V2SessionCatalog($name)" } -/** - * An implementation of catalog v2 [[Table]] to expose v1 table metadata. - */ -case class CatalogTableAsV2(v1Table: CatalogTable) extends Table { - implicit class IdentifierHelper(identifier: TableIdentifier) { - def quoted: String = { - identifier.database match { - case Some(db) => - Seq(db, identifier.table).map(quote).mkString(".") - case _ => - quote(identifier.table) - - } - } - - private def quote(part: String): String = { - if (part.contains(".") || part.contains("`")) { - s"`${part.replace("`", "``")}`" - } else { - part - } - } - } - - def catalogTable: CatalogTable = v1Table - - lazy val options: Map[String, String] = { - v1Table.storage.locationUri match { - case Some(uri) => - v1Table.storage.properties + ("path" -> uri.toString) - case _ => - v1Table.storage.properties - } - } - - override lazy val properties: util.Map[String, String] = v1Table.properties.asJava - - override lazy val schema: StructType = v1Table.schema - - override lazy val partitioning: Array[Transform] = { - val partitions = new mutable.ArrayBuffer[Transform]() - - v1Table.partitionColumnNames.foreach { col => - partitions += LogicalExpressions.identity(col) - } - - v1Table.bucketSpec.foreach { spec => - partitions += LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*) - } - - partitions.toArray - } - - override def name: String = v1Table.identifier.quoted - - override def capabilities: util.Set[TableCapability] = new util.HashSet[TableCapability]() - - override def toString: String = s"CatalogTableAsV2($name)" -} - private[sql] object V2SessionCatalog { /** * Convert v2 Transforms to v1 partition columns and an optional bucket spec. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala new file mode 100644 index 0000000000000..2ef2df3345e8f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -0,0 +1,225 @@ +/* + * 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.sources.v2 + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} +import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class DataSourceV2DataFrameSessionCatalogSuite + extends QueryTest + with SharedSQLContext + with BeforeAndAfter { + import testImplicits._ + + private val v2Format = classOf[InMemoryTableProvider].getName + + before { + spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[TestV2SessionCatalog].getName) + } + + override def afterEach(): Unit = { + super.afterEach() + spark.catalog("session").asInstanceOf[TestV2SessionCatalog].clearTables() + spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + } + + private def verifyTable(tableName: String, expected: DataFrame): Unit = { + checkAnswer(spark.table(tableName), expected) + checkAnswer(sql(s"SELECT * FROM $tableName"), expected) + checkAnswer(sql(s"SELECT * FROM default.$tableName"), expected) + checkAnswer(sql(s"TABLE $tableName"), expected) + } + + test("saveAsTable: v2 table - table doesn't exist and default mode (ErrorIfExists)") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.format(v2Format).saveAsTable(t1) + verifyTable(t1, df) + } + + test("saveAsTable: v2 table - table doesn't exist and append mode") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.format(v2Format).mode("append").saveAsTable(t1) + verifyTable(t1, df) + } + + test("saveAsTable: Append mode should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.format(v2Format).mode(SaveMode.Append).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable: Append mode should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + val format = spark.sessionState.conf.defaultDataSourceName + sql(s"CREATE TABLE same_name(id LONG) USING $format") + spark.range(10).createTempView("same_name") + spark.range(20).write.format(v2Format).mode(SaveMode.Append).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable: v2 table - table exists") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + spark.sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + intercept[TableAlreadyExistsException] { + df.select("id", "data").write.format(v2Format).saveAsTable(t1) + } + df.write.format(v2Format).mode("append").saveAsTable(t1) + verifyTable(t1, df) + + // Check that appends are by name + df.select('data, 'id).write.format(v2Format).mode("append").saveAsTable(t1) + verifyTable(t1, df.union(df)) + } + + test("saveAsTable: v2 table - table overwrite and table doesn't exist") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.format(v2Format).mode("overwrite").saveAsTable(t1) + verifyTable(t1, df) + } + + test("saveAsTable: v2 table - table overwrite and table exists") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'") + df.write.format(v2Format).mode("overwrite").saveAsTable(t1) + verifyTable(t1, df) + } + + test("saveAsTable: Overwrite mode should not drop the temp view if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.format(v2Format).mode(SaveMode.Overwrite).saveAsTable("same_name") + assert(spark.sessionState.catalog.getTempView("same_name").isDefined) + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql(s"CREATE TABLE same_name(id LONG) USING $v2Format") + spark.range(10).createTempView("same_name") + spark.range(20).write.format(v2Format).mode(SaveMode.Overwrite).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable: v2 table - ignore mode and table doesn't exist") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.format(v2Format).mode("ignore").saveAsTable(t1) + verifyTable(t1, df) + } + + test("saveAsTable: v2 table - ignore mode and table exists") { + val t1 = "tbl" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'") + df.write.format(v2Format).mode("ignore").saveAsTable(t1) + verifyTable(t1, Seq(("c", "d")).toDF("id", "data")) + } +} + +class InMemoryTableProvider extends TableProvider { + override def getTable(options: CaseInsensitiveStringMap): Table = { + throw new UnsupportedOperationException("D'oh!") + } +} + +/** A SessionCatalog that always loads an in memory Table, so we can test write code paths. */ +class TestV2SessionCatalog extends V2SessionCatalog { + + protected val tables: util.Map[Identifier, InMemoryTable] = + new ConcurrentHashMap[Identifier, InMemoryTable]() + + private def fullIdentifier(ident: Identifier): Identifier = { + if (ident.namespace().isEmpty) { + Identifier.of(Array("default"), ident.name()) + } else { + ident + } + } + + override def loadTable(ident: Identifier): Table = { + val fullIdent = fullIdentifier(ident) + if (tables.containsKey(fullIdent)) { + tables.get(fullIdent) + } else { + // Table was created through the built-in catalog + val t = super.loadTable(fullIdent) + val table = new InMemoryTable(t.name(), t.schema(), t.partitioning(), t.properties()) + tables.put(fullIdent, table) + table + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + val created = super.createTable(ident, schema, partitions, properties) + val t = new InMemoryTable(created.name(), schema, partitions, properties) + val fullIdent = fullIdentifier(ident) + tables.put(fullIdent, t) + t + } + + def clearTables(): Unit = { + assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session catalog code path?") + tables.keySet().asScala.foreach(super.dropTable) + tables.clear() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index 9b1a23a1f2bbf..cfa6506a95e76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG} +import org.apache.spark.sql.sources.v2.internal.UnresolvedTable import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -493,8 +494,12 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn sparkSession.sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source") - // use the catalog name to force loading with the v2 catalog - checkAnswer(sparkSession.sql(s"TABLE session.table_name"), sparkSession.table("source")) + checkAnswer(sparkSession.sql(s"TABLE default.table_name"), sparkSession.table("source")) + // The fact that the following line doesn't throw an exception means, the session catalog + // can load the table. + val t = sparkSession.catalog("session").asTableCatalog + .loadTable(Identifier.of(Array.empty, "table_name")) + assert(t.isInstanceOf[UnresolvedTable], "V1 table wasn't returned as an unresolved table") } test("DropTable: basic") { From a493031e2e34f0d0ee92f0d541291dd473ee164e Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Thu, 15 Aug 2019 14:43:52 +0800 Subject: [PATCH 070/149] [SPARK-28695][SS] Use CaseInsensitiveMap in KafkaSourceProvider to make source param handling more robust ## What changes were proposed in this pull request? [SPARK-28163](https://issues.apache.org/jira/browse/SPARK-28163) fixed a bug and during the analysis we've concluded it would be more robust to use `CaseInsensitiveMap` inside Kafka source. This case less lower/upper case problem would rise in the future. Please note this PR doesn't intend to solve any kind of actual problem but finish the concept added in [SPARK-28163](https://issues.apache.org/jira/browse/SPARK-28163) (in a fix PR I didn't want to add too invasive changes). In this PR I've changed `Map[String, String]` to `CaseInsensitiveMap[String]` to enforce the usage. These are the main use-cases: * `contains` => `CaseInsensitiveMap` solves it * `get...` => `CaseInsensitiveMap` solves it * `filter` => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set * `find` => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set * passing parameters to Kafka consumer/producer => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set ## How was this patch tested? Existing unit tests. Closes #25418 from gaborgsomogyi/SPARK-28695. Authored-by: Gabor Somogyi Signed-off-by: Wenchen Fan --- .../spark/sql/kafka010/KafkaSource.scala | 4 +- .../sql/kafka010/KafkaSourceProvider.scala | 124 +++++++++--------- .../kafka010/KafkaMicroBatchSourceSuite.scala | 7 +- 3 files changed, 71 insertions(+), 64 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 2858ff1162b58..e4ed84552b964 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} @@ -76,7 +76,7 @@ private[kafka010] class KafkaSource( sqlContext: SQLContext, kafkaReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], + sourceOptions: CaseInsensitiveMap[String], metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 372bcab1cab30..c3f0be4be96e2 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -67,7 +67,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister schema: Option[StructType], providerName: String, parameters: Map[String, String]): (String, StructType) = { - validateStreamOptions(parameters) + val caseInsensitiveParameters = CaseInsensitiveMap(parameters) + validateStreamOptions(caseInsensitiveParameters) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") (shortName(), KafkaOffsetReader.kafkaSchema) } @@ -85,7 +86,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // id. Hence, we should generate a unique id for each query. val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveParameters, metadataPath) - val specifiedKafkaParams = convertToSpecifiedParams(parameters) + val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) @@ -121,7 +122,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String]): BaseRelation = { val caseInsensitiveParameters = CaseInsensitiveMap(parameters) validateBatchOptions(caseInsensitiveParameters) - val specifiedKafkaParams = convertToSpecifiedParams(parameters) + val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) @@ -146,8 +147,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { - val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) - val specifiedKafkaParams = kafkaParamsForProducer(parameters) + val caseInsensitiveParameters = CaseInsensitiveMap(parameters) + val defaultTopic = caseInsensitiveParameters.get(TOPIC_OPTION_KEY).map(_.trim) + val specifiedKafkaParams = kafkaParamsForProducer(caseInsensitiveParameters) new KafkaSink(sqlContext, specifiedKafkaParams, defaultTopic) } @@ -163,8 +165,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister s"${SaveMode.ErrorIfExists} (default).") case _ => // good } - val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) - val specifiedKafkaParams = kafkaParamsForProducer(parameters) + val caseInsensitiveParameters = CaseInsensitiveMap(parameters) + val topic = caseInsensitiveParameters.get(TOPIC_OPTION_KEY).map(_.trim) + val specifiedKafkaParams = kafkaParamsForProducer(caseInsensitiveParameters) KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, specifiedKafkaParams, topic) @@ -184,28 +187,31 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - private def strategy(caseInsensitiveParams: Map[String, String]) = - caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { - case (ASSIGN, value) => - AssignStrategy(JsonUtils.partitions(value)) - case (SUBSCRIBE, value) => - SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) - case (SUBSCRIBE_PATTERN, value) => - SubscribePatternStrategy(value.trim()) - case _ => - // Should never reach here as we are already matching on - // matched strategy names - throw new IllegalArgumentException("Unknown option") + private def strategy(params: CaseInsensitiveMap[String]) = { + val lowercaseParams = params.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + + lowercaseParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case (ASSIGN, value) => + AssignStrategy(JsonUtils.partitions(value)) + case (SUBSCRIBE, value) => + SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) + case (SUBSCRIBE_PATTERN, value) => + SubscribePatternStrategy(value.trim()) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } } - private def failOnDataLoss(caseInsensitiveParams: Map[String, String]) = - caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + private def failOnDataLoss(params: CaseInsensitiveMap[String]) = + params.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean - private def validateGeneralOptions(parameters: Map[String, String]): Unit = { + private def validateGeneralOptions(params: CaseInsensitiveMap[String]): Unit = { // Validate source options - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val lowercaseParams = params.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } val specifiedStrategies = - caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + lowercaseParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq if (specifiedStrategies.isEmpty) { throw new IllegalArgumentException( @@ -217,7 +223,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") } - caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + lowercaseParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { case (ASSIGN, value) => if (!value.trim.startsWith("{")) { throw new IllegalArgumentException( @@ -233,7 +239,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister s"'subscribe' is '$value'") } case (SUBSCRIBE_PATTERN, value) => - val pattern = caseInsensitiveParams(SUBSCRIBE_PATTERN).trim() + val pattern = params(SUBSCRIBE_PATTERN).trim() if (pattern.isEmpty) { throw new IllegalArgumentException( "Pattern to subscribe is empty as specified value for option " + @@ -246,22 +252,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } // Validate minPartitions value if present - if (caseInsensitiveParams.contains(MIN_PARTITIONS_OPTION_KEY)) { - val p = caseInsensitiveParams(MIN_PARTITIONS_OPTION_KEY).toInt + if (params.contains(MIN_PARTITIONS_OPTION_KEY)) { + val p = params(MIN_PARTITIONS_OPTION_KEY).toInt if (p <= 0) throw new IllegalArgumentException("minPartitions must be positive") } // Validate user-specified Kafka options - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + if (params.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE) - if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) { + if (params.contains(GROUP_ID_PREFIX)) { logWarning("Option 'groupIdPrefix' will be ignored as " + s"option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set.") } } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + if (params.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { throw new IllegalArgumentException( s""" |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. @@ -275,14 +281,14 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister """.stripMargin) } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + if (params.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + "to explicitly deserialize the keys.") } - if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + if (params.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " @@ -295,29 +301,29 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe otherUnsupportedConfigs.foreach { c => - if (caseInsensitiveParams.contains(s"kafka.$c")) { + if (params.contains(s"kafka.$c")) { throw new IllegalArgumentException(s"Kafka option '$c' is not supported") } } - if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + if (!params.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { throw new IllegalArgumentException( s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + s"configuring Kafka consumer") } } - private def validateStreamOptions(caseInsensitiveParams: Map[String, String]) = { + private def validateStreamOptions(params: CaseInsensitiveMap[String]) = { // Stream specific options - caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ => + params.get(ENDING_OFFSETS_OPTION_KEY).map(_ => throw new IllegalArgumentException("ending offset not valid in streaming queries")) - validateGeneralOptions(caseInsensitiveParams) + validateGeneralOptions(params) } - private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = { + private def validateBatchOptions(params: CaseInsensitiveMap[String]) = { // Batch specific options KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match { + params, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => // good to go case LatestOffsetRangeLimit => throw new IllegalArgumentException("starting offset can't be latest " + @@ -332,7 +338,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParams, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match { + params, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => throw new IllegalArgumentException("ending offset can't be earliest " + "for batch queries on Kafka") @@ -346,10 +352,10 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - validateGeneralOptions(caseInsensitiveParams) + validateGeneralOptions(params) // Don't want to throw an error, but at least log a warning. - if (caseInsensitiveParams.get(MAX_OFFSET_PER_TRIGGER.toLowerCase(Locale.ROOT)).isDefined) { + if (params.contains(MAX_OFFSET_PER_TRIGGER)) { logWarning("maxOffsetsPerTrigger option ignored in batch queries") } } @@ -375,7 +381,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister new WriteBuilder { private var inputSchema: StructType = _ private val topic = Option(options.get(TOPIC_OPTION_KEY)).map(_.trim) - private val producerParams = kafkaParamsForProducer(options.asScala.toMap) + private val producerParams = + kafkaParamsForProducer(CaseInsensitiveMap(options.asScala.toMap)) override def withInputDataSchema(schema: StructType): WriteBuilder = { this.inputSchema = schema @@ -486,10 +493,10 @@ private[kafka010] object KafkaSourceProvider extends Logging { private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions" - private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxOffsetsPerTrigger" - private[kafka010] val FETCH_OFFSET_NUM_RETRY = "fetchOffset.numRetries" - private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchOffset.retryIntervalMs" - private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaConsumer.pollTimeoutMs" + private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxoffsetspertrigger" + private[kafka010] val FETCH_OFFSET_NUM_RETRY = "fetchoffset.numretries" + private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchoffset.retryintervalms" + private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaconsumer.polltimeoutms" private val GROUP_ID_PREFIX = "groupidprefix" val TOPIC_OPTION_KEY = "topic" @@ -525,7 +532,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { private val deserClassName = classOf[ByteArrayDeserializer].getName def getKafkaOffsetRangeLimit( - params: Map[String, String], + params: CaseInsensitiveMap[String], offsetOptionKey: String, defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = { params.get(offsetOptionKey).map(_.trim) match { @@ -583,9 +590,8 @@ private[kafka010] object KafkaSourceProvider extends Logging { * Returns a unique batch consumer group (group.id), allowing the user to set the prefix of * the consumer group */ - private[kafka010] def batchUniqueGroupId(parameters: Map[String, String]): String = { - val groupIdPrefix = parameters - .getOrElse(GROUP_ID_PREFIX, "spark-kafka-relation") + private[kafka010] def batchUniqueGroupId(params: CaseInsensitiveMap[String]): String = { + val groupIdPrefix = params.getOrElse(GROUP_ID_PREFIX, "spark-kafka-relation") s"${groupIdPrefix}-${UUID.randomUUID}" } @@ -594,29 +600,27 @@ private[kafka010] object KafkaSourceProvider extends Logging { * the consumer group */ private def streamingUniqueGroupId( - parameters: Map[String, String], + params: CaseInsensitiveMap[String], metadataPath: String): String = { - val groupIdPrefix = parameters - .getOrElse(GROUP_ID_PREFIX, "spark-kafka-source") + val groupIdPrefix = params.getOrElse(GROUP_ID_PREFIX, "spark-kafka-source") s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}" } private[kafka010] def kafkaParamsForProducer( - parameters: Map[String, String]): ju.Map[String, Object] = { - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { + params: CaseInsensitiveMap[String]): ju.Map[String, Object] = { + if (params.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys " + "are serialized with ByteArraySerializer.") } - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) { + if (params.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + "value are serialized with ByteArraySerializer.") } - val specifiedKafkaParams = convertToSpecifiedParams(parameters) + val specifiedKafkaParams = convertToSpecifiedParams(params) KafkaConfigUpdater("executor", specifiedKafkaParams) .set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index bb9b3696fe8f6..609c43803b591 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -34,6 +34,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ @@ -1336,14 +1337,16 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { (ENDING_OFFSETS_OPTION_KEY, "laTest", LatestOffsetRangeLimit), (STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""", SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) { - val offset = getKafkaOffsetRangeLimit(Map(optionKey -> optionValue), optionKey, answer) + val offset = getKafkaOffsetRangeLimit( + CaseInsensitiveMap[String](Map(optionKey -> optionValue)), optionKey, answer) assert(offset === answer) } for ((optionKey, answer) <- Seq( (STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit), (ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) { - val offset = getKafkaOffsetRangeLimit(Map.empty, optionKey, answer) + val offset = getKafkaOffsetRangeLimit( + CaseInsensitiveMap[String](Map.empty), optionKey, answer) assert(offset === answer) } } From 1b416a0c77706ba352b72841d8b6ca3f459593fa Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 15 Aug 2019 17:21:13 +0800 Subject: [PATCH 071/149] [SPARK-27592][SQL] Set the bucketed data source table SerDe correctly ## What changes were proposed in this pull request? Hive using incorrect **InputFormat**(`org.apache.hadoop.mapred.SequenceFileInputFormat`) to read Spark's **Parquet** bucketed data source table. Spark side: ```sql spark-sql> CREATE TABLE t (c1 INT, c2 INT) USING parquet CLUSTERED BY (c1) SORTED BY (c1) INTO 2 BUCKETS; 2019-04-29 17:52:05 WARN HiveExternalCatalog:66 - Persisting bucketed data source table `default`.`t` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. spark-sql> DESC FORMATTED t; c1 int NULL c2 int NULL # Detailed Table Information Database default Table t Owner yumwang Created Time Mon Apr 29 17:52:05 CST 2019 Last Access Thu Jan 01 08:00:00 CST 1970 Created By Spark 2.4.0 Type MANAGED Provider parquet Num Buckets 2 Bucket Columns [`c1`] Sort Columns [`c1`] Table Properties [transient_lastDdlTime=1556531525] Location file:/user/hive/warehouse/t Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.SequenceFileInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat Storage Properties [serialization.format=1] ``` Hive side: ```sql hive> DESC FORMATTED t; OK # col_name data_type comment c1 int c2 int # Detailed Table Information Database: default Owner: root CreateTime: Wed May 08 03:38:46 GMT-07:00 2019 LastAccessTime: UNKNOWN Retention: 0 Location: file:/user/hive/warehouse/t Table Type: MANAGED_TABLE Table Parameters: bucketing_version spark spark.sql.create.version 3.0.0-SNAPSHOT spark.sql.sources.provider parquet spark.sql.sources.schema.bucketCol.0 c1 spark.sql.sources.schema.numBucketCols 1 spark.sql.sources.schema.numBuckets 2 spark.sql.sources.schema.numParts 1 spark.sql.sources.schema.numSortCols 1 spark.sql.sources.schema.part.0 {\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]} spark.sql.sources.schema.sortCol.0 c1 transient_lastDdlTime 1557311926 # Storage Information SerDe Library: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat Compressed: No Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: path file:/user/hive/warehouse/t serialization.format 1 ``` So it's non-bucketed table at Hive side. This pr set the `SerDe` correctly so Hive can read these tables. Related code: https://github.com/apache/spark/blob/33f3c48cac087e079b9c7e342c2e58b16eaaa681/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L976-L990 https://github.com/apache/spark/blob/f9776e389215255dc61efaa2eddd92a1fa754b48/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala#L444-L459 ## How was this patch tested? unit tests Closes #24486 from wangyum/SPARK-27592. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../spark/sql/hive/HiveExternalCatalog.scala | 7 ++-- .../sql/hive/HiveMetastoreCatalogSuite.scala | 38 ++++++++++++++++++- 2 files changed, 41 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index d4df35c8ec69c..03874d005a6e6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -363,11 +363,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat (None, message) // our bucketing is un-compatible with hive(different hash function) - case _ if table.bucketSpec.nonEmpty => + case Some(serde) if table.bucketSpec.nonEmpty => val message = s"Persisting bucketed data source table $qualifiedTableName into " + - "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " - (None, message) + "Hive metastore in Spark SQL specific format, which is NOT compatible with " + + "Hive bucketed table. But Hive can read this table as a non-bucketed table." + (Some(newHiveCompatibleMetastoreTable(serde)), message) case Some(serde) => val message = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index deb0a1085714e..007694543df15 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} import org.apache.spark.sql.types._ @@ -284,4 +284,40 @@ class DataSourceWithHiveMetastoreCatalogSuite } } + + test("SPARK-27592 set the bucketed data source table SerDe correctly") { + val provider = "parquet" + withTable("t") { + spark.sql( + s""" + |CREATE TABLE t + |USING $provider + |CLUSTERED BY (c1) + |SORTED BY (c1) + |INTO 2 BUCKETS + |AS SELECT 1 AS c1, 2 AS c2 + """.stripMargin) + + val metadata = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default"))) + + val hiveSerDe = HiveSerDe.sourceToSerDe(provider).get + assert(metadata.storage.serde === hiveSerDe.serde) + assert(metadata.storage.inputFormat === hiveSerDe.inputFormat) + assert(metadata.storage.outputFormat === hiveSerDe.outputFormat) + + // It's a bucketed table at Spark side + assert(sql("DESC FORMATTED t").collect().containsSlice( + Seq(Row("Num Buckets", "2", ""), Row("Bucket Columns", "[`c1`]", "")) + )) + checkAnswer(table("t"), Row(1, 2)) + + // It's not a bucketed table at Hive side + val hiveSide = sparkSession.metadataHive.runSqlHive("DESC FORMATTED t") + assert(hiveSide.contains("Num Buckets: \t-1 \t ")) + assert(hiveSide.contains("Bucket Columns: \t[] \t ")) + assert(hiveSide.contains("\tspark.sql.sources.schema.numBuckets\t2 ")) + assert(hiveSide.contains("\tspark.sql.sources.schema.bucketCol.0\tc1 ")) + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === Seq("1\t2")) + } + } } From 3f35440304866dde2faec948f87ef83ed0b1225a Mon Sep 17 00:00:00 2001 From: Unknown Date: Thu, 15 Aug 2019 08:52:23 -0500 Subject: [PATCH 072/149] [SPARK-28543][DOCS][WEBUI] Document Spark Jobs page ## What changes were proposed in this pull request? New documentation to explain in detail Web UI Jobs page and link it to monitoring page. New images are included to better explanation ![image](https://user-images.githubusercontent.com/12819544/62898145-2741bc00-bd55-11e9-89f7-175a4fd81009.png) ![image](https://user-images.githubusercontent.com/12819544/62898187-39235f00-bd55-11e9-9f03-a4d179e197fe.png) ## How was this patch tested? This pull request contains only documentation. I have generated it using "jekyll build" to ensure that it's ok Closes #25424 from planga82/feature/SPARK-28543_ImproveWebUIDocs. Lead-authored-by: Unknown Co-authored-by: Pablo Signed-off-by: Sean Owen --- docs/img/AllJobsPageDetail1.png | Bin 0 -> 239175 bytes docs/img/AllJobsPageDetail2.png | Bin 0 -> 70557 bytes docs/img/AllJobsPageDetail3.png | Bin 0 -> 94804 bytes docs/img/JobPageDetail1.png | Bin 0 -> 73262 bytes docs/img/JobPageDetail2.png | Bin 0 -> 25120 bytes docs/img/JobPageDetail3.png | Bin 0 -> 48152 bytes docs/web-ui.md | 59 ++++++++++++++++++++++++++++++++ 7 files changed, 59 insertions(+) create mode 100644 docs/img/AllJobsPageDetail1.png create mode 100644 docs/img/AllJobsPageDetail2.png create mode 100644 docs/img/AllJobsPageDetail3.png create mode 100644 docs/img/JobPageDetail1.png create mode 100644 docs/img/JobPageDetail2.png create mode 100644 docs/img/JobPageDetail3.png diff --git a/docs/img/AllJobsPageDetail1.png b/docs/img/AllJobsPageDetail1.png new file mode 100644 index 0000000000000000000000000000000000000000..c76d2fdefb1aa8613a06c61044895c7892254f3b GIT binary patch literal 239175 zcmeFYQ+%b*mOmVJY#SZhw(T9eV<(**qoWl67HZW~;VMefNbvaZU|?WKvN95CU|^79Hlqa}*^2IlMxRCA3-|4hNu9rSQok9B_}cE1G6GqfmQVf|8?ZeF;{ z%9?0^1z7`2{zC}SRM)gvC`Ey4o45O7XYb75r=@4k>vvar-b8Src#q^0kYuN*K$G=J z=41FkB@ON1(O|(BgSe;ov-)wzOax>E0d~7W_1i%c)4J(jE(8tvdFaaM(wKk(AF{pr ze$U7T-Rk;7!Dj?l}`qqs8MaaK>5)5sv zyO+j8--YM$jK4Ks1e)qP$*t-LlQP&D5*xp9P zrB!)oAMOY`?|SzxUfDv4p>wGh9p@$gYZf>s8J+K7U`${CHNZWJgx&svFs`zSk}!L) z@F=`+{uBRYP&`*jZC7zedwX*SS1@rGa}!r{Gg5bJS1VF!Sw$7C0Aw66Fj6pC2~iCX z@IPHki4@Y!hevt9LK=grX)viD{Oh*19p19t(@(uU=~6Bv$s7d+oR5?1XkhID7fkU< zd~9ru`|jiwy7L;4jPN3Ek8l*_>QWwW`o4Z|PtIlbdVMe$#t;fW;8?(^dZAK#VI)O9 za@EGe&Honz!ytvI+-W8J_YAOqP2V2)+KYcNFwU;T`j6Gx1I~E={8p&6h%fJGXZMad z{#)jM6ZqG1J^5`cq)?6esI}F~-C)-LBK+^q@&;%fs_FC!9`!4y{ulCJeAP}s8O}JZ zWvVC<{DWF*@AtoKWfyEy{}1_0XSsnVSC+{=X^O!Rp5{_Rp2+v;7D8D|ziZ)G~6*yXPRS*+13;|8h``v9t6K z4wn7?V$H&_JR$!8Tb2IO0g}I^|3eFF%73wkBeasZe}GGHwV)2AU?cYbRf_TJUu^9| zU>B3^f5B}&@I|Mk@GIMY@-6%qTg#Eo6#ECLPyN?Wd}w|X|05nt!1-Uy;q^95edHhD z8I<;b0d;>(+CTLE{ui_NCw7Ma2k4XqX4`ru@x=KLQ#9xN#m=X`Dv|vIti~`#a?gV8 zvG|9+9{ytgzfxkE{o2{(^*^=vS3Oj{Zms8&BuYg8kj8=huO6xB-iZH!HD~-w?Ejni zcf0+cdj8|>|Fq^@I{j2 zeF;_W$2`vV?SE3n6E2)973Z6^-pgF082X*W_Erjt7v-o}!d39cFwSw-#pOayv$s}% z8dr{kp^@+EyZ1f#`^``+_taHNbvJL}g_>4P^<&J}`i0$~o?<+r zxExeW@FbmNF5B0^JkIh4QD1I-eVcC+p{oIe6b(b*$lA?U<`&`l6w%(9NodX&cHO2+ zsN5!_k0=}yO9`Qx0T}YFHq))qIY#DJN7P%7OJPQ90IfGNWBh8Y>n%pot8F6481*aAx$=Uo*& zk#za_&9DdgxL7>qF{1$hsJMhR9w(^y0F-Sj-XTSgtc*!k6qrh68H)fg1*$2@!Ih&~ zs{~hyS$K8NtZ{nz&S~0-w~8&#ZjJS!P^1V2nAL+`q1e(KffEU<~!|v{-1pp24p0>7PbU&4rW>2CJ{lPuj6_dDb{L} zLAwGBhq74Min?5$3;V!GloK`AIMGVV6DrYm(Oy}W_8&H9s+Zpd_?czh#mtVr- zU-2H-1EuCl0Lb84zk8;zsVhi%^f;9lPic||FtOVf)Xbs=fl>}SDiOe z>{z|!Jr34Z>X`MAr3waVH6*O>Nn*3Xz2YI9(TB=sRLqf;GP5pNSOowsU2IFyXIBG1 zoTV+XfdfT@W_wyFVghx-LXzLMbZzBLHjJ*4Y;Mk@6@4nn?0U+=+(_s-mUUa4pFqbxOErxrQ*?N80iMpsAArV+Lw%|JEvinuI2 zW|=CK&(qVg*uQ+2MV&|X-@aR01vP@G70HIZ(il!Pbi;tmlzcYiNs|qK-H29#gqcFy zEr8zReKnMV(&Odi|Q0a`7iik0k4%@~w61l(9F$_#6<3zEIFA zG@kCfX*v@?qsR7aVAD!($-}d{fF#SleB;Damev$)xS*+tAI@|jWr#Okzwx~hQht-W z5L^ORQwSHd`v`PKnX63x(^l8hQQ|Z6xkOX?DN>uk7p{y=|L&8Qq28ei0x-v)&(7P5 z13XOzG?S$lo%GTeGim-^_AV(^uWR#qg-JCfNamTVu(f(w< z@Zo(TAx8FhzudlsdAuZC-*|L~(VbEeI4-3Jryk2!xCNvNXPIp&)t7)DDrcC}Y6AG_ zB{bz(Rad|mwD!9P8Tn#+CkhIF_Kl*c$G<(G2Oq*FA?gD=yCvKjyBpK z6gc#>BVHv4Ej4Kg)*03Nh>I^XK?3jOTp!PC9byWUzTGuZV|SyY>*bxIR7Bqn7pfWp zj9{Vhk#T%33|g{yZ{dm~IM=CDPkhGzJJMD|8G{!LfuFJAmNZb0-ko^y8VrtFpS=Ut zQ6$&Zy}PntLSbHqL;0nLg9?KBHdP2(6V|Fw;MDC-=giTdD`%EXFOi|eX$mTA!X3Y13y4-J zCs=S?_*@@JrZ}%KRZav#)02`S2o4g>_zaEV`?qec3DXGi$E9rHjq4K7H7x~O3%q*y z2{pyQTOY{I1P~v&&Cc4VJRQM5QxSU z1nPi|aMZjP206vr?i^65+HtonVCBgdQ=w>_s3bjV?9is>7Ufd7;5b9Y?=PD3Po1gG zWaB96a{y=;(L>U0QP(79q}65;3ymsFO>FWQPpZsR_IcnH@=L26in2r7G-^DL!<&t! z4pqyP$BK0&b~~(jG$5pPH@E{w^a_2utn}1*w5ezCpjPl(1cx&Ip7HjfOM?F$)0@-4 z4w=v}xGPRMCKIqSx248UR_2C=%)0n!^z=M@>xdg)^8Tz}uYbhYl6alwQJ-&PASEC! zGSKuEi$Jxq7(qXqg-i3svWCDPNUDn0el+Sq!FPRGD5gZVFg}=KIA9lFU7(O|1X*EL z2Gg8z)ZjSZGhi;1<0mmzI;XH_Y173onHu!Q;n16!M@TBCH!o9*U5C_KX=1|FAr9RT zcC0iN!E|W~%DnsV-6z#)57UqR$fXsdjC$AUJd!HHY(ZL7v(lCmt=oNCCjM6(p=Rcx z{O>rD1!YAxK$R^+zndSE&ymJBcdg@dZN-@t**eC_lS%X;xA{ya);;bSF@*$rUR)2Q>tTBw>sonZJhes9ZWG<8 zGMQHx76jhXX}KL=G9?mG+wszkDvhn-j15MSk{Fq04555e^W?)X=kgG9^TgVDI}_VN zMw79h@a~lw#)}$RtuD)Zy zLOT%H5+KGkYJCyZ;%(lCwBl)NLS-wYJL`;6d};~jaZHAV&6Jm-uQC3CK^_Q2ZyPVG zwldZ(gAO#D=Q53PB?ewuvs;O2CYc#DFj#K4Oen1m8W!_53~H(g8-(r-qKCSi53RgM z0@Xq#t&`d-?G?&HIgVu`Iw;TT>Ao>qH1bik8`}yPwK7@K6{g7+d^yutn zb{gugWRP6VWwrOf&StfHr!C4$vi(}SDf!Q}g9xR}Jcm;{`f_u9GUWBg@lzeTY`7Cd zJqyu3S~>cPrn2^!>o!M$W4vLH)>9&RL6)L!I$jss=IB5ba(vS+S-?WEHJ~uUknYvR zS2e5w9;1t;wTXPV9!}K~n7Jgn!UM47>Q5O(hKa=YY@%f;(<$HL&VXzvtrT(oW34QE zsW~d~vzZ6y;62R3C($RK#;d+!q^wx8J#b}xykq5;-OK*$ApS+-xJ&m=3)PRsj3eaD zbo|e47`Sz*y!PntZ^q=$11WH%vkn-MY^ZSk{fd!hctzI&Wzu3gVeL%#!RtW5qYo4R zM~`9QO-^?&5K=~ZF*yrC7R)LfL2fe8;83zn?yLcLdz9;!X~J5DdB(W1qfx}!G|Nu3 z<+04}cVzo00Gv{N0sSctI|fO@rN$-IhG4I+t!myx9!}HG>))S!LEBz$WlVU$OD^YX zZxI1(+n)U`_ z!$aomOoKfZrs=O-B1DLe3D4SiGqE@1NovrKD~La2eS~SYsqFLjG>v zP;tUonZMNn1d%jx7Hl*m5;>Hu{OL?gDNEGSyL;!$4(_rxh!|v>&a=Nn2lJ}Qlug3I z@aumiYjCF9CvNUyOM=u|xL^ELU7W0@bGp7TkbUjBHOO#Eq%~v%f>iG-(PjuBvG;4S z)~dy#L$n4}i)@u)LyV1De;MY7>)SWZ{h9`%4g*6BpjyqhMi`8$FkzH{&zANGdd&)O za!ydCcdOQ+A8v$c#vkYP+f1K!LgyfCMvqJcKp}CfL`u-hcMW>M1{7iV*g^m8{prOc zWrm$oi6B$ht1oFIgRqRAB5C8R0h=6^%#_4P6uMG~N#>P|a~e8YSb(I_r4SUVtH9pk z?OguQ?-~YJPdzqHu35}UTzp&x4~M)-G(YL_5U{ayf|!4jt1tTs`3XT_ViZaW#NM~V zT6nUIEk$zB4`{-hsdGrpAXDzFfOzljHo94lzZn9K8~OV!7+Pd$no!^+Qo=7AdwA7+ zTE@yX0ThZUbvXdjw4!ew)gMJQ#Jo0Jt(Zf!*HnT zQ&y=2y2`rx-O(|t7TB6!49~!E4Y63Ha2U^T2ulx zp=~7B_-7R6Fmpf2Y5l9K8NcnH){6wig+zugVrQ0kB%4{ac)~Y1M=l*q#qgHE((pqQ zl^@%|OTU$Zpvp`(!WdO%k4JYNZc}Y*TKQhuVyz?QJ>%=uj%AwVCBNS!oeJB}y}5Sj9I;1d z*H+JVjrdaojk#h)rC|uFWTiIhGeQUOnb}{O>c3_TbYE_5xL=aM&8d(=x4!z|ohiQ% z-X;PlZyJ@fBsn4%=Tz%qBkppO^i6n{4J)uZ5_WZqXBrLy=5}||Ft5+c55LyrrlGoN zsUj_0p0uz0L0Kz5DV}8&hNqN7f~DofT{iA-0j#|2$k=6>qXvyOt`(Be5~SXZfqv>) zHWfJa(~xU-+tBP=g@%5ie1w>zl8v*uif=$#xlt@QK062?E?eN3JJ2Y4VvE<+R9`2iAl&p6Ny=kjFxoa<5icuTn^C#qOnb~;S>xs*0r*j_3B z;#}?@uZ_}W^12{+*S)G_B&~2COMNPS>Gw_1k#bsEsC>l&vXE zf+=C~YD)Zb+lr&VH;NuDt?$+{K_GLJ0SUzq6qcD@B+CehJidOm^dU6USEd{Cvv^R@ zQihYBC+S&tQAT&!9b=gztTYw-(!xHKOS!l4c^QQr?xjLG84bLLz?9^jOom#F5}e~l zEy|a}%{ne}#th(0uxF4{kLt?da%7TUILvQbjy;F61Z9++*-e8DO#^0wI`0Uqa!~jW z*J7O)3!%F%Dpp8E*&-@&hNK^VT+`B0_?@$APJh|0PUzEJR7dK*fm<9&|J+BBAINJQ zSgG%e8Q>gqe8f4&MKar#7V|gcd~v3ChLQl=!+uSsev6$p`s zYQD4se4I<@c9awdaNQ)&_XcivV@SX6OoaF$BO6}fOZZD)7!U~!ii#*-=fih-JnSB2 zSQ4z>W%|jA%323Q&p|)?_i>2YzKx`%BjRuDy}$`YEy?i9AzJfPs@auSDg?T|QM{FW zetW3$Sj8$A?$#*9e9h$o*70K+A?vaGduSX`ZU>_7JhNOpeUqV-;~0XH-6J=&xMom> zj6c#=my<`3+gY$Msy7hl)CZi4vX3q!=Vg@pNEWI3<J(6D*X2B8JR@*K{Ya2F0CiV&b_PXm zpm+lZbVua$5sBG{OTJ??*nQfaaD?s(m)BSL)@%wMLC!QSX~S8i5TQeZcG+lowD0#P z(yz?;-+O6L1G^b+ppeBYFbxa>#%K$_2!bm>Xgz)e-5)=<@PSL)KJhX8&CUC2U3M%J zjHqBmee8sN$Ktp@?Ic<5$>4{^{#kZ`QpSdLp%8WstWsGBLyXsiK+}~Eek!SPhiUaQ zether!Wm>08a?Hrw{;cpBupArE6k%QBFxF_Aw@cRMbB@yE~d8?l>KFKt8~NxVtUYI zhcAMDsVa|Pi;QF$vsC5PT`FkBqdcc5;Bn$GE^IrjTGTC& z)jo=G=W{iy&S1qu!XJFTT*e|+882cBgA__0ZhL_ZKr;uXaw@I%7p9!;3*EFFy%>Lg z7FT?NDcqR``RM*Sv%-=vkw1MX9uYDioRU)HbLhMtS^e;H-sp}dW(@1l!`cMzo?#OR~ zGxYmJzk!X($j&e&KT)6-V6SX4?nQ8)-8OJnrk628-PwO*Bb>TpHEM_qjt3N@!yI- zAkNN_LlO+b4D}s5IrZ`bxf-qJ?gsy>9OCt*rVjot)9Gz?4T;L|R~ckGU`$_O`Dk$w zl8RaIo-TjrD+g%OWM*nPagl5Rs>*rk`XYTEX6_r z`AoWYZ<>RW#KUI`HbPm-Qn%MbK3oP2fzNE2%>ACI44M^_A}#4kgAjnr;n=SMJW-+V zUD=l%)t6rHea4o;Tcblt1%+W@+Q}|DG)XIb>6SLr-eIG`lgD1rPuB?7+(KU; ziNJ}l_9m#0|2>$d{4}XpYyA29C+1*qLGr6w+YU^QoK65O@>k-h>|OFK>AB0lX4=Yr z^L&$tTiV_yV&B5q<@R{f+5&X9?`d$_5nQo~#lFPz?&d}rYrx6F9=21g{bVJ`Eeq0I z3MsCz0-C`b>gzVhL{A-b-j%3TZ`K{JZ%$3e$K`;T5k{%iqU=+cr#}(8@v7dlp8hD( zz$vcs;G+5pSeT|?3CYA*qiYZ`fC2TZXJ8(2s}Q)Q}%hbM%wqkDHf z0ET4|);8U?R)Xhwj&HG$?=clfigpW<5En(9*fms0&BeLcbMrF0yv#FDRd%>kg*QcocV_e_$8z`@rdG*-btEgOAUF& zg?0@-iqQy6xfo0|agOJY@V00P`=5~aF8QNC5J*Z>=UP0}mldVtXyQZ?^mz5g2P|@8 zvUHk|V_l!kynSvv^HQAt0D^Tmws7>uj@&VgA^Lr?3?{O^-ykq&zi*cbXQDVidKg10 z$gzEaOLZ&Oqx5f+ug}pAUrnf7!)%;fO61}nK?`B5IHPdfQ!qq!4xR=v`!}C~h*z>;uu_0@Da?#D=WG)@wRMG{pji zxd>kfexFH<50u~7)CoJS8z{Hb@l&j$T^}pCcoXd>?=J)LR^PnKG{fg3?{=_AdcXoE zdlA($gTdM<^WP5bv*zs)2Qb}g0((W%mN-q$+P}>_8M1HdDr(N9++MsVL@#)uI;9F2 zz~KFMyanCo44ESn5`|H1d`>`lWN+MxaasMx zFYn>z!{K(j^<6PqMkFGH=mHa#muqN4PU%)w2m(UGQ$|02ObsiTgNv}DXHRYlRJBM< zm@3HsGjEQZu*FRT3QV>(uLm&_1fQMa!!7e~oa&~4HB(rN1S>v3|qYX_X_ zRx9uEnGZ`qPSD>Fqk(QgFFQb1EwA~ll&ol~zkv}yYMv{RUE z%;1buP^{6!j@huZEjXUPU_t zi+=Au)vr6)GWZq+#2^lUq%wrG$scj)iyjV!eqPlrne{&U#t?(nNEx+VKC==KvuF@F zmg$)>e-fSU&s#}@iloPmCh9zYK*$2Uli)gO0Xe2{~GdKwQ%NJ(;^0hdkg)2K=t&yJ4>N16fpB9~;ikI7u}_ zH^b(bbY}>q!(A)Z=xrn0`|zhvCzJ{)tKMqcU{f3;bQO(NI-#l*R1|D&EGNzHEA2mD z;jv6TdUFs3XMf>6yaNb<+WW|cRFdcieRLU<4KtM<&DA~8YyP7}j;L+Y33}0PfqnfU zugxp&zXiA6Djnb2YqqZE4dg`&`BDc|Fa1hOO2DzAc53JIwB{%$;*5($x-cMmLe%Mg z8HAw)6JumL`_Xe zS(YT;mh4>*C8qkANu-X(5!um-6%0c57bHw{Ov?$!PuHM3bCN zQOho2tr0S7)@sHk%DRI($(3V6_zug_mE*|;8DCCG;Ayd2oB!sUy~v@Vwp6|XO}@*P z#1H}9RFI`sJ!@@IHj|yJ%oHPJ4-|7Z6~;0Igw>uZQfX1iX09H0S>AWFrU+EsF!K;P zUG-du)1`LQ)qQ_)DSkHX=|6PQ$JcnkWEO$x3-b+BlC0%JjF-5--8uwRk(nT(>U>W! zhFLorFHWuzI@Y%3s$J3e&>=AT&A+vDHh8zQ#+$XBSjhVZ3Z_rHf=)jtY{nb(3LA~I zi7)zs^3ud(8Bb_pHbV<+^O<#7P2SY&*qrd=Z~kiMW9!%KddpE?oz2Q`BHW)}%R~cr zNQ2{C=w9R*=K2jprbl(>jDd_3Mey2?WE-`=Q91xBK710NCvL${@fbb);2#GWXv=g+ zEoF>X!Js5BgX9L5%xLMvNa=z|YD>$4XP+{x$+{MGE3sj6HHKfI9p5v4k@s4VI(3Xz z6l`G2FVpy>oFP^QIC@|&`VRP3i7HsbLDKRRk}tYeVxDouf9D;b%fLJ%EuQz}=?KLa zdnsRDQxhjuonr{z;PK*!C0{|F<)X1bxL;Yf&GaKjjZyjq1`|0_7n31>oq@&mu_&zo zmITIzjKZ)1I#38IDBB=d2AGpFF*9HtFQN`&6ZnZY?vRd#T0%iJgXF`TolmF}dAr`# zy4v5~zWy3+X&!>IVghh@7^CPhcnpA$8a1#k$>)(CDuy|F%yWmQjahoVz9_({qLqlf zT?j(rZKKlAkJZye4=`vS?&bR6b8b=W%5G+p|Kw4MF-RpcKxCZx*UeU5%{z>d?{=xT z_SeV_WhA_gCR6zPVd(YZYJ0Ox!~?0=$y}v|Ql0^G<X9k=F7&TJy?x@y?T~Srk3} zK)_H#Ri%N&&G89%vh4oleatOk~#Bc7K5GF4klbLe#88W3F%X^uDGAauVb64V`d69twAih?3N{8R>9g(eldfmZLsA%#x zWZ1L1GW0QsUuGNPeIcxCY(b*ld4t~=X>cM91viW`tQmA%=sP0T#bX=O4xee+!>1;o znp1m=3;K=bp*&LhL74&b-^R(kZQmN*l9tN_6D+<*$$OjL`o)V6x-njWC%HG)IC(iH+}x zkZCK0LUp_JEVz0JP*?8$g*_n~*3DeXqyeG`$82jiul~f|5f)*!V#8{%M#gcZu^upz8M%vRX)lA)CQH z#_B5&+)hmwRsiFilXhiP?=+BRRx(W?(WyOEQJKWR=k@gxr%BcKa2_)13R=LHrkhk@ zvAG6WVC6<(xZv0!d+wHSs*Y&5k!V9|P)s~vRo$8iONk51t=<}SlRDI2G(tMv*uYli zS}xy;h}tChWQ|U4rjBFZP}&eIL?hJ2jjY_6uVwZm2thdY>`asGON6UcA5&>>bPP^e zHy~|~**^+u`ey-a9p(^cW?n}MioZEc2g;vzA{v|g5z0}v)-hT1N?b|Jp$Z$$@d4UE z14*4GyCie^feF}et@5F5vGFtZTXPu?lP$g*%9lRVc8zi|#F%xMb3z3Cgm_`2rqKN0 z3R;-ncwOMUaJ|WILTy258h>bYBxUR1BZkOx7|&Gag=7c6-N`J^8OFnl6D+?3SpIp8 z+Uvfq$o5+bf>y#$O+a9M(9~&U{hmmK+*6piclk?GX@HF_d`?u)XF(A#W5%{V#@#{I zwE%n?7lsa~nswq38NU159k6dL9k51YN^s)T8{A4+P|$wI-?3qN>x0kB1VJ?C-MJ^d z!~D8X^=A>d%KEm&b9L8U#amo_c2%LNp}`mtq(sbxUTM=fSQ0?nN+ujsIJsorJd)_y zm_20?+0by1c$qRo(~s^js);0RkA_O4pVVe`(}@?Y?wD@dAiMFX`%K2Ab=sz5ubCyi z{yM*uLvHk(*&?x>1Q)NU=O%CYk*d^y=9JowR=&9^XznyTQ#C+LuFFX-ZAGh7KFT73 zQO=l*N~y|9V)>1+D@@#)cc8Xvra)~V#U#3c|ME08eDZu9L6OG@OXMX*;`%%@ZR(}~ z>Qu8o-Fnt(za(53*KRxD+c$Qq4uV1ydmZ2j*;PS_&pS?Bhqe$HyFMPpfuu^5#lc+e zi%=3vi0*(7xL$MdQgY1BtY!E0TDIHP66tn~y=N|hMt=*2)6^xnG8e?fjTGN-n1dOp zm(EOZ7#}&oc^?-BdRSh~C?`nxjC!qbr}WsAKT*GOfGSlp(i(4)U1D|Nu8t=YZb4N8 z0Sk(fCSs^P0ofhXa!H`|a?`W5p~qjqiR6VZYLvfOV(lAH363LsS%clBhe9|b8jo=9 zSurcjlrlmm$xGGP;HiDFafU}ARFJ6^HGb}aC{J^RPZAX)eV3@HtR3|#TRt;{VX=!# zFs?y&oIjQaQL|O?T}S?6%Dt{4rlDh(p4qRP!|pk0+IE`wZXdt!go_J*?#m=Hgj}C_ z@ppp$O^7(B{j+FBm2o=wo4PcbIF|x*O z9bx#>p5$H3vxb>D()o-c1fJm6L!zxGPDb-M6v8q~Sncg>W#4Xdh*A%-QxWf3>bcbE;eDNl<+@RH++Nao@>s zGtc(bUBpBL%>sJ5E@P~B*K3%b_=%cTf8-UbuEh)QEhSYv*}@!0i7loxSJ2TpbkU=R|3sH6clx zjlC<&7+r^m>G$P$%?{Aoew-%V?fK{Qk?^3K@e&uE!HHn>{;O8&^S5>;U$O8K>J7zd zU%PBOzhFZQE}1w|Zwit%QR4~@!X(5$=c?4!ZH9m|Ydq$N{E6CoCe1AxAWpHAOxS+% zNt09IBCEpE?R%e+W9u2p7mKYVA3YvAohJXcWrkK(0)^>*YEJp|jg3D~ED`=0rbn;S z^=ez=fTmH>561`Q|L&`b7;mmFjecm&TF+t{5vCB8Blw)KeAP!_~&*FJBp z?D9d2>e@_`{T~cmW7SNPRf$xkH9+#0o{;x!kO0R;-q`!Dn8*8=fdgWUmI38D8;uf< zqA^;yR`>IsRkd%>7sH>y)mpz){;)~9W^8nIySy&>y}lj%B9Hi~qTBv6evJYAc+)Eq zb5=X`%I$Mz8Z|S%ccRwgyI{T~9dUa=HRYV{=tP9S4~p>M3!7L(&`tI0Gv&%Sp=kdx z^;4){m+UQQUytJMx$E-^L!UU9wY%tU`dlU8M6D?R=qay7ows(1-L3lKq|(&erS|b@ zI()c#hx3a|yBDosyS⁢Nf6?1}QmxPbL{~W|O~IOn5pg^a~JH01x}*{R)Hj#YhaX z^O&qAf=RFd;_5BI<3Ts!Ckesw9pySE7HWC#NSt1(xOX;aoLXj>D$hePi}rz}fU2N_ zFjJf>XI60r6Q>Eob^5*2m7h5WV*$LRFi`AS^r#PMS_#i};OTbpyAyK`;S$9N4__>6 zy%sRMWKgA^OaI#ayPcuo-)gSW+Y`FAL*1)nPhnKT3vO`A&knt-&F%3miNlqd=`zwG zEoR)Gi+S_q@(`bK!4Z;4(94lVs3Bc^aCkOpecENbk`Q+%s>bWZS8J__B_x6fxJCWy zBq`9o8$hj9){{`M=LQW+bWu)sg;3X}~na;m&A}aW{HZ1WI@(Fcbl%f=keFr2vqby4y z#|PIp)76T~L0p~(Kco)m+Lcq!ySasb-5{$~go4)K5FENeog37kl()7o(~wjWX1=;* zo2aM{{z`iwRVW8~k@V-bEby1f?@bLuimLqsNkBvvA6Su zNFOF68w-LSu`blk@K7@8s~l=&-Z4)pq^Of``JBO{=sOINYn=m)SDJ{;580}Ogup~{ zh*r|M*)i9I$v3Z+oxEvd^9Tom zVoL%hF{g6)iy}Em{3RXu91}Z}Xp+NL;dRK#sKEl|0SDgVg47rbV@L_S4gsC3{nK9s z^U6KeV*YQMOg*AXTxF#g4C#NbFGn+YZ0iHDFV%CrJ$Vzdp;C&u8>O0bu{s{oi>uo} zyQEz$Sa=`N>hF6-oeI3~x9|;0L2;E-*CH=@1dXnerTnxGH^a&wZd+i2U?CXx;KJH} zp2M_nQ;&hI702qoBwzZAOVkrBS%~%}}oX1hSqcG{PEHl&L zr*8?)(upz6X6^sJ4FxRK9m=BTe*l(~Cn~tg_|B z5V+Q2tq7s}t>fgrVXt604!^Q|;Gx2464^LM6R*b9x$KdD4B zV^%$9O)XvZ(W8YyHH;6-xd{aX&F3V@phx%6R08Q1dbtQB+^F#bhT-)6LhVA+q!wB@ zjhnk(x6pLqY4@1l8{!=$&tk-C1(JDe9o-@5PsA*a?ZTL;@GFA036w5A-M^W!59|Px zyb{(pC2%+(-D=eA@?Ts+RVy{;?~5Cp8z0jRG2Sm^Kv$aJ%k%+m%2R5ennZ;ZznQH2 z#yMrzz6qpGYgvy-3RA1pChgwf-Hbu9u`9lT2w{98)lSWFO#*i8A}ACF|NO#SOOQ)` zRH#h{!>`6f-24p_sNG}3DOcY)X92w%B24}_cRkNqvH)90VzYn&>?s0eN(wnumd46a zOsx`g%*x@&1%|LhhQ7FBn+B#2$lsq?jUq#Z8RRD*lk7@_;PH8u6?+}+G2!D5-<6OG zRtFr>zV>L#&A_wBnq3Io*8Tw=rUD-L+C|~;e1_3Fh&Cw<(N_Z3fkH)g(P${O`?sSk zu44*8{J`oe6*Nbr@iG00JBHy0fpHu)#*-+%TSQxs{gD{uTvlzhIaPlmhP^?b$w+y698bF0yn%4#+s?>8<5pSV$N}AmR}B z^oV+2M;0WWILgWFZLHypnzK!qlP#`@inLx9|g^xskYUUp>@9 zxxR4ngDFy-OupC{P<4d@fpda0ZwC=(t!9jkM}UgpNeSt?(B2Fjk5 zHvi<*)bD2E?W`s!QZ{{0$`1O~H6EItzBaV}6mjC!h}Q_;5YoHNwk2iJZM4{ILZkpc6_5B zd^R6^_Smu^y;w9TsZS1a?8<+NAG}~X9zQNFj+{vFObJj{tWcLd7Z|1@8ir^7nAV3W z2fnyc9-h(3!-oMT&etBhul~LxmvAg?4|_qBr+QP4lWR{~b8Hb|#F1R4DsTZt7|0e| zR!A%FIx63sz_S^t)jGq{f!zqi7PbG(rE~8tMW#Ct+1g+icAO!ofXs-JZOLhDVWM|@ zf@JeH0TP(0Tk1;@X>qY3b4LV?-7^xaARU9>;>O^ecy%KOJ~QA>^*{SW^I;wFN_0_J3@xspmN4`S z?jV`5pi6#phLu5tk{JeK9;aXWpFhe!Q-7W@&N_`C{PaGJX&){QD0u)M3YNP+yq$=w z;8wN1vVWoLesX}Bj7$$qqg`@^aY-_XknjMg_aaOl=ULB@3bO37qS_%BINC*8mKTR3 z#V}^=PJMf*w>2`FB`F;=r{3$Q%b~3J`=leb`{Oe9V+Tbx;Ed4NLMh}HzJEF#SVrV? z^4SGEUgy0aA^v8@T-Jp%pmg}3e*xHBhUCgEfC5Oxdn+p|C+^f!e{XoG#ywS_@g8b& zow+#2VVxBUzCJ|`bRO7z2zI$|T~NPz4|R1`o7~p3h>|ZccE*gjPyI<|==wQH@+;&e z3NlM?y-*a5vHqSzfATQ#dEbtUFGuND!{3YQd@Ev{TTG1fD|eM*SVMh2sWqFg|3;mp>S7tFZ2 zXFYvtQM{Ofp~Qoy(WjM_L}A8{gU9O}MqLLT0(Z@1sgI7o81UrUF1IXYn!Hi~nhR~bttLuT^B)nyW?mxH;rg`zB9MRT_Un6B){ZLI z$f^nDxpAounO`gAlEqnEq2?%s`wBxR$6zI|$$igs{E?@!YLWz>U{M~T8Dorpf~)gMMi#SI2G~B{Uv<2dRplkh zLrYBrjZT1peJa=Qq^^`9`!HFJc4#x%2I{hUY7in zIPvA+h!ocBccWc@V16mnAUA;_#e)ed5B_ntkNmoN+xZM*kUm-iiSk<}*P2W+qLe(f zcKF%43Fsnub3@J zMcoOsclNf;=Rb(!M$;e(!DfFs?H-=o=_Yb1JH<&brvV>K&8}X z-MNP-$jLv0_v|&kClWB`P>|DK`B^jOvgE(5E5psHUb?aj1lE0A%(Qr5v^am*@MW*O z93coIzfs7bGh*I|!6OY~OiC?y31mb~D(7SgzCN5R^$5Rz_fCjN)bf8Hqi;fgG`|(` z0gz9+nd4ri29PyOf*f}AnhIL3()wx2_nyX3J|Ca~-*Ag&`C=m|6#6AGe=qh?XvcvF zVvF}~N6lRMGJM$6G>*PqPi$rHyzq~tZiV%Ue$u=XcFUwHEY@*A$HaP~J>F(#cq^Tz z63?@0MBY&O@zRhr&045&TGu zaE9X~-VZ(R?xP%0-!?c7?NKvt^p9MG)Fo;0$JM0vkg|%zt~pe;i$%|H&NF+pIMrqU zPTrdtT5&H(?8y|?6L@{B=*$+Bel9}o!Oro$uE^R8?FeFa%X$^xtFF1Xi!hDedwy(w zhjVxz5?y#t;x_cT6Pn7!_BZT}`N?;qM}HcfO`na#<6Jy> z?d_lxW#WGIWz)!zLfB^-I}gBlE$uaZxAE-~8cHG;RWua1_UjM&9a+T7q%JUt8zwY? zx(HVb5T2e-P?pCSA&pPxvnP{jaXJ>?zy4gWelisO2g<_&5_3NnF&r4nRH_rX#;yff z+D8<3%eOHrqFZQzJ9`+Fp;u3FtFLfwbVuC|RAi{^R18#o_ii_srX^C?MHFt_G;NH@ zBgi<)xc50|2!hFHScr=X4!dNBD`0)4aRs!bgmy`Xr$~$Scr@(Ot?8Un?L z4H)PFv5o#}qv19;L*sEA7^J{`Qian0pqiccrL(=iM^)XHV^bB$OpRg+y#c?7ZYd!d z8!(V0n-j_s!Gek6pT3 zt(Cn-s!@As;6YKiDLMFy0SJmPJ?#>CeI$a{7`&r=W9MLl%zK)vSwrv6?!&KV7ibCa zx$R`NQe-g_>e|j7vmCUcoloTy(+$$J2#WQ&u{tS((PBP68J(S;K*qJ$6mj;FG(e2x!+r{*&7m)8Hk{Jn8;2}MoAlaN4}t~j zO0uv{r)31w#bUNtPNs|L1T(vfWv~jCt0hETQm<+^#g-NQdbk#gPG!H6z!xtJ;S5x> zHLS!IL7BAS6e>I`(CE*0!~MPB;9$9TI2@|OM(rB{VJf}N%Hr20ks2}>^bu*s)AT#K zC9Ga_&r43$E4rp4bfc#<_tR`gNUjGXV3SJmtX@^6tU~ig)A3?5onEYGXWC>WW~nMH z&Ccrtds-v(lG8gfjGl(GUk&tMmy8!T_zur7N0Gn{uFYZNggBmDoXswt&dyI3&rb9C z%5D0s2v9<(#G}j-zANLuUbV|ueDUJ!YWiT zM2mJaG&}vl&dy+O4^~8fXV2>^vZc@ffa=K0=eXtP8w^ly`sP7XiR5^VV7r*yg#Q~1 zv2~V1VU}pYNpk@X4g22tpr2!Y38URO$BwC3v(;t|0%2oMaF{nRpWMJAYrr2)laW(s zAkahr+j|_{L@H@G)PYjMe)cfp!dxn0MoB{OES#I}hddSGJcRwka7mK0-BsV~!tar0 z{^=!wpR(}3HDuEDB=w<8u+4&LDF$XXF$KSwo7W~qn@7z>EX#>g zsr!vl$Vy8dR&RN2R7EP3j-p$rlvo0b4I!W%Nj^d~B2}Fc8|Ur*rIwA}rPr@sh)O!@ z6RNn@qRFSlgRYLQc6k53m}!zO^?K4yKW$}fGNe|@&cV)+BZh6@6c;LdlG}Q+BIVYM zp5~L&)oMP8l8dz!x`!C{?BnD}30(mS5+}p72Tk1219gjZ<{BK-B#6hW4Es$yjR*Vj zXAk$`8Kc;$MskKkFi!BpGOoeQUAC-(lWh4koL)>ui^nJ7wCFdD3c^?f2p>CNGY)i|z8@Vu2v7bUGeJdv$LKlxa(yr! z4`s#mZK%gwB(r)EWEA^6nNOCdxyfs$iQaNoGI#?DX6qL7FF&Vl0$Fng}h z?DM8^+_aCBxA>p_dG7;0aCIbd^ncSdyjV;}&z{Y`{qySNyKEZt0(dhYihEX=MKiOh z4FJ0H+w|^N_VMQr?;QX7!OoX=`d=KlheOMQy$7toa+`&hH?+1~Lk+!!9z50M=<_Kc-L--haCt$RDfDChV&hNPW_ zQ%gn1GAp=Ih$zfw;Y_N}jAER`i;KnT$z=UF5a;qycu<$Ed`#ZMT{S`ZRdj*Y0bWyR zAjpVTyVQC85Z-=f04?7bWwB1Y2h6-FY(@N(`>Nv7?t9-$5GkatZ5 znvE8&&5mK4wpkkGObxK(q8i9*9Grbq;AXp#eKz{>n;H=S z)=52&Fwx(}bpyL^Ez30~F)8o>+vpS|@bi3ij)r7iC%Vuu;w*n# z;Yl4)U@I!GuhkE4bhkJ+S?_3=0^2*(6-~TajyN1*BM5(2^c-839o3gJ8G*J9DVK{t z#A+o+F?yEiJpJxz{MSe6_vcxor~5lOHiF$%dU6Z5C0qHV>h;G|4~G`Lt# z9xq3aC(p*;e;fbHAM*eEuW2o$qkC2#8V5r$$yd|?>T_?X>|v%Ef|Bf$SMlwT9;6X% z8YV*!^N(EEGpxmJi!~y9#G@rVc{K6co&T-ak^NnBXQlVxv5&5@uS!0t-5c`g5ziJZdY7+X`Y|m; zg+tS-j^QpjS}jJi>BY(E^y@z@zy9a-S}5k1s@k`AY`0qXZi1+9Fjv5Y58>hx)Y zO=Yyh%J@%#Ry4i(*AJTn^Paw94-5@Ki?L-h!WVUJTQAc17SFKc-6W$vRF^0eDOK7C zYxUPjJP*(7#W&jOQIyTbm2=t-o|$)+_hrVZ}!0<5Wl@ z#Y*#ZUM-#!)6zcv?2A}ZD}Sf%?KP%H?zI@s8UmZ*P!+f#!8i{l`4X(8&I2sO`7Rip zU3@?N=8wz&_aB1a)2xVf{{beI;ggR=UoKh(!IO%ro2wh2)MJOG4A)t~IxG?vH< z_~5~JhE`s{NNn=QsejzH_Jb{9Rk~=;m}^d_X7nK+nx4YE$gYPQq^Iuh{d#^ zYU;K$F2yt&0>L5VuZ+b`VYQq^h7(Bjyotw^W#vZM(;fIq$nXXwErnXwS*fUbMiE3C zO+$zum?n51u$!g?e^Dh!#&s5wt|=rdIBa`VxjPqQjUhr_*|WkaX5mRzamqDr{<#AwCW^JNONSR)mhQEFO+3E--bstgg3 zi$V)0xL{K~ZpL3|lbJu9L)sJs4HK#wjj&&B^{(0u*v=coU0cS{0uy{oG|i`9kDmSM z;;DG@3^U7gR~PY4jy-7&U&cxV#dB@f?YX=A-eBms4&r5@ohv&}wY1C<>|3^)2diVLtvHbF*fcXsL-UMb?hOZ^T|PSPiq3#+ zq~yL%uEbcqOdsPr4Scl~isf5oX%S(yoC#qXW$Ah`UX3qik54g=irGwkG5345r&M*C zjUo3$_yFT++E5o^TsEBxWmtF7&EM|yHze%pEqsga?8Si}JeOMd07lFNA|nr%(P9$L zN0W1S4o%17@NAa=G%?!JEv>%*NphsXDK?i|3{0|95cn58E* z&c|~Cup%IeVFc&;B3`ER^VMuLot%$G-=97C`t+GNIn_?d(?Y3IWrb*H4T1_ed1KeI z|2uzq)z#$D3CdmtXHLS7;i9UHDh=ax4UNj-dx=$hPZ* zBWE)3l*YeV+DjBQ8pJ3j*o{* z|Mc0<*aN>N9gfRHuYoXR=nUH7pd1gjrD2wXBGQClR#S!Xi)|N-XD+TZ(zj*A6;!>_uI5o*_<3ce9(H-$aU+Rh^?k z=HY9e4aDZY(4#ZhbY)7HboIU}t`4-~nsB$x#h2GFUn)1(cCEU~pc?40YebIfV{XXX zpi`!vHSs6o#gobD*=Rh1dXlYSX|#!2#YJ+!)qP_qR+1RIdS4bDy5?Y@+O9bn&bNE^ z<9d_DH=D}6vKn*V)CY5%48t^J?Gn9*|2!U$PkhvWC$gnKLvS&*=yIYz!KQ_{Gl&i* zpiRk;fZNQ__OX2^SdFi)_t&plc9(M0&>rBI5|V%d?w2M-5JQ3>5xJrhN|gPbO7=iA z`Xb)>^Y&>%sd{$>lmxa!xC5(neOO=Lyn20g{_^tT^~>|!YMW*Pv~-6?s;TR)@k;I4 zwehU*o-D%C$^7*5v-#&w!;^(S4Q()R77NPTHNDe}zq_|=kFbIyyZUfmU_!paPk2tC z=GW`?d}Cb@&oUX>En}katY~N)2HiBPomFk-MsPL=Pluz)GMrC?AxZ|^WJaqKkk{lw zDYr2rJv)@;A%EGmUeqq*#=2IBTo}y;u>+8nxYYe6EBfG2?>EZ%m8p7G&IHNeG_&A+ zgaol4&L4D7umiLG#F2_{z!4EC@v7^v`{{=m^vLV|?{D;YSpL&LM)S|L)6=tYIP=U? zaxR%Dr@O!E>L(dBWO#nFDdMnA~Y=DIA_Wr^D2 z4LVm-%M}~;jy-Ost=9fGms7D|sx^kE4<`1X&#y-Zu;nKy53mIZ+?9y`0n$+6 zt|n}Sk?K%kr9f?S=$4zZ7an36`w<5}LhgV?6JjkQsuQHsu6Vg^))@G3ahO_d4H8Ew zD}CC+JEWhP1n=mwru@W%BG5$XaYk*PqQ;?&5=gPfRt+sWPA$Mj=y0Ga@r~PTuZNHAOhvlS0r}1qnU_LK;NNms7=Y7SrMKI#=&T6J=vp`h4#O=lP9y2FV0S$ zEXK0{spk(`SIdnqZL=tar>?E6>|j+B^N`}4s0|h24vmP?`#9;G@i?-TKWl@WAc5`v zNpl$|gS4s~r_#pCV2mk}JOAzUyHhXPjZ&?u;+i17%j?~je@edoH|6ruJ9zWp%>Asx zz<#g?rDk|XS36%-X&Ujz|^!eQNl3-?2HH>0LVS7 zF`PoPY*lrABILc+Y7VR2P4YU}t2-E1!Y(>7441Bl=D*1qH6R zeDb8KwV`8T$N}f#_1dlxNrh<&<>+=})cFbgnG0l60QH|k7@>~JwwW61I+%xJqTZka zW7BEb`WspvWIB>6&nip|}|N8aUuP%3dWVV4h`)t@% ziD_m!>NxHq>tsB7HeH@9=JT`p`02twor8fhY-Ye?YX7@B2k#SHZzwJ|U#Hh!5_f#_ z`sSNe^gp)#H~F*N0|0NIXzCn-Km|KOt?ENn8_)E4=S@zg<&py{be{MW|HalH}qhKBY z86)jd=PIQNb)!GX234XLdvuR|T^!IbHIkZOVZ@vSR>1UdQZz2MHx#-$Tff@u^D8yh zrcE0;T|UywfDou#k6AyI&=lB&fUS?)?vR(!?hsvE94`KI_xj7#DjnDM%zr*uTimcHOq0-!fv;oe%=iJQE4HBPh4S9MY8wU`W%6?*L9b-=h4t@c0ucO@ z=UH45b_6$);*48&7Je}aFbXWDXD8E>*<>>Eah$n>#?ecxR&DNQ4ix4rNm>AAY8zRH zT20gz?XD)ifj|fL&b8^4M!iu78?|}9-S00iqpcRh@xK^onb%bT31>kq-#ai^wYfw} zmfJdCrwI;?=&P6IfBvVkX?;ELtf4!?<5;o=)P{*EZ{!CHb96fA0L;CF_br@F9p8o) zxg2z!QsUOe%hBm+V#FJ}**T6v28sedFD4VowKayy4Ud7k-lqR*?!HhWUyVO^lWd&S zt@?XFD`v3m-t>xv1 zayVbkC7v``&fWP0LBiz6mx}Cd={vTu*x}mJEmPNlMxCv7i>#|gp}9_~o7&#gB+MrB zU1oNBYjhwo7hDEuR&29)>XpUB6zu(nP0J6u58St2heNO-GDL?PxUTLk73aWifF*($ zFs61^n@OFeHJJ#3vLSc+{pSZyRW5{!5oAaM3NvHR8P#8I>OTNQ9&|%jk9{!3wW7EG zX?FjmR5DP|c%n8aPIn!V;DaG1m)(Van2Oyk7VpdyoAxmeI zjVtMcFYkV>pI(2Q9{5%1a0nN4dq=@kRFe}@LFbx{)}KEyjD7z4I=@f`ugQ~y=o}bg zjUe%vM<$r(5r-GD*g>?6+xtl7X<`NRlK@V=oEFLFGiA75J6|V=pLt zY%N0F2%8CdkBGLO(M007;9M7oJ{cT%^lID(_xDlHxc4)palbd~)sEh$Dsu;7p-tYb zDkHSKe+-9DpYoV4=F4$-ItrG72lR}bDz<@Ppw`-=t6)R$hoL)K_|a}UzFs@$+wPlf zvp;BYO*ja723x$gz~dkt(V#dz?DvO1>-$LCotah~>S?Lv?En$O1iW2WXb17|cm#?} zN#iCtl(>+R^d?Py-$vJp{j98m-aF9bqQVgnilXeDo`fe)7mLN|$!NaxXA?k0zJC}kCS;CN>}ynQRnWIS59X!V{waf}d= zWMY(MKL9@`#t%}EPArz1iS2rNXe#j45KTg{tbHO~>8pnhKIXw`{oUmT^!ox0ClA47 z_#*gy+NFnLbF*Jxu3x`iUtZlnHv4k@s;myp&d9CS(u&4SicnWZ^BEocHf3n{!$em%WeQmomc9j`47Q%&*e5VXO=62l) z9Yzj=uE4zo>cxmGDWlI%Cue7i+4wAU zmcBLhj6kFr0vGuM7>-WHXYIC?Uz32EfK-DMlu4khXa40T9c-I=+ZITxU=9Y?_QS>n zlO|!YBx6z0l+a-a+UOOg$8_t5pL3jWTswAHbQs$i42VkeW{L}E%K+YCak(qItW)DA zh0iVr+?Pt;G1*Nb!9L`s_fmuuYTR+*_5hOB5}LX>)Joc*O&|DPYZZe;MW|O@1V|1P z==+7zV=q>2EFNL92f>G(LbzIHQ3=YA>|ru&r)4+BIY#)HYwKi|^8w5_pLDp#_U_Mp z-mglB<6_a_Jk)!_r!h+fe!7@kwr9FoY;~M&=XLaTcv+HH%L$*_i$DkYt`7o9y9I2n z)kCxbHoX=4nqC-91Y9ED*9SNAyvTKYa-Wm-)U_P4JX?n7aG*3X2@C5DdWl4A>w=qe zSiKz{4n|`g+x`*T_HpinEb^iYh(6aZ`r=GDXl3T;;nyi-&^evw`y&72s{T4I&RhsQS_wfXPZ*K#ggCUy zK2<7KoOSLF@}-`^*wk5239w3V`)QFZ6G#GV=}Fnfd7D60Cka|pLyZR(u;P3`Rsh-l zDa>iH-L{GIF6ShT`cfE z+{;{5r^7Q2M-<&*o zKAld7Ll;XAbdIFUVn-(uNMafY=#-U>I7$4MNwQgwgIA%_hVgnB#n)x7tJ6$@YE1`_ z>S&ckz+#n~H!EdN=(&>3nR(`_@N7I|of4=Lk#$_uAJN70XNsd0cRL6Aupq!f&9IzdbIK7$ ziDDf<34l(fr`bLxSR_eN9}NEMczCi{{N{A|*^|X|eCk+BO9O-NIEA(%CYjA3vWb<0V+K2%2JjE*B%oquc2+2_Bct=XBq(Z_KkG;Zl_>qzbHm8* z`zwA#ZMUb0m_gc*BfF?N+%!spvPONaV)o4A+?oc$5zs9k!;#9;LfA4Dkwo~Aq=#)r zGz=?fYr>F)4hKgJa|aM3p~iu$1-?R5q1fS$aNoyY)~7Rnh93A;>2SBRg(z;>;8>00 z9wwuydNR}M7p1znfh8jzoWGXsC}Uhm1_vx_@K<8#Z3&biS~cWj9V*1gNXf~5h$1sL z2Ya$CFaYHsxLsH<#bHe?RZdTW?Fkm+qu27 zk_jrvD#SDb0|;v&ei9`k@2EJNf+uIoE`YRU8Cbq)5kQt>%OKhM9Dvq*xtuRg=Zh0a zH{G$XhmJBdJ4wDSs6QAm!Mq`#bEVw|3f8U>F@D!kKsZ*Ddyyg2g9VK`Tkz{M@`4aw{zoYUKHdtdK9`8giSe)_&6}0%?H70IDa~S z_F}nQo`fzK@C9212uM+p->7JGHuQTCjPR^?EQ+*(9=bKc-M@?9R(khJMhX9=-d z!`fWA=B`qztF#&3bc8n>rRI|>PQ}24s-+&O(ZdG_pNK0BSbXQBSoS0|1TeIMt7Xn(y2pDkZqB#r|R zqc-+yGI5Yi4!1#uI?*;@JP24V@jHS2m7mjyAaR>=m4;4x9T5EP=u z(HZcE6@-Xp3YrDfL7JKq6QSs(utJGX19va`uO^4S(R0G#a6TTLPA1dQ$V51cE`ab7 zYTmwExje2@rW69PsnLF$;GV%!j zm864xsF;}HKlbsaNUk0p58H${1t)PZU}Gv8d2XsH`DTd5jkRdk#Q|w~lD3MJR+w23 z1ib>t1->chhg&G!!Rd%kY@~8q0sIkaEG`N5wcHH;GK>s`cxUc8+U19o{y_cg5ul)f z6eZ891P9DkNGr>nJiu6_x$|HTMv4|-_aI+5o0XwM(2msp{j&Linm&E>GxWftMas`m z_d`W89vK*7bTS4*+a0&#Q>FNaMzzzyZf_QuR}IeyVUm3A0|SvFU;u0_?8QXd$&Vq~ zfW86T7#9DI?YT$?gs_B!qUnj9y2V5yJR^>QX~PO^f9Q`wZ!{tiB3$dEX+ABwtkKGr z<-hIFBP^{QsKi=Gs2I{lCYRLJ|B)9Te$Xo6hLrYd7U-`+8L8od1kaOjycn^(4ClVd zo|TMIE{a}Xlc)tE?qm|vMdA$BCHiw5Zl*&g9UE%ZVSA=4{zPS7N_LR0Cesfu9tMD{7-82Yw$XGD7nolRg)8X?I ze{%B0i#ZA$+VOPkPKapf-O8UN<~x1f-wL9S;fUEc`MN5$Nw#0@)0cm)UtX&R*C>X% z5C<|0npAHDhOK)3U^I27OLw^-Dd$LVv%?ADPq!f~-1B~#$w=U(%sHOq1p)0{t3_L7 zI#g#t7lj#_cVlfp7+<5tof#5CicP<&%<3krB4Y(cAlxVMI;+g0MHvDHPCmeI$J2K% z=Isxtn#LRzKYWG&c~9GR?wV5%8lTB%Ih!nJPE`mD4nW&7L~=eXzaYh4o$)ak^#s} zmGZ%Xv0xr*hH<52KY*RbLtzwvk-$1PRVae{ur-> zk&-z|`fS(AL&uv~PJ>Ix))L~T(BTdmkuK?|sAPFMR3vtlogoAWJRSywJkTooh0bC5 z1JwgH%V6l4&*$y`p=j%(g%irMw8}Me&&UGE6$5!#_yq~KFer-D3@j4llz&cpF zrp~MofR}w%2itDA^Jd%Ut3BKrfV}q6VIP6pY7+DttFEr8RuCF(@!CpJc~5Vd_Q%q*d72y5}=F?I|;~6 zSb_vs{yyzO>JcgW{sTGKp~XOgO(JWj7%h@;p@WzXrXx3)JI>rPM1kv_FIW$b1@%w; zB$w$U3Z-GxwjBZEnFVgSw3OdUJX10duU`uIS+>wfUsTad#Za-g>NF_8u;3YL0B804 zx;OF#HyzO=)&$xROBte8JoPL4Nkf`yswBlS2o@HUo>Q1$5XT-(+{N5wK?&KJDMKVI zs>?f9j0VZVMW||nFAKs`7d)P1YsS4R6js2@xC}JvTvsktbp&-h7>um!Qdo*Gb5w9z z_L}B9#lHV-FUpVd9HB(6Dk$;5t&BS7O!H3d=ci+EW~0f;FgO{q7;2>Ua_%9_j;i^a z;&1z1?j?`+9(Uh~3zZ>(bH}$JQ30j{L7tLs)7Yqp2#hzY?e*35)y?Za{psRQB#uxH zznc^Tztw`K8&@4s6sO};QfNJ$ot&MX&R#qn&&R%S4x>Q#gYCU7`1}0=j}q)+V8sqy zx^3f45l4q4zmD3Mom_W0h&e} zFHXE(w?#4Qxz^bT6YxQV8O~kgDzlY77VMm;X=J81;l4G9>LuJhN%wKQi9jII&%~67 zXIt3*g(3D$oA3I9++-F3ZQ_yD0k6Uxof_V=peo)ynI*Ao z$j#W~Ne+6t*l(g}r{uXr*Ge)BE$p;uXO9M^7Z_$}=pc53UN=R1cr(=AE%np;U#bUw zojM#tg8^&l?gU>cyDr5)(S{pmcs-e1ttZ#l*UH6te4eiI4S77ZbL{!YMeYdRAkPW( zYfNmU349hzEwFN8tP{i&WZ)=sth>Dt8jo#lwMm-}2;PTH7>?X7PwE6t*v+A5DS@8E z>+x|=mV_g3G=b6<#$EiIa*N*8TJGPryS4rPXLql$W?|7v1pH^Z^?a6jXZq~ab0;Iy znGW=+#{O7r_3ybo%vq5g2#$d^B$?MNaM~p?o>~^ED~L`DWnC*X-S7nIBBI4rBD1yI zn*4iZ3WS9awYIZ+p1j_EiT;JT5ov>l1tDiO2|J@~7ohynj1gXvE*2??ET`{y16{W> z@PdH{d{CAUTJum-MeYi2!UF9^EEBx6;Kelyhm_Kw$%zgpeh`{~`Tg7nejIs&39 zl2O6K$_DoPy78_%-+88)5G+-obRrdZv^yRQpBa>9gm*IXioPx>$@C6K^`QJrg#~pD3g68v3{QEUrC3nmPHW zl>N3nZ0dxhN99I0zEOq)+gykP3ZxGfr$&sU6U!dMyf_`7&4*_a2mt&QrvHbA?sq#2 z1FKH3CDdoyBCMMQkvLvini~Qo*>JRE2EpGaAuM{TE7z)iQq@IFR@QyvZd>xKVApT3 zelv@{Piow&SBzemlNoV#t@-;dwEczd%tF^Y3Cy!VUk1tqh6|LTcgEn_KM0}H9Db=#Cl-XH&Jz)ek+x#8Vp2LnzeD##Y9n~jp|DK2GT%VaF;^z z1Yrtl8xDeg^veiE%=9vqSaaKjK`5z;U9?)Qzv0q#We?AatGWmw)E?X)(08Jb^$vym ztwo@c23od*WGgE}yHdk?1aB_j8LHyoCP3aU39o>~0D>`yWQD0j8uC^-Kg%2OY=Vyt z%y#Vi*@;KoFByVF1tMhwx-PiB1tx%9(zp@SEeLldw8RVg=n2e4h!KuI05jPhKQZhe zY_d{(@wky#6)~EEv4#B-N0NQC?A?t#aNueX2mi@d|zX+XOq z3%@hgys_yHVeSn*vJmMm^h^Ma6WKD9dqTUFuX}8B-!KT%9l{boE&vuIp$^6%F`M-p zH3pn4xczH+>Jd+vvyc~hy;mZ9c~KkPq}SF4R$zjp+!0bAwaO1B)5Y}Zv&j=fUkZ7d zxHnNSn4TXph>z#u;zNgn_F}DQGf>5iKUPKybGmSdjT^WlO&ckYbBL&af8o@O;^gb9 z2ljsKTg}ADi*N@+$)2zWB?Y^NjZ1i-U@OgibOlKdlaiAe7;;*7%v!vk0K!FwLp@{P zr+MsyH7B%MV*VA6C8uwIbX~8&9k;q7E#;GA4DW3t-``I9NW2hu0{REHLbbbgp|rG~ zYv^x>TqYs3fLa;}Q&ZX5gSz3htYSZ6v>{aydS$|2g%RN4G(B;aXTkC`#8*A^X08F9 z{c-X8zLwus%_+C7c<^NTu3u=8X@G;F`SS*~iS3PGziqzBR;F<^Xs)AZdv&$D+-K+h z?Dgj$sB#PVAh`}zFYjipwluwy@oaH+Hh=bPN>*>~X#f>dpHt*V^ugc7Pq@)w2S{`p z9lG77*>9k8W4()9bE9~nIUq_>oQnjlZaTt#MF$IgcQG2BPQAs*4x#)OKVokWeoTjn z>D%&2h7gqXgcPgZ_F(F<#$3X(?UAXOF^vU9R7-785LTEbO|+{?_?8kiAcF(%rVOTj zAU!ym-ZX==&|E+=pnGbi?jO_{Y~BXYQy1zRW8rD@-u``@NAE35Mkfl-zz!U(7?$z$=U|2VaIR^i`-qG)&ix@yT;UuM%oH62#;+sMy`P5@(Pr@De46e;@lBGhL9 zFkP~LIZkT~yCKe7e0{eagZx03;hv$*A(jyD*!AS~dwbwbkN&h5lrQm4$P?eLN1>B1 zoOoc9*A*QIQKMW5EKhls7I^d<^U#p<5l6}lOR**5;c@Mw5|G!@$?SAJY!HqzW)vXL z5Yzp2iykLiX)a#e)9kLH**b!4KJfbJQ3@sSi_LRXwean+0m|0<;^rmEkCbFtw?oBo ztn9NuamU7Z!OpB&Ml8>J&R*Uc1zqu{~F zuqZoYQI$%(8#ttiFn|Xkt6`FZYF#Y|&=&!IoQv%L#D2+ds2cUL$8a3QMZd_=eaHFA z3;7X`KCYe4ywh={C^;xfx|t9*jnLqtAWzn4Yf{e#9`XwU-Kv75UrGof2)9fC9FB(t`+lO>V zMHYlHHiz1l*pIX-%D9Yo^_56G!I`Of<1*V3X@VHO-p6dUuzM`++8VWS6sa2nYw z8Nm!@IFAU1LW#!^n^k(;cu}3%ZRKi0g>icUKQOK+sXx%+h<(&r!)`{yI>5LX5MDnt z2fzRTFktC9E~Vdh1=NRZ~9)#fZqvC(m4iUoFns zBy9;@fK5*j6mrmJ1Ou0c0CXf<0f1st9|vu^%3{S6{$C)+1crEU&gFHzi!ZOz1F`63 zv532~NtaC;Kq^%L>~#o-Y%MbpD?XZbk4_QI1c}fCj+s!xLk>ocDRA3o$5y=k_g03y z{o&8@P9!56P(oq@tDM;+*4Y`3vm5HWu*~;LYHGW-i_<*UNNu)4Cl9n+!AV>DywfFbv_9u5khra5MUWoGQ*h_cc-k$vaCr)PzFO1DfcYvXq@ZTuS zHv4+@O}ioSL;x$C26$jI9xCC;Jq_)lZ(Fw9_+BnOx}x_t;nO!CqX!SrfV{)%L4!#bh$bC4fC7_hy0cQB(fue63=k=RJnjx zWLqO5QX~fZ9*lLEP?%^ab3{+>%WV`TU_;06mjUp z1fUTYD)Hwf9uTPx4T-dZH3SP?Q;yNf zAOJ~3K~x!l!-XF~vDHY@Z&85JwvGz2=)!oXMfu_(mxr9==(mzxqcyqK?zC#vWaLYY zuE`1;7irl7nUSa_)F@htrY1k-kYZ%VEv5Gb9yQRuL>iD&ueHmr1 zZi>@QIo-C_4ww@B!*bmOT{xN>>Z6Gnj%^QPv&(QG2?k-gN885-U;Cq*|5tY+U2%Ne zwwxYB>o3J{R3;IIc{0~Qg_E!|Lt5bP<7`GkLllj#)gj649n%P0V;JbL6~b|aqZ%{u zqk5Pr$*JgYcwb#l^L)dN$>waST9}ipxwR_6?6dYtCIpBjNc{n4e_rF9OvwdWpwY3p zdX&0&cXb$|C89`d)TT{ZJ0lDRm;uQv0?n+>fD^nt68w7$Ma0o;gS{TP1@x3C0GgfP zjI=Ry16TlR*7vFGX`okE*}_6oOEL&VC3;p{BRSse(qu>yBoe6PVlh&v$45rDu=bS_ ztBt#={R2GN7-;~)=tzdFVGDE6%Wb3Oh1C@Uwdk6(%;PliVvnR-661+si~TtSji5EK zsR=w4{7fGT0fm4Z0|fMA_Ow6qaApADbthFqH`cooG~mHnaqV*4tzRhF8pry8q^>Ck zhPcj;@u;M-d)6C=5%#Dd_Ra+yZHu?R{4oW*Q{FpoDG_3lT~Tcb zl)bg%=_6Zn$lYLt#+aPqm5EnDm?^e(cQJ^SWUX9X80229kW*1p%zSkUdz%S9my*6$ zjJ~t5N`GHG#cH1&Y?cDVWT@bgM>+4Ug=y2`L$cv(-OU5&?Dj0;Y>EI@822BS`4=^4={dAHA z2M&M>P@1wRBU(gFda&@fKJ14nv&*u&8tMyAvGFXS8EK=sIa38JuH<8NYsMtyzrS4sUVag5Mx(hXKOZY4e+;PN6@Vp#n9m( zpu`W6x`sD_Tfy3gyI$I7yl6Rj-I1O|$;P4)~>$zXqr`JDU5B!zr zaL8EJUNZb4zaWe7z~{FkQ6ZoaA+LD>%Bv)Kd~}dl)%{NXM(*L@MiJ5I=7E583-&E^ zhCTRAb%o&=vT_dOmP3&S#J}Vbhoe*Wd_=H``0f3%;GwDJQa4H?YOh}zX+BtQG!l@K zpWZS$#|1CU2J&_d_~BV}l!j<=7_~%)W5a0-Y;lGTm^Gyz6ko&TNV2rahkXH$SmydJ`1a4O0w{CDkmWJTMhL*Ul*#nZ3%IXO^I3@6>X5Aof$3_K+l&HXj z^Pt0t4XH7){dVAs=xdVH5h_huJ6pk#D!h+( zNVEhfO0+Y^6)IQHoeesQ$eOxQ#Z4gQEG8epn<8@_HM<#EsDyx8MY;fJQ$g>e%b1pM zkpsFT1IgnGk}KpoiM26df1}wXo2EF`>zWWx>fKb3W{XNFrgkDU^hOlVnhY1rV_`_7 z54b+g5Tq{dkbx0-W&k>`I8n;qDRI$2^Fi<=vyW6zoXqqV;yf(^&Ij*$wAoapb=aVj zSy;yJ2s?$Z6Ml7CWP&wA6j}tFvKC>}y3CSMXgC&F6+wf3ggDfb}3qzE5-c7rO}` zaDjg}r~b9*aLfQahs2N}9uO)bOXNU83r`1Glk+C`GTFBEnHF5K+>=@>_LDeBk`N1B zO1h=&MCjIG)viR^BcM>w0AC?o1=-bm@s5r>Bo=Tju`2Q1L8-44WldmO{Y_6}bUR}OEzq-C&Z+6K(DR)_Q$N+oQ1&XM6g(+BMJl!T+cI5a`=n^FnZ_RAq|w*@{K%6lchuZ5ARNTJbYv$VZmekus4|kF?jeHv}y1GLO zhTh?M=e~Vpso#6rv9S!!{yN%PEB9pH;4;o$1h9HFmqD3?cflU(9iZX=J73e=3qT8Yv2}kv+ zx5IVF8SGi?oW!6d1N82AZ_RwcgOhtgB%A>V7wmuJkpQ79JU>vnVPt%)PQZ+!*%F8V zlhJLn1=$7lzEUGj`x)9#8&SX*fnu%~)P-!*nBP6#LNpNC(GOGhEaXHs5wMeR`K12c z|5p01;=QuX+(V*Wl&$cG%NWL%58?`zZ_!1|yd%$^9Q~*l{Tfg{xRC)HSN2$#<^=9Q zde^`o4Z|>;Ou}ga-`a>9iv_W^#hyyKx!&PQV~)MV-(TuDUOo~8<~IUwa~d3m-h2t~ zrlRbWHqc_NWzSnin|0zHG$Wa+X)?%4J8MYEeB3-rC;lo9)=d=jENu?EVsn*mF0viD z6_b72of|gEy=-MU8ccmJn1tSJ;w>i-HWTQFmt6MPpN?No5B&A$aL2(fqo=pi+;$<7 zg-bxQV`j7HA-N$v#3D7cgWT)!acs~5c0l^Zz7p#ZDG=FQ3WyG=pOfm8{DF0U`o&o91tb$Rh> zb$z+M*Pt*)@-DEEJK3Xqb>K&`p!%FQhMw=U%dAN z%Hv0(E{a94a&SKh%c(#Hv0{*?S;cHZ0Pe?@L99o8Lq+@Sqg7xG>(FoEL{PI@Pih0p z1O``|mhdcSM{L4?j~F-Hj9ky+F5QEB&kHjt%tL8g+zQs*+copO7yP&K#RuNzdxHY~ z7}GM!2I#G_NSX=T_t3w+Cf?A@|VA?+-%N}{X{Xk|TM9pmNm^DZVJXutE z+zmZ)YDjJTCmakt0=<==gWN>*Y;7jXankTTZ9ZzfveRo+!n(xFPM|nqf@Vu8{dnUq zJqfNf@TRl&v)>SF*36!k*PG&UuU_s#a;U}xOK5wBHufsACmZZ4QDn<9LKq`=6@X*t zclQoQ37#k?CyUwqB!qc6nN89OAPzV>CFONoBwiUixo^Yf=Muz)OT4$=p_kI+qr!Zz zw4d)KRpD=Dm)e3U#7;0BdG%9Q&D?I4#w`%Yc5{f8?W&zv(C{XEbDsue4g@R75Wxw} zkAK0|es4eEVIq%Lmgl=Dx>@b7&JU|KY%+19AKamG_^&BgaQ!rJMkfKeK&;cDIrB9T z2y9_K9lxp`_}jQ4sBTDAc!7blkvIv)w9>_caa7pAbO?k?Jb&UJg*lA9I;_+%8Ya8+ za2-SVu(?{V+Sf`|ZmZ3zJiyR6HJu@bF{B^M{!HR`T4JsCN^+@Pf>}4B)Zb*&VczIO z_F#o!Nnn1;D*pcCsKE8#i{k}TnXLYzfe9XjN5AmA|Mw4m(=RRPL=s^yByJpX9z4Hy zb^%r~(cy4$^g5hC;*OHJS9$IVMQb1kE$Q+IuI~M2y}h{x!nR)BtZvpf*Q=}5ezi+? z5aFZwmI}f2-!58rMZD*k6X~OtL|zj9z#;CY+vE7+(M8?;?7h#F)E6eHJHV*8m|j?B zVWkt(3qu3liNW&m{ghNHmm+o3|6Cu&VzH+YJVMAbFAZSnD3T2rZdpia)mW^OjDvKi zmaj?Mk-7DLq1+^pQI-XV&)%ac3BK;#(Dd!? zOz3S(3T!ztNthv$R1~vRZ!AM^`M`r}g)2u4WAYGXifta)FWdqB$fLSmePjqAXsV_m zK%l7Ge$T`wGDeOw^!=jZxb>33Bt`NNpnoUUYZOL8s3^Y|H1f?Oo8LVcO1H{nWR{OV5247$-w*=1E#Y0#JSxSMV$7#9v7hev!fdvtKKNij^6$M3PHO zhhY`DE`o=B@6jqF!jRV@MI^};BC05}IFHxy);g?}t=l>vQppei-<+D}dpI?b!F?(| z6%6`Bh2TuHJS&?iuMEAmNE@F|*ID7D4f*mAi0B7!9o_Q~K6K5<+}lHz`97vWmKtgK zQLG-(Id>;L_W3xaC`SbKQMhxpcQwhk=oFxqJCDxC58-=SFHQQBE@ZZ&#$k(V8-fO8O3}Cc;lUr2s($A88jR>y#|d|< zmGtL4@-mM_GR1l|4#h;=rhF)^__hePYSh6PM1?_l6NeTl}wbr zLX2O%;)x!{C-UJB|M=b+A{pp`{%5_R5K|nQp~!*briv5zwx6|PBbPn}@;Y}uWjEQ*+ykF;eNsFOfYUo9vL4UQPL|KRIVoifYlv1Vg8vsY zZXE{zH{KaQyJ5T9#MdB9>7MRXE2vy!r7H0bYH=HcWUS-2xlD2`YD%9~MYv6e1k*Fl6M>*xV%7Uqx46UawYBM6wBG0uFJfoV4!AARudZIP;tn&sciO z$d(+|wTQvCJ|T-hJRI^ECIY35(-bADj_Df*}ijzrKc0+f>=wv@U#50OXuHyEhBJbpw@7l05`%l zjF^@xF$3}dc86n8xJk-_q-U@#U;@QL1yq{Lmk8W@Eq%DFKIGG0iIU!+bRbX&9J#f+ zqyR#j>(Pc%?}rrmsB;Q!y_EV6eVZ%T1E#Np1=molPmS* zH(eX#vvRWujqORZC&@R#sRM9Dz9p81pP|8zx~lh|Br8A1y?e*>d;jU*_YEB;+U;HE zyj`l)@NFf>SNH3YPZ-T?P2?|#PdWYegnqAN| zUrYavb3{z`w}lTLs6>=N1B_Pyi#nEWw#;Q()N!hi8P-~APm0059Vg8mvN} z%jb}0@bdWY_{s4qdjM~)N<=LF00pN2#z1MU?lkJKs7Ihl!M)@n#9^3&eVNe?(@ol z17Hx4M~rP{ulfICP5n+qC~$a1GH~ADh>|#7ucFI8A7204?iyRhq$~nWANtBa&&b+6 zdbSu&uz5No#~7I~%G~;bzfXoZQ^RSC`2;7LF1!LL-4`(6Iyh zwtb90cS{bJEX%C)vb^GUzqww~neX+_Ppp-a?h2#xvAv+6A}Gm6g#P7sZlG9@d8uYo z*?BU^BejfivX-5eOMY)UDpSqhlP#`_=lk78S?}6#2XM}B-H9?N24J~<(e>#Oy}?M8 zaVy9(F{t+4EQVycoA-L-drG?NFuD0C`WPwXcF8|^@Zs%oe8t~yKf3#>f0nx||3$wM z;}CCde}C^cec3YlCFb&I9C}Z2Zk;Yn+sQ3eKBAO2<@E(gQ%P&b1BoZC7RC7}hGahU2haU$kqY8& z?bSx<(eEC7*smR!mq`2`)WHlrPB*N%{Ui$0@+2#N2WKCAodv|5L=3~v$t&msKqt!q z)P=}b1OZ_vU9Hst(lJ z8$f$Z8b2sr`qA^OZ%u8KGL_VTd4yt%r1_4V~Pe{8`w zUnTYCOpQ;x!DqA6vy;X1lhFw|io8Cju!d9Ps3`vhT7Gv0pPs+l1Aj|8+;^|;dp<$e zuyw-A0;AJC%?wS8kSJ#o=!>y>ePg&6QT;OC;VV4UFnU3WDXeX}@F-!qZ(&%gnj~$U zxXrwdd@I0%I{>R91m9ojo39Wz7_HIykeIUS_uKS-vo2fZ4?Aww=Jbb zeKmbH!+s}^Z0;Z81?pE8)?pY&3X(gn!o8IXPy@&-UV6br8qOFKU6+-MGJwQ0*$d|2OHZ80uY zFh!+Fy)V@~KGrM-P4KkTv}akfaT<3ZbxuW#%6&_2E^=Y7xpMcfriK3>Va1NEN? z9alR&A=Myp6;7269HUnZEr7>Folf1+r-p4sMB9!di{Mfwf39$-a>Ogu7R7Jxhc7iu!hQ z{;`!*)SI&8Yi8H&mv8;@eQ)|2`3a0RLdGOH?D!8SF(Z(&Zhs~Pi<;N#Z@oEPf+ysz z_G05rvyZ3qmTM6yqpB`mLT`d@658V=EVHE$suyXVB?LM`NVHFcqci4ZES4Xk?JS`^ z&dXK3CmWrzDwMsrZ$-tHUekvy!NcSxZ92yxPX!s)K|8*zOWR0>=tdVW$GYw{S*K>9 z*?0~C77+9x*4_-O_w~fb`kbqW9@@obwS+L-t_^=f?%LIp_77l7IR0m+{Q2VdrvF0I zPA8dD#m%^!oancfa(cX8f~P&v#97QEbkWi2TAgY(DH)h^I0bwGRN-GGE<4ZKJZ=iFm2@;%{qKa~ zdvDJh!7DmkZbl|@C6-TdtVZ*tmEo{3ZznJ=VP z{YOYT0d(V7l$Gyjox)&7ux^n>>F#p7`tp4J>d*Vv>pZoL@Wh7RN>}C{xjT)v7{FQE z99;d2vBw7O^!dqWpDmX2d0@{>bz-!>Fn@Asy$(lcTbpU^)+k*fFQY9)d~{cKZSQrG zSg#5wJ9SU2FVZU?c+v8M_a4XY`?gf+fWDJtg*bls-rxS1FZw0?SP55xb|80CZm{7P z{zRWH2cxMsFh*rE-m9KQ8Vv=yWXN9f#)<&Vu?S2^!$j7nI^P_&?R65_UeX=ZItjXL zq~(OON@UHQ+Vf-D__2P&WwQF0Fo@OAMCT9(jBXIIA}U zv4fETaMyu6zyJ8|mvl2WPGGQxEt#L#6`*yKNU0L^%HvKKMM><-#rP%)9Ek$0fhM6fy){loJRNkbwjcW6 zsYX6Hek1yBufj>0M-kD-PROOn)xH1s_X?GBi4ld+mGm+M@k+EQ;;K~Cg1aPmG}&@V zuSv7<4@meffHxqleH^P;xZY@b_sXSmZnA6yb&wRQt2+}{sEmuYCec<#&={oa2go|N z(SEy!YkhZ!yx7?vOwzR`hMEB^27y9K_XxHCHZgPzYMiZDzUDc$sd@%E4&Zo%ga?6~ zHEphYTP<2$ei`Pmk#8$4FI}SvJk{|BAQqS-BAf3u@#lKUeZlCf`&3C!4GPs8r;}iE zHq;tPs0J2J`J)!)zkjV}q8_irdz;P^m37?K%XAk^( zZq4lii(u;aWB0r3DDT93(SOPWk{vCw<~RrLWas^#Jmxk20>Ttw2*A(*0cN8P;%2b3 z$aPmHt4;1&=>WATFO|#`=r5JAmZHN^HaM@kD~qm3yDIA%0uO}@qu37s03ZNKL_t*S z?T0FrYr8#%I0(>Xh(^)9K#LJWTR`~13IopghswGCJ(5>^ca%c>ZBKQF7^Fwoz0yOp zS%fnSj79Q2cB)t`(B43hV6f=p#aKXD&fq`D*fSq)cAGz4tbYH;?LN*2zayuRJ#by6 z@&=u)X;lS3yjOb@!=HQMd^}mq=ZocXwg}9bsf_iO?2VY~dMB~Ctu%Q18eJ<35@=gp zoJZCWco-@qor?2W5;e-M`-3j`{-yuW>%O~X0qd8${_K7|?C%0S^U)5jgWE;o9D*KRy}GPlD;pxAb8d553y9wu*fdMSyi_u+BA-@k8&0J!}jn&_GRJ^XrN zFhAO?`X9?rByTFC5wslh83ns58-)j2jbKQy7daZ4*>?R zFtM=xi0CKoZJmV3iVKAH?%_PTSi4_;siwbaJzbei&DqioAOP$&qvKZR1Mcbr7xlQ0 z)W=#!u{l?-RB^vFOMByq877(+jr+vISrSR6;~?i9xBCa6EA!8v0<$ zqfjF$JaOf)b+T9N^HBFtoX1sod)DoklqW=`q}amUPx>d%u6-x)Z8sJUB5`=aDMvC@ zX^U)BA;p#7h`|eqD!cJpkCJ@-s6(_t(La0HDLNcUZJJF6S>MWMZpd)yW7rWJ&*-Yg1~@t^g5h)hso!Y)m>UvlA>&PzTN!y zXdku7KevsmV))1Zf(=8=h6BHw%vEb)%})JrHV(pBfHr4NT{VOSL&}rm*t2Z#C9V>7 zmX0d>uuzRxv<+_KE>b#QC`NBjKK9)4hmY*V_jJcE|1s^Qi_xQH7u%_Btp}A84E(V> zTbwSQo}HY{7UQ9pYuDLNczQ8MBp;R6yt zDda>F0vFO;9hKR;oBM8qxy`*Pih5+i*H~_d?ciPi14iua4+K(=eXAeMvKtWv`Sz4I z=k(@ILZM*+vMl)hjX??ZnW#6t&kz$I+a!GZ3}8*kp~@ggv>=H_sTQSLLF~rKeS6+* z|$@ zx6wa)0cYnpDr*p?7~)WgsQcK(?=k*k(bz-_%7$sfB_KUf|&33c01t+C;MZqFxUv-^Ceng6K$b4p$} z5`Y`tc;Y{K9y}R`Q+FH~Q(N)-==g7UA3s>hrxSkJ9$?J>MGvGc+Y;!oKsPXpVgFBy zC?Vl4ijC|y1uTXP13t}6N6$k}4A?D9b!Q6GmggB`M+t3--9yU%qyzB^X1{gR!q;7& z+*PuK=gdOVyYizq7xkG#D@Z`TWUp`P;L_lanwcCu(C4l3W?Bi|uB0L(<*&a$l)=dHuTehHx+7 z3TzEEa$OkX>L$^4`|xrzAFRv9XzC#;q9ng33@JbNA;zjAt+PXw?8_|4iaM&g7`>F4l0Ri7G(wvQ&!$yuAC5{P;JPSDvOB6%lbHp|%XfKK>bv05usN>*_ zY?DHba|nc#5@udnDS;5*)7Up(^@=CDRD}YVDMiwiDGFh!0=qZ%?Ef$bGHj`mYk|HS z{SoIfgc{ZW8zH%+1QI9Mvg6qP0cp{n#YIB|ji!mw48&?#TSc9b8yXx(OATxqq{cT5 zeDDSJFpX(;Q7Y2x6v9HxsC0Rv>NFdMJA|irg6Ru%Iy~cOZK4yCKmqe1UbJ56c(CyI zen`7)iq=SAs7P)^!>OFI8cJ-sQ09qfhoh5yu*i6fyIc?(*({Iu@#gAqeSWxUwwq$L zudfn419gC_po5yk)))jhfc^|C@7)yWY17AcV zQF&c{sG{hwiC$kvs|~835oqw_4$XJQ^K-_+n_2K&wlTd1aR3xj4{CH6RmO2#tIn#=tU% z`0Lj`(b@p$kmivGS*LIylSh>}^G=CcbhtK8+d68@ovKEPUUrBl$s^P?vD)pF!716Y zI~=K#r9ED{KDH%iY^dA;zt0c(Ky$gXWM<8BB~9xju1IrMpb|6y`4PdDE@<`#1=(9> z$RAkj9;o&g{fK)d+B_acQt|pQ;T7g+5HX5auvJNHEeF~tZ)fC zt@oKed>z?cX)&(x#El041P6m8baun$fY{u|^RHJi3FL4}L3-+p^dWoUutC5s#BN0P`Y1X4l}(YXDZ=a{{MVwuA*y)L0W$yIl+JfDqz5)> z_8gBCr~{}Ru~1>MVPhfIG1et?xErEM+77IJljLobb_x27A=WwJxOs3CzNzBPi{pGN za#7%NjF2&!Y~HpzO}}oHQ`BrGn<%H+|Igl=Ho0+SX_~(8jvawm7$hZSmQ_tvnf3I1 z7@IMh-#=z!HXqu$r>eTUvN9!#j1BwdfWw!0jw47>5-E`qGsRFGCx}2G0QY;}d;Q#V z&+;Dplyyn`8`VwWL}USXX`wLma@?Qo_0Kmaf9pl_w`NJT$hU;hhv5C(*zIb2S8R8P zLI`ko=rT>ikJ`Xbo$lD|oHJxqwvjMibg8;t+xEn$QwRO6*?^)Fl8RGF9SF9sqIMF= zhZJlsASiv@!9Y>G85%iu!zF2%sOcE1flKfzl(Ht2Bs@$T(ZL{TMc}qCUVQQ}; zDpyGfMRUBV_Xn(mgchR4(I)B~uChJQBv8`rihQ+)C!>a;Q*6M0D-Esz1Js$b*jf_^ z;avi%hve5b9;L>Fj2e#-f}2(%45co%>jXp{h!^%ahkt6+J`U{<`m~5!Y{F*W3{tD0 zjB6QIW}iftqb-)H%bnG*MX^FikP$VHu>HfNBCCiVpD6;|puhQn?*W3@t#P2-&}4&= z;+vbl{N?uc4r-kp+i8}^EegtVeAwV`Sa0>WBDh zWy6Zks4^iyq)4IMeX?)MHfHafAKy)BN}e?iGnllk;%C!(qG4 ztkot<2oUiPxpk<>36NC8pF5C&5=}?O{x;i7{?~G`+d4UMW}=M3-N6Y&P!cNwlSgGG z771KP5|2aJ=#3msSzka;wuL$=>pvXHv zM)k+h|jDYWFX%>VLZ_Zz<4W7T5wYhhT);T~}7sYW?FM&ZF@BII9MM?@w zxP$C^c(4TCPXVC7X6+Gu6u3#ti90T_J8qV6>uyRb6$_qxlz;&XE9sJ{hZAqTJp}7w z6JgJ>ai^Ae``?`KCm@wc*`x)qwOp?bt3|$DQ{Ic4XBpaPgs=wPm1v(OprDvG-kVD$2HPx3G0!N#l0iHlhi|1>+NU)c3yo z-6fiPiQVsR&qlG>tO~YLWYxnhjrQi)QmvhD()43i7DgkRVuf}b+7XdnzDfQi&3^0T zoKVhw{{vM_UcWi`VDniT-(;ImC78`5o5=gm$PfOD@M{iy20U=>qL*%ku+z@{dg&T7 zhKwDn=ec{AFEKtAiPpMvHg3xX8V~+0k^935sN8 zB${3}eH9Sy`$})`m3KbOao@JrKmTS;ZQ231Cz~ENlyD!((9y`IA+HDMzIn}lioYIz zebeT@VO#Z5ABw;Ss0tr?s$cXu=8yQ&Dz@B}g}0QP^u=a!b@%W8^Xq^8^B77ozgn8r zn5?D=5%Ng)YxAptU7p8v?|(O(o>Pv3s2BJl{A9ua^0ut`sP^Eqr4)Q#KxnkaptU71 zBZs?)I2yyHV6j*LIF9ZHUeILZ2@4BrD&+w9>vs!F^T~HVy%tRHBxz~AbK%rcsY@2B0`R^G%NT0g}9asr97G! z-#hj3rF`6(S1nq}`hYW=60kq9V1E)Ez?ba0 zy;Ux){M%~%|7N!@uK&&XzyF`|_I{+e3Tgx6#r-g%5cWU);j*5rtp3`e08<_8LyIEt za!35|;423M-J@I;C=~f_4Rz0^UPI_ADufi)NK8W1ScnD50@4N}!4qfM0XBiqv?7B_ z?P6}ZKR5FO{~0@yG`05~C(WFw@a#j1^@hr+ITckf-w3rl5t)|9>?Rs3vld%Q9dF!l z?*zGT7sML=u$}MvoterIBNR5fY_Z)WFk;%&lE4JmUECGxFm9V zZMO@ZS!#Qs@KL`zU?MZyd;wB+6RpdQj>d9S z!`{{C<~^n-=>nU4x0IUDJIbqY7veG(=LOAf)QZZkb73N^D=~M4zLSPlLj8rvfaOpE zUGEhmghb;6mCYmn*x`vmO`f=v2sGxoQ`wwUn1gV0OKG|03UBxBt)?E&%OD|GoX99l zvHA~fdxMIxGraL;(SEgTQ^uOACzRF_MVUPM%)uU9;x-__k+NKDLk+ajqdn;z$i8*G zzrstAP;cpGA8-X3-4FQKU*jWFFD(XGdP&ZBPf+abVh&#aQoG!9WxCs`KQ zp>MPCWcjauno+gHH0GXhP4vh8aWKA}6z5m_Af4OA*xlWQ-FDEiy{KzhL|+OE_Un&+ z@K2UtazgV1#<1CJc{g5OkJnR5gYLJMwI=G9s1D2vEGy&?E+jXA^$W>Fc!tp0MkH5= zeO#pT2i=+6?^xIIW%eb;nc!$hl8R&>I@^u4-21CCSv6;3wSK>JM&2U0Hf6Qh9#D|7 zyGb^=Th2Ej;u$zohvE{%%nQ7Q=`W15vRZ}}d6pMhc50fCZbwbQM7&8Ngq!~eK?(RV zEb!*F?9&=ev*{PVqi*rQyde>LXo})ZmTtfE;M)Yg93(b*vc@U5xE-%&(9lt9&?JUV z%=CP>*#yqAUpu<@y_gG-KaSfM) zLGCl%zgH@lKQu0ct2;G|zPZ`k@!qpmt^HwSY;EqSA`q<_b_#Aa8&_VwTWq+b?aT;b z%iQ#kghlOeH~+A)-*0;J8zV`vk;8V8t!|f#Yf^}(Yq~onO|#gJqQh0;-Wa(D zVi^52a=R_N+qGLAA`wV}Z|2B5PR{iB!Axg5(I{jMdRpZi)@{QLS5BFQtE@N64#~Cx z3$$9y?Ogdd!`^XX`4dG_3e?T@biKTrkrJ>ap_mY3Oz1v7KEA}wZImwU$Toxp3S3mVEsH40AZSHYcrht3 z^>Y#NoSk;S)k{r)j_sb<(V+?~Hj}+s-L==V>COnrs|9C?kY;NrZI*451XA*gFvWoV zJNy3`zWIZp{tR2FO?o&8$Q}G!9<0|0FH)cI>=B4Y*tUyl3lqOY{6l3?^#)EKUlKK~ zH#R$5b5MGuF1@I}B~)&?@;A3h5yo!ic{vsggj$o|@R2>@-p|W@mQ5Fnm$#FbFD9>E zOLdLBdsq)T!ZaV`X6BNGuCRQ|&nMYR>ngZ~zf2OGqXP#*EBgiT_qZRA^X zac4}deN=DO&T`+H9tsv}@#{7#)^U6P{T|U^f??LIpoeO;&Z&H`y}rq=UQ}eVXQ5P{ z>Dsj~fjQudD#xOddT4G=Ou!nl$>>n@|4W1w9~)F<-5++3j-t?ak!nx64Ve+ohF|$l>!6Kea3|a@pnuujw^U zfK2C)Z@<-)=3}4V(@}YQ-2*`hzq|zhNaS`Cz%7cLsQIkAxPmZ-hGrbE`>hwn&~d(~ zjb*mBaVHr!H40oWv0Gxs+bo00x5c!C1#3Ml=38%kNQm?*Y{?s7_!>R;)1v|2!X`m} zPZ))9KT7{D>hxQ~{gaE`zrNgKKJNB|G%-t@smogoBLQmYW;=hmaEw`+yJvXC$ODQ) zg1OQ3-#QpheDyP9e^+F;S$2mZ>w1;HxQlOITJsCHplp(;_l1}tNs_^w=_H2RG7>_K z;)rU+rcJ!zQHKLcIX80>nj8uIZxe7~Eq6||3H`fHSv)Zo`@K!6PCg{gJp8H_%w4;~ zbdeilsP%~e%^)ruKz5f9kBA)nUSr>C_v@GFD0xO1_Sts!azVVu`1*D^P4+pwXw>Oz zlp$tsNt97*XY@j&>v?gScwI)cZKW+UgaNSG>HXY=juW3xq50s16si=bTBXL$$lkF3oI`_aEO(UVS|qm&>)kcC1BwS!%3ETI|#!>ymC&^`ED(g^J9o%kt6(g zbzo)R%~#oKvfN$YI~jok411 zxP+QaMxhak(2jT(u{}N-9p4uXOIq8&D7$r%U?DglGm#y*kBbKB-;CfJh-u;xRmcOj zWLm87jgzGE&3JP4VzIl-D+15m9>KMZ1QvH(gBe~O6gP>!liI2ko^7F&Cr0PPj3jR^ z|J%K|AA59R5D;%*V<%|nyjcB?V+hU>JjiCS58_Uz*D22jRXN;Ds`kbS*DR!FX-15O zn5MIZ{StH6`jG6`Pu=OdlWe>84t_#7{tTJtZ(NF>w$E5rWzP6+lz7R&OZQ z;qH4ZGm{&Sw~5it?0DxVi!_c#J*T+K7F^OXcP#nbWgexOJnV4!4AR+gwF*|YwHa8O zHMzoG90X!|Y?|7brNRGx#&d<{u4dbo?QXO7k^oF9k=V84J4p8_575?P28YAf6!p0kOXWvLEumwARwn#_&a@L=T@7d z=r+bSHl?J}lDQ?ki4e3&;rnA^Su<>JQ26-^>L(O|hmQb)>o-5Ry|D)o>vFK&O%TNG zBse{_tN&ElZ?;_&5DC^OcV0Fz?rzAmUysspzimN7i?}E<%GBGCe+MR& zT10-ZaIlg#H#VKTzqe5pM1Mmyz|8`a2it3e^J0BBp50C-*VFN~&y0We!l-X0`Wiw#|sTajzwpos6)9GZs7_YW>^Lm1H z;oy;wVZkFpf@`_L2~#_#brRKS-#8gKy)N>ZH3#=E_0$^=FsEe>uv8!_g}@412(BsL z*q}bq9|#g^>>Na&@Jq2bcN=57EV5{~WASrV5U1}6D95oZG$&Z+P`khii$!r@xtlE} z9JAY-#m%>iyO;IMt2SZ9Dh;upsJzlG$CwZQH58qE}xVY~lB^F!R|jvoc>v^pJD#q&d0&tA>bmsPST z;yK}$s3nXV8sOUA%_?e>O{j9wpUjs2bb}R@04^~MyZFlAKJBM_jVi#7BnZv4YadSC zP30wK;N_iac5bX|i9QQhItjA0Y&_zHW#tC~9>qNx&K+~!sV1GGh2^)7ARJ*QA~iY- ztSW2sa98backAofy_+Fai)7Sp?Ib(eG7`(I{lKU+n? z*1-zha`KKBhOS@N#DL( zlq6?_e44`D8$pi()l#9u@@+Uw>Hy-iMw5GMdgx0<;P;Qf=b*#U2DAj0X?zX4BuGzs zk?}Ne3)8&XuO|D=>~KwBT4Hz|*J~0WT#_cCJ1Xb9<<0UA;Y>3Brdq~j>TP4Y6G{lQ ztopLVi6AR%QvwQVE(|mfz<@}CZ-NN(nJ$yFi1fhSBkEg zHf3y}+1lE8Yw^r3%=M}!r05Q7MrgOu`KXkN2rhA!WY8Ylj~Qj z>%SPc=jF^Tx6tmZn&PD?ll>H|SZZj8V1uDf46 zHD0a{LpDhmYzoo@(6hvjT3d&eU*566ZpORu)%s;b8d7UHk0Q({*z_W0jFG} z1bo+BX{$^Mp<**UZ`TT2r+weNJUDQU%~!X1=XUE&4_4O?L%ezzr;Sxy-Z`5JJ!EoI zUtz_v$ka?*agg{FM?)cCLxRVfz*yIY6jVz`v> zMAN6(`)+Dw)5==*Vd~W>P4K z*+-GDj?izP;6{g$ldb^LKeVta@Ypavk zDcbyoawEPJt)Nx~=_N^YuerNqrt5EZ1+Z5gJ?cCI=SFXNa zoAAX2b_NB_Qoa$)KtsIiqT70Yz^a4KCD#bAptnyT_Dn@G%r6sVCLEa?B~2Nux+eEe z_fo+Ick@Dbqmzv8hX2{ltEL=whCcO4BTFr+iNO6C%CqK)VPfzvhlh-Qcc-@BYjjji zJ8jGJ^S9EfM=FjlgRN|bAt8-|05#jIdtNv#fe?2YF zD(=ffCet(VR7V`wAXd`8Y`ZysAF{>Pn6LF@9xat>aR=yeF(nZ~bt^%}^8_k@{3fLI zD?**h!nuv?;2HHOg?OtCnc2-RUlsFya>5kLEo(@vl9AWSkK;F!UiaVLn}O(bMy1jV z!H#12U7eC!b0;tnzP;=jajdyaR$>8`_stRF9nBiyf-5M@qlEg6VYU_rsyRaXpY`l( zquoDp(R1ooSreSq#=AP_^&GbYQB0H-&y9oE%8vUN#A0UcP;sWSYN!P)?T8(nlKf58 zv)iP}G!L31-J3M}ruIkA8W(Lmbd;(v>rLX}k84T0CnBcfR%lzM80=GT?tEMO_Wgde z09%22#D(fMxC=UigG$lvTc;ZPgict$m=VUOyLfqQ_*x{PgFlP5d-}r|TmWmI?p^IL z*dtkXNLLUhw-=MKou|mB_dUNbXNPu2N__KG){_(SQO1#H#JCpi?QaFHTf^EQ@aK19&&^^Z0@;2Om zt{Z95eKpRke$0nQHxGYW(Du-$O;=~He1h6IJ@ew-8Th_`jFPCfK!(aG&^6gs_c;5? z)Li<~I1SnrMHEcLsfSGp3<@SNxlcQEqNzL#r7`Qd%M`)%T%x~U)@$t00{mcUA*XwC zZ_iA=yOv6JTHJ{zDHIFiq3}iqP!se;RP@yuPv#FyBzhErDdb=T4hD5ORQ3`EnwN;A z!4mL9=B6y4=)6!*MKHupA38od>y6*P5Hg6NF6DkfEf4k%HO=cXb#GYO@RC{Vyn$eh zsa}1lmWC#pOGlM~z3xFx6zYOpOc+r&Rsx|-tkWw2~B=2D+Y&a zM~M~61umVDW+OK3+=a&pSR+q{9rOw{&Bwp`TYC7C4}{FSZU*1NDe0c?)w|+D=r1zE zzEk3O3uO74$WwuwJlc&p0@zn`ZrQUq3p`uHmnoad%2$=r{aM<)#iF`ypW(5GJyllC z_s85GXr;@fe*X}MtaCl)yURsQ%aSje|E1mp)2luc1N*}|f*AqTc!c;@pw7oZ?jd^Dcadqlzzsdk^K}4YI;&J)xPm8LkxL{Fc>s;sg zG`X05b)^mc(Znv!_#)3^$|!DS6gC1;RhkB7k#w)-!@ z-*1CqI%$_c%C>5@vs+N!sQq10YVu37iRo z^`CrdFNmBf2rGh;kX8FWiAoR9cs9+Wh?HLwP_sr6Oltxk7@9_YT?f zJm}5eh%L?(Uc$xN_aJ1&;YClSnQxB8GCu8CNgJ4(P#8H}9(YdWqpMB!B4+?Qs&NnP z^Uky=%g|_BcK^6GpS|#q&^@Fex)nZL1xJ3c?0yqL66w&YEb_Pkojje*`|_rZad?dO zuC;0!;*#ZHUIbQyqX8C6-y-L^BcJm|XSY%n!W1Th!~15b4!tgL$Xmy7MFAp9 zn?=c&NN^I(i(9)ACMD2@-@sPh6^;8PUsq*iSm5;6C&K(bhP{AXAlsKU`5AKFRBl{* zsas8;4k6ag>AAO?2k^I?J@sg+2gt$V>5Tv9lzb^6EH->ix}z708^WijEpUW4TA_=#LtQCC1S`&|y*?BL?P(f~Lp!YcGgx_Um1T0F?#w`+P<|55;+~dC&(veyQ%lj_ zh{M+`^CD)PViG#zUJ4I65(z;{Gc+~gjSYxq5iysBT4_N6GH*&3`VPbk;34kJlpo{( zjF0F4#+oO)LsHu!k8EznZ?Rg`M3iiZ)Ty9=RbG_WV|F3Rcs}R4uJp>gpat)QfaFIf zlut0tGXrelHug^7n(ey$Ai`N7~rW)l8}~ zJGb+=C{&vxhoO_s^FKz87`C@{E~bmUf(DC>A(fJmR@tLaAICnqGHzIs$)YXxE78lO zfX=*ttoj`?FO&`m!&tt5wqoy-Rd@b$Bz2H0!2NN&Z0Ndj)YvmuvpoBUW zyC39hj&9uo`J_Y+dr=6%S5>34%{FR(>h+G`XK5frmfoOUlksR~dXeRE>bjaGl3->};sz8jp z6JLk@6tjT6i7p03b@11NXg9#Y;)jW}_FhM3Qk{e-ry4O+;9QHG7$VN|)~&WUz-m<9 zv5a&;k7dC9o}pZQz`Ul#tHtx`U^(D50Fb)KaZ;DM55PV9G4p>?R3mGxww+dAxEeRbfeVZQ9&BaJiUXDWG%D@fQQnt*ei8mRUD zxUD|#Ta-?Mv{>|4&OdTqQeao$JrU5@s%=K3w4&_gHbJHOnGV?d9-&1Q4x(XL)mqFp zc#O8A3+=diV}s6C@W!9zofwR?`C8lRD3f>%xG{QCP{?GeTb4hI9X4-TRi}7CS&=x* zHtn3lFVluw+@jMzUoo}WE<=VWOI6K+4HEmCqgi$Hg>gSnE9|%R~fKE<~5X$^F8<@8{#{&wEk(6_rNgaZ0CvMvd1f( z!Fgk}e_}<6x9cT`scf<-^cXL`u^0F(JeBPF= zLrd3y(Z14rwyka6Uwn$*>?lHF)fOp(;~LSka-UccH(N9DV*`Q}Xef z)IHvrJ26qoT?sjj8t#_;Q3`Ewfxx8~MbaX&s|9fL66RYN#7;1NIQlzz`7HAg{{qa~e>vTcI?>GH&T*%i9J7%$d z-SSpyaXN0ytz+T7H^KhyiYx%1$f1P%4yGFDYh>@B<+!+TABUjc-Z*N?Wn1(V@eMtB zYQRcZLkSnUqGV)_(y|7J6WF)b}bRVeAAr}MU&!kfC9pC8Z5 zZ262en=-ufsvADvd&l@B6mKTUiDwZfyXmIZ00fAMT z04iuzeWN;C!~aXodGR~dWDZ()&PF!TIVEtq;6Y2a79HJ&_!_0$$9(N6KuC+a!e2^DblIfmrjg=l2 zFq%3qx~M31azcEPVZ|cvvi=NN*;;Ok-B%%R@9d@e%W}Hom)Dcv1U;$A6opMPxlH%P z$cVqJ<#z+Q7_R{VM&ky*jet%{4>$9Lp-4@Xm3h-!2&^UjdhQ-gEg9u-8G1f9Kp8EU z((gKjs_hlK>Z4dw{dTsE;d!GWa!1Rlz)|YA*;)BE{LoW+${+6)!S;%O8P5~>#kCS0 zPuyF>iz+f|*&<`tILgLzZng`~)Y`I?=)^{A^E8Zk*o&z8AFKCnuv}7QpkwGbaTf@&qzi3 z6{!OOuO8F$eUjD0g!16qnQNB2y2<<@sXZ>63ESfHQa!uFICYUH{Q?sEYCGTwV)D+7 zA+gF){iqfAdLff8JXO_}lD6#3El6yVmw#nR<2M*%^{ixLD~%9h-mIv(2{Yre{hXdp zqoGVumNf>WDYs$2UVe#$a!WQ#5kl*3S@qM)UVDQKEN#*lT`7e(YxPU!$9) zaVAHrqKkAT#j)yKwk_*^O}{{JYU=3qcOQf_HE9l<%=@ zXLXc68}t1m5H`EEBU=Q=u+oL8TKlp`HB@hmUa)GB$+|uL1>`{TBJHi9&_)@fW5f~V z^ZUEg)qJz3DX-ZwAcw};HUIP6C_;zhJY>+}(Qdc@xK7N!pX!rUlP2$5WhFkFbm2Pe zQ*ceqFt{xlEcVot#fQ`+#L+N`-gd6^S-{s$3C|kXFN=-IV1rIznQ_0RR=uoqVhn_< zT!ip(8$vI>D|!F9rJ37uQT4>k=QU|@go>O_jcVPIp}ni98zY=rt~sMyUN+4B+kf1{ zB?8W-3V}c~PvmcFm`JWd*Eg%*6+~$W$~ME)aex&E1SvIY{kQhm7ddumTR+(FKM2wd z2G|@;Jnc*jowYf-uJYn}_7Am$`uXmR*11{f%>D`)8DN524LrhS=v|T+ zNX`)t`{zCIjo-m}i;AH@o)R>fh;N!x65nfOl2m9PPl(yu#F&_aefP$NK}B}L(2>$n z!qTMod*;~=*Y5kxJkhOG^2+B@j7##1A+bFY?N>X`N1@EmBV544tT+boU`F|*Xuuzd zU-UE-xa8HZ!+Aow&^xld4uLAA~pePw%tXLWyWxw{XaP|G1qm;DduA!y76j~<-;A2pD%3%H zXfC`3=QSBKvG5Dm%TeDN2tX}9DXuUI;uZ#aF-&vW^*k80H*0xF)J?Mr+vNeOkp2O% zzVh4zhf%J`H3LE!Uq2MIPb=P5=?gK$7>?g_SLsiI9;p_Jv9Ly}y(B?{MO!q02Ay-m z#30eMYTMs1?jOp{2Z$xFPOs_ts;l{pN9at9%$&`1y9RX#J&sDLLcjJ0aAQi({N2=5 zv4sw{ms5)r7@-d=fOT~fQjA9-As^BB*FHM)$QdNtVs?)PB?Tr*QMBie6{HN{(g7+~ zWGkt?&TXC9yDz7g*8KXAzEMNGJzhi1g2z&gnU}d4YA8mddCQ_N(>Wtj1;`j}3ORg7 z`ln;m)5HA$YU(_rgL>jv?(;9_Zp9NBcUm1D8`}Q4P|#E%{b!X<&NXHCS+C_yv=jSJ zgl0f`z^}9jN7L4jnIVoXg#KyfjaBx)VQ9oEM({~VfA%UfJ1J}$8{ajNJBt+<UQl`J>@J+s)QxKL%^bL^6Cu5U|&iLQ7X#~XP(KNPS~ zxXj9orP1GnmX{Xv19W@XcGCIGpr-d5lkAUEojm^D9t6E!y^|vrjfKZf`SAC%?S3)K zW=hmo=<;0ZO+fO@i+~rmrkZ2cQwR8&@peaITIIqo6psEZInNm zxLw=@B)Gzi!a`3q$grD5>eXvj_*A8w^zx31IIG)Qx~=p$G9En7!5AGl(r@$F_y9Xh z&PX6K9V%vdcA`H_7dOT@wjtsrhwYXB4&xRfjtLJZw!oC8Sq*NX7GS=p4;kPv;a(Ru ztVTEbZF_oIQBw!kjb2}`@(FadbMgorWx49)BO-2QgtF)2?&Tbl8TQWE6s4Sag|rfv zBC)m07w+$0L@rY!*iP57qwiquArb(5(XQ!DN0jzPSAUcs%X~d}*9EuOv$x)l9fA@b zI4?;r%fE*$NG0=>`n)h*^bdPBf`M>(G;_P6K!JE$1}J^FPVt|K<)tW=lM=7zX|@YV z8X5`@_c#0fF(n<7FGL&oKGWQ&X>LX&1Ov$_tbINQyGSAS?!(Bl=}8Ny|1>{9B=RDpk8$Or2|BKj z67#Orr1P=RLhnwI3QZre!n2}3vua(}B;4B-I!S;oNRc zYg_~(=n>lMu@?I)qqjeM2=zm>vyMQGdcYa4>D`I>vbElbM zn5?z5=?h}a9}+Iv^}~y6b)Eo6Av?+68+eglS7E;CDC(yWpd2gi|3a+8jchOEVhRRK zT*l|k&c4?Pxaa7*!XY2KP4+%V>LH%EoqYLhKqmw^_hv>&O&xHbjB9J$k~VB%W(rzJ zHsK~C@+gj?IxAmuuimY1&cgfUcG(fj zXd~32ijb>99Jn@}g7ep0NteaI#<5=8mZhf_aQOb!>tVMn9>w@YX zb>$=@+`OjXvqLxA9FR+&2)`?-gd+ zP=w`sNC1M6vTex3L4u2l^vm)iv;G(OawK=Idv8`Xir`guqCkJtqe7-><97SIo^!FO zHd*f?)|c&kxAj;b(d}nN8`_bXBMrv~1%G6M33PZ;ac47<0q`6@9n7u=)L1+Y7&r(&N z-IoF=(_KY?T8N*ar%eoVt$ketd&8IW(2_^0xO4br^J}~pWl)?72OPR^E^tl(-d~upUXg89kVC za&v82ah_%wVsc7_=Nq70yYG6vOt<#XULK{BQm8BNg5tCEH#)q`^=jPjbne#HZtv~A zTtpuA*h4d=tUs!IH&T;j%JU)jbtE=m^>%>~No4=jV7h!KKI;3g@jji&Xn7)9WgfD5BQ4wHIHm$tQ zGCnBs!D%{;qAyGpu`RtBL?(>^s7YXecN;Q^@Jk@M9 z!1W98WJPMUUfzi!{1-MOP`;T6(?j8Ft#oEt#sKuMi@tN~^S2C=#K@%m1)8bKSr^9 zj-5_FW7x(TB8N~HWQ16=6hMA(5+CYw4vf66YP6A#M8`hFc?V?eUjJ0CnFAbGpI&>9 zDG}gdNbg)fwiL^FINSXrvllxY5O&2M`_S(@UV3i}UrvAYvL$oY#QrR8TKdhADbGfX zMl|jd(oOkq6v054YeJ3fkjhp2^P#JkDU*Z-E=#u^fW61k(JdU|iIQVp);)+8jqI$E zd?sv#-~8+X2Jm({Htw2aXD(!hb}PkBda~?ngadGMS6lc0`UV3=%!xg|vfN}>x}bPp z9^ytTr@g0@l=)Iojb~ow^fLeJ;%!;G-SORQn99*w$5#;NyPl`;TYe7Ad?}c1BqRW${N1j}_-Am(h55`#7kE1^;|T z*7ug@Q;|llfCC~$yp3yIdcW(uRa}8T`HulVdwCTGi%@}0K1hsOQojAxU5>`tIC|Q= zcD}sL4e;*K#%Un8M)6MeHDmLSH~Kk&A`Gj;qDvul*hqpmpo#XZ_MJ|-vwD>N_KydE zzP$UQ8;c2A{Gx1KW3uzHmpI!%pWnzOU}5mi*S|;lpviY6X_54tw7q<8nrrFo%Kx{1 z)sC8-sU!TPJ2dsjJ6PRYi;-^qj{0ey7Mh1xH9+VzZx$s@9g~aXq_mmabXI3ONt5E6 z33BXf2GzpynYBwC7;=U1X{|7WlP<5<`ys@dny>287W+(K46PMp{W!1y68U1olrL;8>PR7f z(I#JR3O{`t^CujB$H1j!pdSe9|KpShdr|cYsJ5QgZeKz~7l_N_l$T^b zL*NFg5HUI5w$Vq4$9Vf!K|N8!VWTmSOVr7}3wL)fGOZ-jA^fI^c)fS|z6?4WxpLzH(?Wm`3aauq}2f#-(XZb}_}5$8z7?;Q1-I}Ixo62556VWj>z#WmIo|KxbbIlfKiQ8q z8e24ux(zgCPi|c#^hQaqUfOl#x{As&YF#0c$_lW)-;nblMjb@ zFMYU$g=RghyFjRibkLYp488j1-2Jbp-TU+e$66B71Iz8@}ie*m@ zjdNJJAEJErZ(jF*TG;mMk>MPM9kn>$=|nyy=?j1ja3G_tqX1(rRF(hwy4!bh@P5?s zYcJJv8g?IT@f(BZIiE018nyDWa?h6~2Cl>MRi`ue0M5_TYD+UG7Y{zdGxgeS`PpaL zQ9>sR>6vrXhX=N^w~IUvGA`ceESWiHblX#X;V`g%1n{wJ8k{MaTst?wduo>6`i9B_ zMhmGAxVf7UAI1uu(Sx%3Pd^O82OvhNm;gmc2MG5*dEKRhn#37SO(sJc zu8OknMdx9`z*O7u#AyR{*ZrC9_nPa@y}-a$OcL8m1)Lo+hr`DOQqh`-^s$Ofa-%n4 zn*iMl#L<+$lpZFamA*g9wOszvmhd#{Ge^J!Y7JFfeP04 zUkj&1Bs!rzr>%!c)!A~Nwji8~tf%*fZ$ZDTB`smF%sd<#hn*2@VXM`t4l{aRFfDKC zW(1XUl~^uhM)B)+=>-!zQ95)(!Cb8!%0wAMWop=;ZIG#qN2SgFPHHMF5O*E3$RIyn zAdtVLw)f@+Jthn&zk;*|neCWiO% zgVCQ_T}g@QxXYFKt!iexvQ0n`s+}?u%7(zm2}ds?(&n`|mONT**N#x$wc1?I?96nH zf@HWMk`X}iqXd?C;Ebl*jTlmZa+|ofOG4}JIL(~&B*PJHXR^1Lx1Se^rm&)?X z$yIU)D0;%2XE!}T|4LLgJ8mCsudT+Tg?)s@C-^F*n^?{|^$XpT2w5*mqW%Ra^n!q_ zMor~QWk=Juq~_AD!7xhv$ar9y|Ic!KOvL=hgyw8djvivRZe zPNFBnG>v+K>M}36xUk+M`!<|V*Z4OR2&^WUR{JmAU@`75qj`B)&6;=$g+;`(kP&pis%p zrnv0{c^ofR z_1$;bB$WUUQ9K&gM#~=!_&!dFxkJ77y#pNDa1Huv;G3fNRKpEZpUhCj)(+cuTq8-i z8O#NkJF2Q~Wdev&rJp58ztzSmT2lZEej>K@O-2MkA8^D7Wi*Br>U3iD zKkm}O5)j6 zWYm7wzO|p;lf%a&ZqH`V9?ijtA=|oZy8Gx@V5@5tc5g;6*bDF`8(`)8kV7w%&XCkn z_Px>#SCH*#C)@pn7nwP}n|P+CeyoVZgPZM=Wv-6`Jwn9ip_mEc%9S99%(KC0Q(ecd zsUKC1o8o7g=GiKAl|Le+OLDr;qosWF>0Bh!+86)9Cz#fj-EG_#!B0UH*SA%@M}67Z zptK9k32Liw$IF*fA`l%`DwJ{&K)Odh{o^l->WKsEJHfpQf~|%24Ee~gv~R$kH#$PW ztramp!&6AhXG6)SL{>)%O4x0)oiq(!kVJR_POt8U5?qg75#~#%cmPo!PN9s_l5E3L zh-=$!(Q`|OU0&+pGebTLO<3l~57mjUM3IJl;oya&=OvQIDR8%tOc%D1fEO6(hXA%2 zn8a3NYnu$n3CAni^B?T$OnH@M$ZQhb=6X3uL&@#xF(4N=$z6J z!nMz=o51|6sbP^S$NNeJ552X)+z20EuNRY^D51o&57rIVlLp(0bCM{v571`Xfzor3oivr43-u^+5ya?Qu32wv~8IFe<;Ey&^!>TLJ= z0tV{b(Z+!bMvF>`&nRoN-QS9`V9mXqHPWR0k`@ZOuj(Ehdsl&^H&?QsT+@7{8#_BE z)9r_KCpx9!+3@{lAKj2M9&Ba_zw|t%(&~kdX||_{O)r z8b^J<3^-L;_y+IUc`;5Is!f``i(TS|a63lGo4Owc_j{{TD zWTk50oqeLRyD6w=_~8tD?34t9l9)`9qa`j(kkXWPbrdhV!e=b{#EU`PA3piG_=!D@ zX5Yz2o*M6lb)TXg_p0RrWHly(w=gVO`@Lfwye+TV_6?jNDNBrKCV}Wk??K@m_$hAT5Ea zl(6K{kWO80_alw)E(;46g^X4Vu#2ws?@xMjF7!q>g>8*oPSPmG?wDh|X7A-I-=Z!* zU6s4psICjdKm`WI5AQEV>^J@5=+$6yG>Js+CEnkb9j`4-Qx1g()pj0t^hs_YP_H0R zL8s)2cm%N+TM)4<)F2^uz!IFNc{to`L4t$dvwfm2=@|fv+T&_OVGu)kLW)}D{L{TJ&{xBP4V!;13BEzj*EMa}AS2m{& z{^P>FCpd&bbmDHT04$TBX7?d z`DSn)YnnnRRCy_iCJITY-Bx(rMca3m>z%+{7)$e2Qr`{$!h^9ARdk>XK?%2hfFktI zZxWW1x)2E^bP9T0x;k`!>n&SDCdXnxTcpRsb~NxmBDsxCQExMf z76Bm!>$jaPi>8`p!zohV%h>KoslKN;=AK_A1Y9FI(l?4b`-p~lR0CK}G5M0C#hvoH zXSAO@Hcp$q*rd;l3FCoQEt3S6m+CL&q5mA;w8++v<73Ac?xX|UZwJr7sh8~hboeRVz>p2x5sy+ zYg3~0e~1i)8YkoN?8 zP{0Pl8r*g1L`4{3suZn6=!{;WJ7(s&8|D-Srh=jE}c{#?R(X4LUyV zngVu4<&g1kr(wTZja*&bAMo$9;oB0x5-~G`Y$~x;rC|&X3|Xb?v^yO+r#%omxV;jY zTP-?8eVIQQ?&ey5-hELh3Q&S10s}{u3EznmUmf)uM~dBXaH0;lwjo*n?6|&}$JwSa zb-q`rix`ct_UXZzwqHf-$#mv!Ny@df3ch(aBx-bSJ{pmy{}O%~_oe-g)<6@N)hBNqc>|Kg=y zeL*7t@8C~xRJ!^lF?SLBp3J~CnIONg%x|tRKdS}9L->cGz zn??M-#r(lAdRzr|*VKKk()hTTb&@JuYT(iDLGn(E(M+6muhMi)I#EX)nCn0T9zY05 zl!ElI*uB}rf8s%QL9>8A*z2%H1ANzk_2m$D0G2Rf1{wtalrJ?1&i2Q?C}Rf|KEmPB>oxp z3#s>u`bqW&mInM^pI02P`7QsL$d0`K{P`T{I*9vu?3nWcYxVNq@C8vHnZZkY-*C|V zB>p|)za|cxB>aTH|A0DkPy26>|AKsJ@(;euQ~^K_Jn?bzio%4*p(_p2W!`M<9L)d5 z`60TCx{T-MCG>Fd79q@u)^14TJ$itO5^K*uWvG<)3r#@YV+zs#MA(<}6A3>~b>he2 zMLHlAW>*QO!F6%etD)^f!>@~`{>JsRHeVf0p1hH=(&E+HiIY}a_$HZ(L*(|jPbczU zl==s6qJP%ncik;;?D$J5Rot(5{k)P6 z`R)&svnszU=P{e_wfGU-|6D}qKh*N&RqC4Rk?y5AX{DLoa=$sfpKJ=dvdCja!Z&%} zkGgZ`&5`>-{=K%H{~#W7V_I3q;c^=LzgKpt|4A?(M%K=4>{LmtNaBH;(PN}Wc-%=kA?Q-Jmx}3JLszY{^uIY55AzQ%o;35qxWvyg%x!tH zNO)~>)D=D^vu|wQkW-ZU?ax3BO3C44bS|S8E4?x|WU;O1B+So3gsp`lSbfwE$=Qnk zO(cH<*kYsytHB$~z#GIL{rUUP`YSeiuUB)ii<9c%tE|?O>;!<*&{AF$CX3}j%-FGo z$XTS`x#OlWd&@_8K+m%ksK7$yAL0DBsBB>Wi|AMNh~yz+kZ)u0D{FvNxO?CaW!%yc zrjvnAZ9t>o$hT~@S?0{H+m#e-o(@7!aWtGj8_i?ox82@cYSsS$`$X+`EdSNZ@vOsY zMwl`rW(C7&669Pd30FgT8HeBok90f+*=65y*xsM9v_=!1LKFN(N@Ta9_krHxRE_^Y z;dl5)N^Roemq7d>7ZplNiU*R?z<{Sk+h5r_|4j!X{!hI5UYX~J{!45A(UBbyL5M1% zjo&u$|3ehE$p3J!_7aPn1^%~+@$&z}SznV%KK1|1`_KFCPnkguKxlyZA0qIGqDXjt z729&J`Tst*Tk{~t{))+ev|kX_^YI^MT@EVx{2%3}0gp%i6o~&t@?TW?f8yf*Qu4pl z^M7UXe^}E0e<{wA8DrOfasmF=9Q~Ka^#2u``PzV5{IP;Pb@Tl{4is_h=k7p^nC8DV zO=`&BN@NrMUwaIH0A%K5XXJ{o*k+TsAc7OyN9leG|H|Ij3KODAHoZTdTi(%dSv>E$ z=98zA1{)0Z4nd~mg!u?u#z*JoD)BPLTl2p+rY;EKPj?W8RdZ!-0}|}EBuYZ>mzh7! z^=mcbUg}DjZI!z&$eCdAny|M{?j-MQCASGz+zJP(h;H{5>VQ!?kA<-JN^Y`=aIgk( zkwtg5`a-RB0}=*U{9n)y{^b%adRp4>|A35x*A0wQOIfzBTb0GmST=9sa98)f%{z01>0QDVSJ!CsYt)l(syp%rGQ;z*%yxjzpQQ+%%tzVK7GVXR zcXMzvR|x*gWGRII<9Z><@XcPJ1afe%R(?xun;(DeHyAK5VBFvTEX`^!z}Ti9xp;lf4=lF%#o|$brg&+fujDL*mAPkthK0Tp(i9R>9%CD zm{KC7(oaHgAqmCg4HR$40?9v&42?>e8Qi-$KOP{DR;OW|)! z3D5og9yUT4KaN+QrPK==w5s!|lEZJ-eFvU^&&LeM6jM@v0D{zkADn29Q#f%b8r{dg ze>nj@gY4e^?w8Rdm9;@-)}oQ4xw*xSf7?=)Iau|TJoU{HcKnIMg40E`iH}xz5w?NY zK0@jFT3Y6aqh2K`Zt^FQ+6pstatHKjb(rRovakM)fDg{5&WvFXK-b%|rOIdTgOXx2 zB0A5&odi=y-`vN%@;~e*VDxah!4{Yll8#I9&4>~oGGPwp0OXovs8vI|)d{Ii3V z5a9x48JW6mhG{DCeOu_qF?V~Gg-?#m*I%h}ew#=W5&(tcM1}{|WnZQqzhDInrbHVD$?YL}MjDVP-Y4!`)YPo#`u{1b+&~|lX zP;ulyiSMvYobgc*-4Dd4JmDLd-_FyVjwGj}WIGi4GiGka{sXbPVk(e&np9#DHKK2?mF)RN}em)-c zf95DZOxvz6VueL)-RPu}st-l1QWN2?5~*)p`8e6A@_#!4wOy9(9}Lj&zbI_SHwdwD z?Uhl3t5EKLD;JN>vVKixZG)(QjEg>|K``VDM5k186APCr%pc}`y>cAvC3%1`Y*z$h|A8qAaKduAZ2p42j`N=zX$aN*(xb0dA)aM2_9Eq%-9*ksBzj{^suG zy0`Xrqw(=j_aVJ>Eb*wOhyQ$#r=^~+rpJn*ANRF7;0D^`oA6038Grr_V^HVYhda!D zP|Zj8?~i50soo)A@kUItUv*96A+Tig&E6{;VlqJysDE=a=I#{8&| zP8+=FI)bQ3HLNO4KCVNJB`|@ZzjJ}leM7HI-)N9$p)P-ZIveTDuWABx(_v@ZeB5z% z@4d^VEHg(%SXdQ*)^yM?KFj!!%}wB&7kz(x-@Y?ExUdpNB;&dM5D|O^eS(@bkH<@_ zQy?F|BmD6w2;_~4ln*$3^G89p^ULwT+r)P%(vQ;t?Z08;G_5T2y+^1>l5xKHcex@G zT5TNOz3G6uoQNp2TZ20IZoS$gR_u`ZWUK5R+Qv{rmANqbgb^8-wop17`6M{ zF{tfzzQ_xr^}&J8ldy(?o+{c`UE@l-GvT4WSyvZ``gAk{ekw-)_-J zm8)2?J}ECk1uv1rg>rFsC{L;CtqR(2#YQuI62D|c@@d+!oD-33VS);oxN|E2mUFxw zeW4qMbfPkUP(?6^+`x~yT|;Iv?64*J0w<>USHgOV*d;+KFPpW!aru(4MR)6jN?*dn zwSe_`K^9sIVsfHpj>5fzFfhdWB;eWeyk(xpU|D}%+W!(m>P2SKMb?JIgHM#eSJH8@ zsx1~TrsdhC^JoCNAT!CLHjj3ch*deIFRu#yQjz~01bcRq2j8PEYS!g$!wZtgq78*B z2TLJvmVvWv2Je;_afgDBYbSoifu$f-Oq@|)IlJI20OGoOM_ZF9uL_?&cmN804dk=) z{dft<24JjHad?@&^9uiwyN(hblhl02>kzmPehC8ZhbVVVSg=`}nz`Xzf}OU0Tf=FE zY9_9@E&iGrr)5mLO$&u4%E1Z|aaR+)*z16eKk@OIjI{MG>f0Shbf~TdWhA-EhC(`1 zAz=T2NH8c1$dAP;USiOoP#_819%uHg7(D+!`tG}m_E^lc8r(tp=DI4mGlL!F;KTcP zxzqU+R|k@bUz{z;ZwC?md5*&eHem@q!{2F$%HgGIrJHK~#!}KQlawX3OR{gN~I*sHAOj@X#_1y)kJtQQbLH<*E7ULayN1dGj$6t&66Oe$ZZ75IhR|o zv#+k9{y*7G*b?*D74ho=I7jGG!UWYGX)ppvq*?M6GoHGNM(AtWs>SSDSqY5 zJ5eTIN}I6e3P;A3D;~?*vvkl#<;pv!sQOZV3$iQmlW?PlDoJoFF%86c)Rpg7TG*$wxGb^Q*vTE+Vi|hDj_@6{{JqfBYFEAvY=MOU) zAdeTejz(&hj=r?MytZ~2tm4qV@nCxP|6=ML7&MEPCDF2N+g-M8+qP}nwr$(~%I@m2 zU0t^I`p$jl%>0GD_F9<{85seRR)s05rO2wpMmC$OCYNERvM-wjY!3X~%g5wv$Ao8a zt4utpHCbVX9SXdu-gQ|P1Xg{{O_$SY8K^k<99DrO90t>Ho@^~m6U|*&KDEx=U?RWt zt)pjXy9U*`Gk<*beKW96OS8XWNKWo_AVFOorKV#T?-}nLK?*+~$!UN{h?U*=U$ee(=m?pVV?*sevm$}bv2p0O%AG~TzD9~pvATeE9 zm)_yKa4PI-`*pt#0o+Hf-)ZC`M^8R@LHUMr(ptSn-@OR#`~0~#iy9H$`IA{kFk^zt zb}DGJ>D_G>>X1c|2c2tgwmH;UXVn>$eDRP@%%9WldfHVD`1R}85F-=61iE0dy=Z*W zRPIQ%aGz~G*jee#xueZ01*>*#087A051FIW6q%U*dxY5+b-stw#D{P_1PIpSwO|OH zzj$m@wz=VZnddh@O|W3W!=ND;)|1V1_$(Y3ah-J)#yo(F1)pVNsu_vS%iH9SsXPP>{mHC!6i{WT0uWp!YO9RhSR77B5>Dw5@(E%otE@cAx5}%ZDt&c(}c86^!^*i(9*oYBoHtf&{UgpHnXg}qXO_b zF*w@1c8<_Ak0lcufYHPQwg?1x@jM^$P9$xj%GkPLJ-(g&H0^u)P9v%^Fog8SsT4t6 zDArxYYRUBzy`)B2pY(TeVahiM~ za{+^aeZX7s>v&(w0{zq3AEf|R^)eNVeAO5scRC6ihm0*TMT_GneEZ+7Ary%Cspb}bHX(u<=g7yTE7QYZQXbr8ci;wYSJmf5?!ERa{gwD!H+)GD&Z zn-@L7y&&94(@XNh%`l^;#4*Xq3Nasz|Ja!xuw|Pld=)uN?E(v{WU49B~7B*66NRIvDH={JfqH zYDVexIQrll8m5IS+3On_NBs5t4j?Q1y}aM8o>hR*bq;0oq=3-zDgdp2PD~e^L1U*i z{S7DOj=x-3f0nlyY6dTGX2DJ@lc62$=B8uFqLKnSdXg zzFPGMUK;BmrQ@K{90mFaUr#qzHlA|Pr>_vuTzusL4##8$1bPbIdjcGYrJX4O; zCvjA`BK1+}a;p_DEQ_APJ3wXdOsw`8ARMHpaYhyX+iER!-^TCq?xnM3oBV%q!(Xyn zVzOJxuzg9X#rQaoyj$R%a=f&D9ML559Dq?f844^iDUD#}*I9ie`j#LDpQS6VY8g_?z zxms)YHr?sNKOffRCueBSA%-i$x9eK=I@l(hhn+wjtJ!|pF%1A!Zm2e&>18(UTFf!K=8M&Et)5wLHclOg_POY| z5Hr9bF;P^EVAe&0sKQg(;SMzQQ`yD_MSWe~GwT_&Z%ZLuQlk_-U-N**O-^(33?qh|y!(pc%nFaPBw^_h|fHe$ZY0&_|CD>(#$>sK3_N?elFnknjum7bK9U;2m#5 z1sZo#BuNh$eM+@kcW}&heja{&y+`L;ei=PxDXnp}NkMrgOb5`L23)TP*B(||6{#a@ ze7IHdR04AT)_tk9))4cq=9a;zq(Lel4Kd@}ryQ%X24Fyb^XD{Vm}@>vcVo#s4WhXk zN{KalVofcG#cWQviu?2ky&rv7Ty;7#LHRaBQ^3bK;@@Q4jyp7*Oy@jp7w@E6j84wv z=OOsin@grIW-3=XV#}1c_P{rs{?5xluW_|1?`;8y?5@XXPaW#wLRBhe_bEY)7kTU?d!iByS|he z?~`IRu**`EV@97wQ!veeEt-NIBKy0tdZInh*opIgYAyEHOraU(-MMRC-} zAQE=!(M5Rt)t9}ie>zmxr21}Ugjqy8@0nqR?i-{z0mQLiw55AmV{!NC{c?G-?7%Vb zExn!ZZ=C<_`dm^0a0)oLW>LYaj+-QI0Zg+${8fKCWpKYZvJZGUZ`b-bYo0@DHh)$d z3V+b9jseQ%1A>@mU;h#*)I_C~Q$ws3`qqLV_K~JRIe2+ZFXMc5XMacG7G3IyFYxj)w zZVX1p&gZDVe@8$6R^LoLVjr4QumXRUzc*Lzqfh&R^-Q`;8aeIr;?tH#S;(F30B7xU z1BO!D&bZX2%z>;M*iaG;g?4P)bfBSb$A7^vqA*B+G8TfzQ6;9AAUYYCqnkXj+-Uur zhE^}<<&Ay4L;vgY_B6v(9G9fLclh>AoJ2~>zNeu!x2;W=?uYQnurm5uJv}Z=4z8~e zL%YTg;4{NUiEgcw@}ojsHz`cqMy(!-M;%oy4D&7L>GS=aW1#0N!+-%})9b8ud)n67 zT5j%xlJcvcZ)r^v&Ic-$=89w^dn%k^m5o-+!?p?Iv}4#`j$SLa6fxCg5(bKCqf`P$ zE@D~1ovd$xR*RkFNUGWTbTDt6IxI^!BgXgzJSP*9 zHpDej=e~LH6RDdo-|N3{l(^p4(v{jr-8WaVb*ejr8v>8{R6}mGMHmPCEJnY zlyrF(G+6J!k=TB0^u8<>^+}9LR3uu97O0l(H&N1z`C!E5C=$PkCtp zC6qZPZ$tb`8$cCbs|25$xeRIB%6aE=n<%yK(56(;M0(Q;h?9SZQ9DldiY@K9pE)bz z(2%7^K~_-vZ_ge{9oVfqRD9xMR8Hlb<{%*s&d`{ymt}{J2L1Ymu+6I@dEBayhm`f> zzn2H5D}X06|rZ&rR9+cL`7b2JNV z{Qm7ce3+2scc<6FmMM=LGX=BJIwPhtHO)NivvL{K2&`W;4Za{jf$K?n=d&Bb5R>YT zNH@qoTXo*C$GqM2`+j#AY&Gpm3rxPBikyDBx;k^`AwatC?&(ptXOW}#@m|*Cu8Yd` z`AzEWrW!;Jyb8nl0n0C>z+NOiUN%FtEJ6wtKUMD@V-yywhc#T<~O zShOGJCp@(j*^;0k2KirI_P@aDxwldvEADGSz@i|SMIh@_%U4&-$o8u<15I3? z)N>XM>wp|s_(lU>th9~Sy-K;%c}FD_!j^Jup}pMCg4yvs{t_p+j@6o!4d_eFfNC7? zp_%6Ts=jVF%$Lf$K;Jf4HAZ^W+BM3E*Kmqdz`Tdi>M61IdY9lB?ixK6y}p`LA;hv{ z$3S4y|INDZ<4wE3I|UY0OcFw5RqUW9$WcfmSP%XTCeegcjj7jY&F$ z+_@T2*aX0DhgM0et#t}Q$RaWZ2YzePbM1!9@I{!`mv~hwPeKa~?)_umlTepJXace+ zoi*L`34(HNVK|GVa#~a!hISu%&@S6i4XlQ9%Rb}^{!jmre> zDi!@el_I0xILVSlBG$Aif9cyp%G*G5MTD$#S8fFw*8WWKv((xJ9w4c^Yr=B4( z{Uf~{X=!ti__YP@ItnY+AKl-@obRH%oz`>YKlnThE9<8=N!9~(8mLB!VY-XpKh#{5 z2Tip&;sXx*x2`)PED3KAy3W$7m#}b7C65#xegp~LFL!wUP2~RPciJOOA1ojU2h0*( ztrVkOdA2x#BJdCEHn$Ur=Y@bd{kC5-19&@Q?V-So}^VM61qQK0=Dp88{i9 zN*VmWpj!Ff2^S%ooeqeDYqV8qk*|KpN7L!{S<&^0X^+G^uzZrT>RjXMokdy|<(OJ2 zgZ#5>s>whc5?e^&GFtVDSusB1IGp&<{^_!TKX0|Yy_OWG;tnvsuCnS31~V;T%7sU| zUiu1KAu_W@fg^Qls~g>ecpiKnYNKj~BhDIN9=VY(gdvnRMYus{KP@TwpQCQ*vJYVp zAoNaF;|kP+3L++i^_(p}Ce8f(*98O!<-PsJ9g#;PMs&RWIqg~HOf4ks6*3lut#X^s zZw~fPe>tEBjj!)wZtB&=#KC^^gRXPohxcIWzO4NJQ#=C@#GjfUN%41q%?y`To5XBe zCOlc1!mBW2h#wvJPjLh>Y#gk8sbrEUnAZ<8L7*imMM1WrvpR44$*ZDNH|$Ye^kv4y zT2rekG{<@9?AR@7oj+u^iz~`ncUDuyB8Ln(4*FpawYMOX80k^kjQ8Va-kDQM!DJdBIb z85`Ei9&Lipi1k8dD7NrPS`p+PSO$srn5QwTbbw{%6LPl*l}RuyyUf0Sf*&`tKXMtA zqB&0Ix#!>IfRCyICKM9U$uwJTF=n)~JKnc|2#cBpzZV>1!+10F7`A8>ZW~o>P8uIq zvqyt&SUxfcT|0G@;)gZfWaZgSOsP37I2t4?FXGHPOU+sv1p&etXZgKsiTj3?}ycm+H~s8XLg7$_)Q&1ntLxbJ6sKYPXl%Dno}*)bn_BEYg1xmbE|xlx>qm-RB!^L*9++h;kFps+V#3gxL8$*h4H zc}?gP91J`Nfvs)mOU+a+KZA-VY*B~G0&N_cMng?FM7n;06iiQ236s2$5YB*^0hT$M z4H^dpmMqvOc}0EXUV-3aATq(O{`Oz~7JU$}h6-y_K z7Q6=qU&;vxOivVm?QMYUgV$m-D^fR8(sQpLSZ6uNB%*uPd3M*FzG6t-RO;ekZY%f_oR0TAjIE&o9l>pCI9FV%6oJEn*ZED>_Kdv0* zjOnC)Vg-7=$89H*7VMQJVaF&smKLYU8kuaPDBCW|r<8uG8`ah3hz<^S3p!rr8&Z2z zI(1J(T>|G(EErWAehoX-#k^(E8!Q(#<-$ADZT10#9J8hgajMxXMn(seNhMwm+BpCo z=m@`1A3UNiAFnhG8zY+Jr6xIAN}IwgZ+&}Jv_d02!k-w)JA-}G`L4euIV7}QpS+AM zk}ry!A(nhnh56u9UXg9D0}&pGqfN7@y(2bg3H1k}P$H8kI!Y}0pM`mSb((&;QRGB7 z3Dnd=EwhMdlWuIu=Km-`{!iVBAPGu<2K23@CS-%r+|l{TdtbHje{qC zPljf6HtkDf48t`ahn7!iMd$Bcc@d;G&G%P|%36RIG6(siV>D5nI7D2x*8Xe z;*R186ig(4Ean>gOgq=#_X990tXg+Lsl;FCvLSuwSO{(^NrXg>K?|PbB!3Wc-mdK1 zo`XwiybGTXtE$wCHJKe|2tvjWfYs9@p%AJj5m1C!1wA#X!`k1^a7D)5z@n7eLlSd= zl+kI?v`$Q*Cr}Qh?_vdy&vC08AvOu<`HW32`%;Kpcg5)c99#K8`i(teT*p!$3dN0F7KgOmj^% z44L3<;xc0&xl*^zxCg_qhDJn=yqyo4zX5|pxnlh(Fwldll#Ea zFwiZw+0AroF-@`AXj{GutM{UI#6A(w}QQOlnvQZ8;Dlak}8Nj6CcE2F03uI;QHKM zUjJ>|HZa%z-br*aDZyc}&{;Hy*=}lhA#NrJ7d#YJ6dKkr58W0bgaooL7x~w00e5W&7Mkl5u?Qj;FJ`zHI~Bp|>G4&#up% zF1-3wToKSv6l2E(O{`tBaW%7VV0V1*qkV%$jo;EbvrQ323alZTo5v+#W3laCCf)wS z^jo!f)?7F>Q*=c1auLF@8a+7fSs6(*bK&Ig7)kaL7#jYl=vzfFwW++<>lGeTR1M%8 z2u&}uSDF1c5a*zwzsOID_?Wan8u2=d~E{0 z8`|tj=e`#(vNQBbBZFg1;ly4z&vdtc0T7T^2U}+$z}XvtkjE6NR$k?*4LK4Yg}^1= zu)}1$M%*rXX3@WE^VxJo zZc3M#K0SoCU}=_O`>Gc-nw=Iw8H9wf6eqGA3MRnfQX8P}z=;z(#zL>Qw6VYikB(s2 zBLodn$mBW-9SH5%og6JXqhbRtT!~&DHpsHEpBzFCv1MU}LA($Tp$(Xf08YM83=L4i z%cc3FkQl%?AxxiONUl0Z0RzRU+Qh)PU`i#S&y>w!3O3m)9KxuBsSl20$T%is6v7PF zUWkFS1$zeQ$9J(neuF@Y$SzIX47t09;@BbBz6%GT`0X{Qj^~U)#Sdq18Vx~4E8f4< zUAg}+{l&82lH=Q4O3bREKK%i}ZkAv9-kZt>mVzG0npyT5jl}r99H&Sur-9_!Tn{pV zH!e6`s>geS3uor-hyvHlIEunWNT+SfIp7y?h|(I-SLQEN4-LG4alLQ9+}egz*Jsn2 ziNkf>fNQD2?-kILX&4=%f}SGIY4C?ACL}#7)I*xXifQX(Jdhoj$%?Xlk9{zym;l*a zLT;H+zIfLCvEk#2|G!MA1bt8f5FlyJTTu^DNl%be(Z2B?9g7o|3A&cJ53zwm*zZ*$ zBZ&@t_K!yW6t9FDZ&X6PTIP&LxWm?t4O$h7099eB{zk*VR=bf4=W%qnYLB>bAK0iB zQ6NyaQ4t>^tdN{0j~!us8^b4URM+p#8zIZDXfaqvP9|{Hg|({p=+o zq{)sb7lFV*&(f4&hL($g3_hlYnZ)3{uFDxC2e%0b04JMOhqnuEiZ%A-gu}bNl=?gRmKp?>ER@0mSn_j@F14Nh)@IRS&jI;dRIZyz8P0Jm#JY4(h1ZB!L zk{$5ljwj9oghLUBfy>9XHxG&xe;GJVeW>`c`LTHx@BW`rzXaL;IXPCS*2Ls71Wd~H z&B9XcO_*NOBL@;J_2lin-9?W&ijn0J7a4VgYW4(Zi&HdMXkAC& zW;{j6`e= zl;L8vYBLw`SW*S@0siEfer9SM=TR9CU98|fZ z^Ce1bI0*T<^7IvbK{jX6;O}Tn>Tl90C_+%U?o{_70c*6ncVSgC^A24-Ss^=|a>UZbXmN_zs)y zuQof7ir(*g`I0MePWRqse144Mdwsj(Qt~dBr>A>#NPN)Ac(*g1}d6^R%?^fD}yx~6F$LHPdb~cXhXSCrR(+gQ9#YjG3RICU~q>RBsc;P!XqE1eg%ag2RfOO}A zKN0Hlcye)nqS!B`aI8sNmZ7PGmA%Oy8kzgWdMI~B+IK1K-YJ=gT2GbVC zJ1g4&`#k1imYrcd<5DOqua@jM#PG@CF?INFcMBW{Npp~r0?SNirR9&>F0%~-MTUo00#zTW=K*gf^sJTMl4BVi4q{QI1URs?3m zKAYl#z%@25CE)(y=puf2N6z66aaTNOI^^6?j-w$0 zzM49b9CaCrp_@X(Ql-9Wu;+V^XMt0KXLq0Zpq8h zgQ&T>Kwyk_M4oY+qlPack2y^05K(!ihh!o`5RC1|Ug>03!S8TG^e`R!{vBS_{F|CO z>>n{dxl$(MoxDoP%1da-tS#`A9Zhzr&4Nugjz7Q-P_>HEx7{$=#p~rJCF05ffvwQc zQ(a-yp~M^TiK##!vC&5Xpa`PAxO0EVR7;hSH^La*GWRN#7U$NiYzCA!21{J7zEhu9 z5kG}Rz>3K@sJ5lRog=B zHc;&MuEz0h$$Sc$5IsWFO2f9r`H_N>5kN0snQf4Cg$y4BPWKy${Cd5zT%(^@lKDZ- zAlewoF3u|?^M2n59Vi;TcKk0iafJ4@go6Z5bc1f|LM9KFEs4$vuCF&F$^L&$@Q5MaIQ7d1D=fw1aVSVOS2-OCl^#6D!p`omU! zPSqmXCxqGVjU$b)u$Nf~nN}UGZGio^onINED=)+%!_pmkeOV1!U(50Py~ohYr}NEv z*S86+KC$=~{NrJ?yW_sZDO1PrUM-3j0G37@8vsKY5-1GLCT`;>G)smQ`s9gbD#%}5 ze3oreq8?H-tymt&0tP$rtce7U4>e+`t1CvYQc6<17f4mO!8b7&cvhGV45V#kh!9Hh zw{Clfv^b)G|A;3XoKsp{9gU;L=vUZe>M%GLh^qtKPaG49e^_3WvPfWEOgYU_weTI- zAhu*F8W&?9@Bz6P^8OguX5Ii+IKax^sX9Mz1CVscSzpx2?Eu=}p+#hOcr;o1p^qV) zWIP_h-jKEccRTxq?ELrT--|CO@*xZ&`D77 zTCyna)z{q8BVsDGZj!%|sGY>x1-in>9fIlY%jH$jD(5lcrCAk{jRynM9cu;QZ|S5d z%EVi~C~>3v;@zG*vt{z`#nNFT9?B$98nx=&;M+}?oA#vT28sq0EV*l5 zNZZ8>>X+6E{Y$rCKxym(g&fZCE^v`AXS_O`@t&{N^FAQ9U!`v9<&m7 z!*L{06vLGla~Hl0^uDSi`!6!UX)}X3)U| zUET5lyaye<)b_uR;Cl<)>YRBs`GBkyNomlbAt}^S3U7r}Gn-uU4f3ZjD>f!)@VVp5 zHlq!rsMLW7zYfKS0{7FZpEe$ZyG-&uwB= zHJep6HN{=yuBZ{MB~NxiHOp+~mQx%h2(kx4Evpn(VULCe(H0%8T5vsT46%ZMK<{jb zCP5^TTNHe5x^imQ>UJ*8&`-yt;d&1S0qc#+X@<*KZXA{>4(G0>;0!fuQ!P5znF%(0 z=N1_x*a&^WI+hl)2bPp-@@8eojELkQj2CbIVWSc-kmBOn8$1`IMaJR?g3b;8qKYhe z4oIEkF(6DB_yOG6diQxu@cRQL16$iRcwe4>+4)ZY9^V8vL1t7Ax7NtiMvzffw-`St zg}HT(F(#O|ixARR#p+f3q@}w@1D{!n=1|tIFZyH!eGoO81*kO|U7WP!ZzvV0fK(ypPDowQ=DW8X~oBU4OqHqKObG7>Q9utxW2< za9#r|BrI%-j4EjOs1~XkEg$XUOVC1uc=yYHtNK5o8j~SsXX_qbSlrB>m(Aje3rYo2# z`2pCyfe@x~15ooIv~ICEsG6PNIq0GUb+Cdd*YFlnBlwBdYfYs)vMPgG(<;~8@k5iP zDGH2#vuARJ3$V&bKG4E^|DFneVQPj+F zsN=RxQqO2>EXWkb3f)W)wr1z09%isv6gCk-ONuecI)YWqdfpw@VaEiy0(OY1s1B6G@3UxeyhYY4H-gD9QMr+}i5jBoFC^(VDdO>K8fNMfwqxv2tAQ zNoy9pOl_(?4V9!?MDbe`8qNhW9%4u15r!Rc|12>WMwI`Vm@Lt=0UA}S6hnpgRYMDh zKq-j81Q=Qv<4drVoU|}3#DbeOxMZ+qn8|Z}d7X(%#REGq1!a`G#`ce(6$u+oo6#~y z-N!I+OUup039S7KuMoz(V;s5YSJT2JSBb2vgC?QSO;jxHp@5VdN(6J))2Q5*^7f) zGgdUk%{f19#--FdKek{DyPh3J0!7kKQMCk_p!*ukoVw;b{)v0s6w^X9!K2a7w2O%i z@Bdm!@K}N_|7zxjZZ^cVTN!e`q5}Y=A93{>(WcWN>;G~G8!1{2U68w?sl9Ccw>fG0 zpT*e_+jtWqk<|6Z>9>EM_RiM!+zT!-J$)vtt$4-gWY(oD?JV}hh9a7`u336i@-^p8 z`J?)TiSAs<>A#nMvg%U0-lEfQll`N7g#&#Y3I2#YvT_c*_!Dld#BpH#8c_#qA>wRD z0uGOdk%{9uXwTIr>u}z@vBmcpVtn6R${>%gq%2%4XRU&GncE$F6 z4Tws25_qfK{86hJt@ohrZ)0YXoxn=wjv^5;AgD0KYKe3r?bBtIOuvd4sOyb3NHqko zts4Q8jaEY#FMb??DTnb)B$Uau(Qf-6^Cv#jTR1Zd%5!wvK@%0#g~T_>8xf?;N7kve z+hUXUJ-a*f!53mT8yqL)SLf~3nm7#rBAY?rxsk*uD5unOS>OFtQU+|#W)lif>R(~O zNvpkVw?krtBzNQq9Qhp(VA{L3K(7u4|GvEa;Snm9ZW1+!fg7}W4*PsDI`I&0^E*nh zNT})Xu#S-pb3}stn_S^eSUZ|lS2djH3Cy7LTB5uzE70DDUUNRBhsXZaaL?>>ld@*> zRkS)Hmi)pD1Ses;Ts7v3zG%iGyr?1>(UFp==8-Z$meUCtq5^2*AA2B(++Jhp)~${| zIGLw-9XHk?2J6n9b@oz)dxd3qy^TNow3xGdX)xEp@cDGJ^o8njP-(A#^B9^xouye{ z$+8HO2eDxHUlI3jnY|E`PSTOU2grBif0v#jr~bI{t}$AoD0z;f{O}>>3xW(rtMh(^ z!r`K+W?PF&ou>Nepmr1@DhONG1pHIf&k354EQL2Y|qJ zptzI7!6ght!up!yck`qE$i?XOn9m*Uz6R1h!Spo#0-(``*!q4w-WjO`a?Hn|h z1}PYIJYvqacMJDF!`+?Il_FZ>=c?i!RS#g-p^1v+M4%W^)g`(%T+WwRWk!%~Jh-hL zE+6{l06iz&W&CU(p zZ%)R=VPRFitv09y8#MtW&?30Yy5Hd#1pETS9S%ae6D>12BQ-A_qwiX+S%Bt;O}7W0 zdEFDruuoAp`FV=V3e3uB>dy2~-xLAYkSp~4Sms6Q67%amv<~~amtS)!VL&^4s7%^G zvS~{ra6t5%$CDIr9W`J3-dcTIe>t+z;cHn|*wRDi4bE z?Njp8V@Fxj4wR)Q=AoGfL(%=S{z;;*J~rRx2J}lkIF|3#d254!YX}S`Z%Il>_>_}e zu<=4tje{k?LChpc$gR1@l!JDO1_bX%IT=@k=HZO_Cur1!gj7p;r-6bcQ-R|OZ^mon zTzL?I1a3eA?B4AaF3!s7de}cRdxNAMSl6G`laAs0_5`|(&`2a8<8k?E;9EQebtNZ& zcw8B91CAXU9(7Q&Rw3K&ypvZsXl29`i4sVv4H2xOA?7w2W7&OfyqgBP*ZFmB3}8M? z73JT>4Q{yZ2Tyz`vJjCOBqIm@@6Z^MBnUBnKi)a`9Owdqqy}EhC1)r!?}rRCZE&Z+ zzPg}J#;xt;MaFG9Wcn9to6R(u9qJ@vG+1>K9J0A`+6q^*Sj@t&l0-9PX{-Fm^uj#3 zd5Hr3NAq(>R@@f`i@8B&Td@`Dlc;-OZF#y*iJ0!_8rwL{?%9T#pVQGqzV&W}F@e0N zU{sF0H7A1>c`BN@XW!4C*6kjB5ql68ju=)vKjD|tRD zb4p9Qx6u!Y$SC#GUzqEkgngI;-ru;-1_rsG7yAJgk$Aq=pBV*!y4t)18Ake)?Q3Lw ze-WQuXRN0Uue_*66gJw(u)bAIZzJe4K{(zWP;llMbK&A;AATLo?{h)7`QGLB{>>+i z#X^1;dWO6fm6}x+I58o-BH35a$MI^b^|>>m_XqQUTeg54hh;}fPssf}6R)Z+&55${ z1G)VM(3?iYKXw##IYa8U$|IrhamdS0lGCiFK?R=br7KC35?WONN6mdm8qJNx_0fM6 zNSTY00t|+aRVM<-n_*y|c$L}m0eII)SzE$W)5Gb-_O>@i0`CCkI;y?v$@6wnT|8V- zgA>sdapSVEXfZ6sIeV!?VJA*7Dm86_qF&4)lw@#j=BgR(*!KkA4_NBzw;Pi-LvfmF zk1^+phu_j-Sev}~cY9bNP%JUO=m)^4YjekzonyzSEc_t!Bd2A*qUCN7TT9vi6A$|7 zgo>&y#BsOTzNg@S)pWN^|KxN-j&+iQtTa{0%O)wX3+h}eo=Z9s*WFF&pv#+32G;{L z93GeGCPb?iW7O8fjNsT1J6s*?iKEypU;j2?%P3=W?hADN{kgG%a|6702=frcD-~{X z;hwA%#M{W3x%LFGexa2z$iDWflKduss`zsd@b(y{_CD{>vP=HE;gb)u$L)Zu$SGl8 zZ~@3Y93+iPslcj5XXsDO_!pCh6Ej{xEQ3WwMGNX?HqCIN1p+f$T-I`qI@OPk#rMTVA>t!KBRVC#y1U!Qls|1<0r1y{j4o9l}4BeOlKu&Pbi z@w3lWJG>(iO*Yk%=zC|MgxK5NsyO5gl$F5YL;H>AibLO|vLcM|J@Gh>er#vtebR`D z1(|tlNsG8keMV8a2KlkadOPpg?>@%{x*krxtu5-pwC=&gNaZO??q--PsyR`zrj3#Y zxh1PQh%V@K6#SgWK~QBm%Hoc&gq>3jKwP7w?kkIIHcnjC*#)TKf3bTMrL$j{wWBt7 zwF4l)~;IXn5hGNg9e~Zjh&gX<7MIJY$6V0*_$Hs zt^c1}z?WfhRjlbzZ)E(RP zH@gWAk-QqO55NE7Nq@I`aOcgAowodjJFj*i{?!%{$P?EFd>x4;Y(9vz!=%I*H-?>o zLoz_?uxdH(CMs z^N^BqcUH^cl9xsCB4+Lwsc~qO9n^(LMp4eTr=ToX3#Xy(V_r8Y^?vd9c=xU5Yf*0i zXKjAQ+$N&pWGEF;E&q~z{ifg5A>|=v;O$Z(;Q4TRK0iNiR=y&nC5ojzHh*MYwszJ*I`WmQRZR@G_g?H3~!NM=FJn>za~>( zCeXg_tM2KhkNocD_V{;y$VwyWsD=$r+_GUC(vL6M*I!ni96VVTqppsAHFl54N0Edm z-J7tfU&)pOa}5S7^9IkxCd^zp`n*?p0&NzkE*mQFs9wXYVEPNH+sz`uYyEMkYPC&NKJ=Wqc<}97O${DN3%tZ5yLxst4{9@2s<0HuPAzV za>MsS9md)i*E9<6Eu~9m1I6qI>e87$jwnBm!2_I!wGbk78uK)UcvG>ZoBpM1qO^`$ zVP7VX%C&l;hZR%iUgauG4UO)B?OCz1%vO=XbD5(gvE5TCE@i;b4^kFXJlu+F%Cr2c z!FPlPG{9DP*;tV?(5DxHMgz1$TrD}(!J?dWhNt+D0Bp_-eQtnta{{o^pLwqh(NJ&X6x$3&LtqBS8+Fpvw5zFjE^+E z%~PS>f||fwTR(gpu3Xa%)Vxw!sg%R=um{)2a&Oqr?5?q_aqoXI7H+j<+o>S$XY)BZ zKhOWMeq15Y)%$vM|7gm7wrQe+gM;^J(6~#4mMBX=cK0qSPHUCBh1r_4wA&-kcRwm(qBM0Y>#i%rw~{5O5l~qj(V|cVjWN|t zA`kiLIwePuMN(nay7ey*g5-{aN%v6qkY!C$hV=I1;)?2ovZ4Tl0&!kp`$5Hpcj+Ah zYHnh5EpS+Y2!+39*J8HxV7gE@q3`NoCE&Z`7?b1uV0Z;T4;n~|_w^fY^V}d8LC#|# zr@OoegZg|P=C}xA;hXpUh_dEhqWCFgEb0HF?i;%^>AG$^wr$&HC%IyG zY?~e1wr$%^$F^Uc2V5Jyun%IpK;Np+Ak6T-kc(aBsa?bxz>X*?!0YvUa`WRzQb&0Bkw5no@YH zm486&wsrFT={7p`)!mdn+%&v`K^CPrHLI^3|KE zEmI(${<{Eljzy z@D|z42MRHdeWM>7tjL*@rUS*=&IG2kd%&!t(XZJ2ojzcQGd(l$xljZ`P!qTT9SPn$ zw90%51HNBj+ftXyCTh>fAG9B&g4M!73aXH@f<)wJFkFHDz!^j*&a)|OnXM%y9}P(E zIB@PD2Y*i~$3~m3)`mS~mjDKh`&uGaw1ktH2*^ZJGQAy+KAD;5k1^%s)S7d;JMu__ zT5uQj1{taSucxC}@yt7j%@ED7lm;V8_P)^8fZKheQT??tGy9{L_luI#9GowHkLyPg ziTIMRG)p=mv2cY=7CIAW8y8#7_*M=+IbYBhiW`ChkXJ!OLZCbJmKHGs7gJ|<(_*QT zDcDPm_Is9?3FOViMdeL(;`R+1YGUTT{*4P3hlXiy1=aUWaes34FHcoFI3xd;Ix|AG zhuuj^p32v>8H`8+Cn-vw7S_oNSyDKNpO>mUcLV6Lhrp|O*-dA3+g78JO7UrACGle;HT zvr9XeiRkyj#5!C^XZgdYRr&UG_1lLR7a^;+=VJcicO^YD^BzREv6yp-3#8+J!V?zK z(+U?pRj*8Nhb0VyRl+51GQ-$NG{?SEdb28+X)l?`hmyEaVG@5*23~ApJQTqdPz)02 zAo64%plU7efYu*)Z!jHCJ7N}V`M~9|?*A)AB;7tC39v_sf_|wcO*a(2qF*gK$U3MI zTByp}z!@b7fh_1FKoU4>hFIAHlV6G*{!`8)_7t-X2zXfEI=D0@7_>{n>68a@0P)2i z$j5LXoCDfvhBLC+avQL-Oe{2@j>!y25zPn`H{J0pO@eIs_P0@9ZvyvfFG=GD4QAD* zg(TK?>3vc&N^J_OA698oxP)6dplLbtp?b&o)^CW;=l;38f@cYCKylCCsxWUMC{VPL z*+jt}ySItQJ&NHGl_A+(F%f?*VZY5jVW;avhvay3Ja})DhMU-KJejydM`*VoqUfwF zvupG-sc2@_{{fvCL{Ml{X@olH0#DDfgNi13cjY7Or>&CMkyCQ;O)mYU0PqGjBur@L zdI+lmS+Ug6TYgRC-Up*pTmtl;T_{!mH-U#Sntc7#es7zTwi8yoUMtW%yVbwWbm3zs z(U|1+3eh-BY+Bx|3Rc?6MvxN%R|99wz9US=*;G>_m`3G5y|g*z3ZyrFD#I=Dv^WQo zzefkzkez4Y!hS3kTZApiIsFYbOb=#W5D_vN7}x4d>_IwFO2afEG_rT9%b3Aw#N9oUdrYD8wj&H~}_;g@XV{kj&vrK-$j!9W{^%3Qq_o z{G(;l!f%$-+fu8vFOUodnNpd}tMGRfv0PA6f?*Th7HUiqAVo>lyC$t@0Ub|l-c~6Y zFL)mS7<4A<_Jb55Qb3puRM#)1dL5{4Z2Y5KD?-O=J8)pE4h$7rYXUP;=gX|?Ca3ZP z{@8W|CRhobYi>$KW9wpBYc>>83q~S=aI74-8}`T8mm_k>d5e?pGg&^gqJOlJ31qy9 z%s_U<8MD8;fbYO?5#EfBOqT2B}TF!U(|LhaV-((?K;YHmHWTd8QL*x-D2yr1Kd`HpeQsFiqo zdW#260Vzmz@EeF4E>PvVFFwAuN%0DIq0fFj(^#sUR?RTKA@Z!@Nt%PJ*4~?6;!Row zxDjBHSA7puME4ltW{J!@FW^vN>BFl<88ka*EXDA`tuqKl0+K{TtL$&BK#}GspDBn&7uO~kC8x( z*NJLy(Kkczfd3~5*nkP>LM%@sLMH^FYgFvDE!Bc|(_Kc_PVN@y=db^;^n16Ul4wZh5>@| zLjlk;>`jhn{$j!%1$5qd@it%R!f5((0+rB^dz_+7YKA@PhJ#De19eR_SfEg_rBSJK z!Lq%YN&)tA=_vukeqW8~DV({`W;mfN6~2^0W)zZgZ0zL`0(hKVEJ|kPnsR_Ovzk05 zIY1<`2$3v8=hk081%5;I7rlxNL<%kPDpbACzX7mkq?g~ zg9i#+E`~5cGe~?d8Q|Q^E%9Xj`Ubd&QX$%`9$2wmvZXA9TFZLJDgXj+(0afylE$6; z=Z)YgjTIN*U8y&5Qo}}5GGxBO22OHG<7FnujLo)i$hqgHe#E+W(;-CQKhZM@*eN)) zRV%3t-mQz;6Gi&KAAFThWxy!qP%EnK>Q#Z?!))UQNz8GsB4|fM*O#kZ3WlR1peG~a zUkks4YNXqyAt2~A1=>TEbduYtu#ok6=)btUJIggAWQrZywW2O}7b)5*%~XBALLVPtWvrqyE{Eh%mHyHTistnc)HfzxV4ecpuEL$5nkO~AZU`X?{@p-X0 zt=jsnLPKH8Xclw*_z5$kn{|H%>~k=$s?vprNT{TjH{+Sll7nII#>Z#2ZQbqqgx-jp z3<6C1$f(n;dVE~e@qSfLH!bP*x_(sy@Ap4}6VM|zu}jb7dC2^9b>n42gWEYXttM7y zG_Px;dWAvrQD7`6=P07xg|JVy#;-B*G-qSK_FQB4C_$qF<|b+_AVhR-p^nKKNoZ@M zc!vhAdZ_dHm!iibX#Bj|+iB{);^V?4#Y;5* z)#s+qqe+vG_fnM2O?yaVyWszTSmG-%y!+defnRssMp}`Oqs_B1Y1^I?n+&{JYkfG% zTvB!R-=)L*Aq4ge*GPQ`^f;z?VL-~MK7uIf?z> zJ@U^^={ZL_Ik2r6E_r*xMlw6J7O#^pVzY-Cuw(so%(SH(8Z_JHuCw3NmjCYpxk0^m zAsunAFt&OL6z>UjE5x%Bs4bWo5IT3vA58fX8lZ`yX{giRpoVM-6yMs~{-?nC=h`6# z$_@XF-}^bhAkv9O{}a8G8|_T-2e3AEc1GE z3TWDdeBEi4z{({3120CPb&+|)gNqwYL9@k{+~UOA)WvsV3xTFN7*Dj9NB=!}@qn+< z{G$6iRa^N_Bdk%F8mBdvgD)W#^77zb+YhbrJl+Z+B$i+Pos2^?ci+|qR@ueizwrD< zR4hQil<#xaagahTjsOAgZr{`sn9)h*0`cVkhAYTFK!6iEm1v;=^H)=R+_F@cQEM~( zzdSf_zw_9af< ze51dwu{Zx~4BLLB6V45|>z?Yr|8F`x;2cum?|Vtw-f8Lh-($dueJ_!t1El@zzbEvJ z{|DcMi>K<@{~qK2homt!E+-jUt+nU>*{j}ukd6sU47G~w~YY-orAum+DNE$PfQk?0ZGU zMP+qmuuqOwV+;538`ucV;6pCa9_ep!E{^7y_HbR2Z(Q0%8TrTZ)7z%zso}?0rsp{) zBT7s6!$+ZX?Vynyb!J2)RM2`A)U^$`b_=Wq;VMyGHM5G050Yjb4hqsoL**Q(cj0vA zNF|m~d*#%GOvwBg@gmy&@HUlS|04eIqk(efmeDzo*bAB^7cbzg3CvuU8Z{!5Q4v-& zW>PN$?sv8*JP+uiISnSh+2zE&TKeTa=>F>xBB(&RFQX))z>IAj^pkgt!G4Rxog5y^ zIzN?U+(?vNk)vaTIsb-Tc#j*$F3Z>p)rADqMqS~z$>{~7hIS>@-JnQsO<0;FiRij~ zq@Q72UH?U^{{RQEaDL7M-mSdj8eGKy!)39b`;YvyOjf%z>%PO&yVKjh6Zb5eDK3g0 zDGsUG{TJ>2BTUS{xS?J8gDq7zU3_~&_HCzAXw2TKHJl%>Rv}b%-w#wzs_1>*%3b*X ztSo=kR(;-62xd6Bx&8IW2JqM_*DPZ^I&_1z%ETr2GV}PL5`5TsJssF0eGR4%uzO@b zqH*lGqg+nOeWnXw4{%{qKf%^=w6e~#BUA8MTCE49Vq|xE4g5=WO)z3T~ zg1t1edGgF$nW;7xCHem(*k8tgukvqJmB zoNWqKtgOvt=jWZn+%QaI| z87qjxx_)=hQ#XlPyMa#`q$ekQh(~RcM`;DLGXh&>u--RI=*QaYV>+X$S}MUVz?up( z)l=R98q>Qo<0o#h=8AnHD7jgqKI0i78NM2@TCz^+aT%17lsM;I^x2K}h6Sd9u^n3G z@%9V5~yn&y2A+$$Gc$& z5)y1izOc8*zOK-b=C;$5k~kzp#6}kVKV)%c(7Hr&@mg}tP6p!Fw06;05JcCU*UZIi zd3?b3JA=wjDUafvFRDNF2Gfo>vNn7xol8EY>X>6 zpZ0~?Hh6qi=9g6g>Y=iX%8EuZ$O{r0N6O(mHTuvNG%a9VKkxfhn%mt&dO}2Z0&F?m zU7DF4R@^b%l8XVz5a!sXjCw(A!jrXdQe?bpE46%|g*<58TlA5%`XYT5a5Z-R#1i@R z4v3s($oI04sN^Zs6e7}hh$?348i5jcMx83L!ICtHXIECHcwZdy_LqEW(Y>;;SWS8CZnLg{?f$rRXW3J4|es z;IBVLG4ab>f*9sT*OIKIR+hmu4~Od+P?&lN+F}CergI%LDwQ5g2^ixi6|Or=cyFHcDmJ!{C(^dg~Af938*Rs=tESJWs-7=|nZSy(1 z10BY|RTavZ-I|#;-&nMMyW5m1(Zsz}d;`BO#n1!V>#GGONo(RII#XNH9yTauM?;hV z?(aka;3l=lItumObE}bOm~~CWs7zEUSq`)Pmcfl~5S~2mX)#DIs$10w&aV;~YR zAI3>8Do!rUgEKCC3_)xO8R`4;=Rs1>wXrQ#ZVl&;+h&MaZ3%VoyzXo~`jir#+UUK; zF|6ujwkhYsStUG~hiph3qY!=EO3#;51TA3N2H64wuQ+u_zK8J3bYqiBohHzAMVA4cs!Xo>VrY`ESy$rP zp0qLlEmw<;C%q*BROM>HFGstFlbsy=oKNE}Nc$KhNlV(VGniwJK7rbha5i+vhh=Jf zepp^xznl7plRG^nuS z!!!A2-Y0mKkmqq9t~pr+!YoRWfrzA%W>Vz%z7*09WQBMuy~}qu9zilVsfj zj3pt0Q#->G-qw;DmVdTOeSxbRrsKD(VQ$W}I0~Cvtk}4%TQe^wuaA3utJ+k*Yn;k> z$K5@2pE>q*&^%V}?n`zhocsRn)uuttzNB)MFi@PH0QV>c~<0W8LX9y?Jb1boYg@DXV`aJpT?q>Q2Fbxg39``^bMzA9a6G zFIH$!C_%PVZM~%hmGYvj=rxBZ#V4ru?vu(}f%XY4Lp*`s<6CUsKS7KY(GUw`i!?_s zg+%y#AV7Q#f4l|m)Dd=XzFkK$47KpBQ?=<I}&fNG(@$*R+7QHlSwFI2ogCZ~y?Wm8Lk-Gyk2#cMb?rUYP@ighg8Rc)WP+kTl4g0K;PQ68 z44?e^+I{+Zy$;&m+(NqVGYbVEHU@=Lnjh98_rA-|zFH90LGJCfDroT}$x903KqlvL z*X!RT1H&)ViC^dUcd3%IwN6T4^$Ri~r0kvHTCs8+_ZqA6k^ zOAL@#>=8rNgP+|y!Dp50jb%o+GI}aXz_PlpyuVNQhC9MY%G{Sm8OJG&f-8z$O@Az& z?(Nw+zu#Q!=HP#p4D$26UV7c3DLweyZ_`W-Z2W@R09x07>J;5NL1>;wG$C1m8~2}M zgkzvATDW1ZLC@cSM0EgDRHbO?n*G>N{``yhQj1&Ngc|(N;TK+oq#m6f%2XQya{dDa zgkP_BD_3c=I!yB}=n!dM$D^@EMgl`><6?cfRNkvS&hpR!c2-=a|3xO?2%zVRh#t!e z`Ex671!N(Ci19{@u-UZ%_V~G=z>RU+hj=?fNg8aZkRE$!Ci29op|u2^)0xj*|N$=K@Yz85KKe z+&KPjg*NbyPr92}hJkX*@cURICSXEe?Cd0x&M&n9f~nyBpiOa}2lf#+e3LPvlT>Xm zBdvv7lR*}vVLsu0|Iag{-eYVH)`XQz(x7es4=;~x!!K>8QJOL9IwRY80Za%8_|RI* zI9o;^I=8&Md5-)y|Gmx%s&_QDGrlv#mr$e^IovDzB18Sq!M=HcdAQ}()$y)}J8@~W zs@+0$uzr0V@Vfp#N?Iwb#%f7+Ity;IsAP(M^u0UciJ%g}Wjs!c2H4R>zx_$4$2rRz z1ZKe3HnVw^=rlLj+{6sgu=m1o1-a7-=H~219-lK`P@&NR z=G4P2H_-WxLwk1JEbU7Aql%`Mrv&Px>yvJKn+y{B$}Bg33$ zXb+d}@?>55J#6cNTB z4MWBG6&;f6z-%%BXgCS8Drk3gGC4LN((7S_N-H=Gz!W3-OCi~#L}ajW*7MDdi>O>+ zVRjlOLgq3a9VWB6ifl)}0S3Ymi(xy+ov3ufS(f&f&@MaC)b+6?3ZoFw6~NomG<(}K zv>0|sph?{({cxinb%K#za6{a*17KR(1-iX6gf0UGc*d-x(CHwDIG=&3f2`loeaFwf zGEF+bN1p>mAGHp%Tu5#EKy+@@v%z*P>bN;mecoyqTX^|c5F%zRI8U!H+TPxHT0}px z;S<~f#?i?4VEDBeIJAt>GSs_LvzAMKC8GS|)tn7w3ZCepMA2DR3U;S*^Ri?Ijl;Y%Z;14&6QN@VZZ`fNeIJ(W3wp8c%8+`br-Ac@b%j_N)A8>&`L!l#r_6Po;x*+l# z3xXj<@I2^s=hgYmu=ZDlM;&rp2etZ513GR)%9Njn67AU=$Wje1*vP;3)B z*C{|VjR|ohdt=#A)6q#5MISPg1XsKN_oZBV)$jbxOKpRgzfEF15mqJ0W=;T=(p}7n z;Tu7?RE=KbRJ95UVKyXNVwwD?W9#=ta*gVSMe=7MrhfLUGS@&3$v)bf1OaF?5o652 z9N+ZRHd4Xb1$vMwY1pxS6?H)n6@bvHWwc*l|}1`#?6 zGL=z{(668ZV@%l2Q88K&kZvYBS1!=O4vk_wG&4MhvW8ODlfO8v0EBtd{{96Vp9r^? zdt88{@t#}R8<4J=A7Lg&_yb?5^jt51MCu9C zGn4Y3{DvevxpUu3Cx&6@+}xD@AH&Axn(Od|PW=GoLCm%V#|kgGw(Z}h8c<~30;htL zGr9V#aimqmUR!$+2B-WQ#Bhee`LmzF*&D%41!Q~Bmo?4ZMap@`b1d?f*q*RHE%v?6 zdS2bj8jD=?;#nN#wb<@63a4q8^9?yx6~f~M5F##`WbU1&F;7-&q0O)2aS|XPzcYN) zO*Dzt5Cc_(ALZaheEGI6R?y*vG*X+0D5X;aX|69^UoUe^w!<+lrVM;a);5D$e1Bp- zxL@@wS|u&!N(FUl(i^v0VcXyOPc8>i^%Xq!f>@*9F&b^8m3v#HvcWO?gG}=qGm9{g z2;=b@1c)5vl*l8aN7tyy=b)smsIYr24rlpR$)nYi!D`rvS=V_q40Z3SU)J`d(wr6w zLEzsd%!NAkR0m;$V1CF#P_k1@i^6wp!nZgdG%6x|6zc1@NxLvLGV-3duEYO1p8w$; z;$4IQ%gt>!>)aI3jwk%!_*aS>eGR67e5N+{e%)_jYT&^taeWA5P&;5u2~Jb7AD6p$ z-9oih@dl&nsTuh49*z9vOEnKS=xWK<478ru6EZM#%wmoutB7it^dE!sO2HlgwOHbjF=5v0zvt~VB zn;M~Im5I+pwZ`uQv0hAi6Iw*jV#%2~Y9pN0FyG%QG8C9RY(=@}cN;S1 zz)esbL8u$D2cTBNdg_-7#c@PX_rpjfD@WkyL!%_e?Vgnb@tAl}n;rCOs2kdE67F$I z=`u#D%6+KBFUUr%W%D4yBCV*@Algi+T#b+iLo4&~T_7q|GJPN~9X2?7o?53Fh8*_veLbZGw3h()B&$_+o!ttl zeA@9%tMEl6EPm_*dR~^$GJT}AlJ1hLFas*6x?W;(WXt$!1YRgX&tr3L7oHoG8|U4% zNh#&t!JKQO+S|J9Fv}$#?hIXkD^V-<8kJOeDS%f=lh!2$a!$y)R{g_YpVz+2L~R8YW>6HULJlbriB$A3R3#Qq^muU&5_cVbS5%I zP7~PyFf|~YP1`%ecQq!{c_qSIIEnB_<KrLt z0&DS(XN5Ip@rg5HFGxy7B~-wyu=1X5{|r^-KaV;jFhOkp zEEXm!sEP0ZGS_ILHufcf{QeFHbPG6bJuIzTTD{=i{^MH0D7CsJj81860J0y@hz4uAsIs%DSzrq2isy`mNtMOvr_e3-1r90f%03^c1z0L)lehM8?y-aQec`QmEu5EO;K zcp8V9xLbF!`L8O$d~oBNo#eMYVeptHdBeV~_io56H(d(ZG0LCdVc*lSU=b=I#hb8c zQ7Q7<(!xeUyTL*=G(xSOxrjyVLXjq6OGx%nT z`(Q*=(}+JUkq1G%>f1;TTxA6f-`^ixM9n7G@}lFF1*}O%uEuSkJgr-R#gA z5)88Ky$cQmW0KzTmsns6Z0txN*c}qD8Am43D(7A$8ap8SY@=>mxiAx;LU z(Mr}o@fc57JR1lY)W=nI>*pvR<7wu_GEb=M4GW!YhO2oQV!>GZ0Usg|>^*NaW}kI5 z(w$aa-o??WywzFd%Sov6DpKi&| z7h!VT|Vv!iz%;?$O*A0A)af?vMT@SRijX2|o!ZF5R5T4`b9rds;XLuK}!P>+e%Q z&u#7Z!Z`Kq-78b4Zoa)Xw$vVIWO&-DaRUDFIcXq~Q!3hct46U?DhHPtZc+S+ev$|q zsg#P0aMlIFBG6Mvex?47PVe(mmfPd_Rg=(;7tzfS%dV4(nTKWLv|6-2>=d;>?58jl zD*T3FEYeI(nQ*r+fe>tTiWR0}6&Zd)CjsJU0raxu0`}x?Ro+?TWY^j`lTgJ~MU)`P zs7^TOi6jt1a8)xh^CPEuxbjhs=+>Y;P?@BwBfd7RA^!w3>4!xSt5Uh&B&KtC=1^Vw z2oY`qoU)WpH~vhMYE#`o`3+fR-A`l4NUQg6nx&`b*F((rFOwW!nM{sHk2rZ7KHHYDDIiK z%Upq?S^p=x!{@{jg+E$p%Cp@3KDc}zCQf$ljnw!-efowXSJi(!S~zge)2UU{aOnpg zf%dbEc84OyjfZ~soC_G_%A1O;+BVK+^Oa37ZR6bVumS$MX;YK3Tc1%zpnt9wta5z0 zKc$r9+G4ml%dWW5g959rs<|qV^TFZcl?-1pG=h?u`-}@TF`x&d& zYqs6t)WSbWw>fh*af5O99;=IgKqxQ4PP5`h1sXPe1k)tJ*h^|F*1Un$O)U}e%NjY_K`G3BD>_-e&47RPX8icjKHF$hF;4ga z8;JB@H!%-z;Ev^nRpD0OosW&vv>D%H^Votf>gmBlEajU1sV{2?mR15fXzElgtzmld zi$_x%eAwT7=S_lj!_QOPTOk3maI-G6JslX~a(iUAbvYd!{ zrgixtND_ny<0e>iOr;r~M703DJU};`r*L-8V}_7OCKFxH86ExAmMDF z;KklCNQZ0En~2}{o51>x!rKXbnaX*6g@D9*P;YqUYh&f`f^S|mkZE7_uu?Wg+_QPL zateL6<@hy0qz>woJ}qRn{=)t*yv>FAZ$9iK7iXqf`_TN*O8S){J#o27@CAfda-Ez? z{13O!9Y-YiqbJo}HFVK49V#dT!*fUXkW}2X*6>5Shlh+wW_!^D^GGRMYv;p70s^;0m^7(~^erIT}}HZHgO0hDv~ixHo@U){{|LY!@$~ zN}5wV;u`?xM#Iz0dSLZPE%Z>fOS_;AhJ~lAn+g~Kgh~vIav0(55oh`cr1S5@6a6_< z?D93?QV-z{ROP4>Zh3Rmyr&^-Z^Y?vTh_0jpxh!@dpC z0`ne3YDz*+fUV!)?%n(8>*{Ln?is_cWd!(-G*#bj=yDY%tw1-*DYvGYr|YnNVH}nH zEbej(9epEhL0JZ~VT)+d^vx%FMDO4uvQmy0FfR#_Tr;FVP&slz2&EU(*H`|*m{0H5 z8F$B#&(0amt1lDGdk-qCj9>|SI<1j&uoga{11<)&sQ|qrmIg4Eh>Ua$w&(0eH>Su% ziL%Sy=$I;6usjzWd0$zGS(rRZvG+c)(T^|O_l9oV5sH3I1WYQEKp@f)acULP!#i5O z41F7hcTuEPk9w8X-a}#ee|rI#4Gvko3LPYoq@F@yK8dpV)%mKsK3JZ8NQSPXw>-J* z%OAQEsh5NEL#P){dP*eJ5}WzWbygqz^+(@Y@Eg)_g9}1+~MqSROkk{pl5K z=RBOgHkArg(jlM?gs8LC7*#%GU0rGW`Rcz=R-u%vWLq?}Fz4PsnKF55Ztv>j{d8c} z?B+=CxgWP*9uM_Iq^PXXIEhWs|aiv1cbsLw236=oJ~&oRznvMTD|MX@tCw`CJQ z5g05LqNl@)3L!@?KjQp_vAZ`q$p5+7&p&|=eyBfLG}oy!_GQ6M^#o(3dyq~Due)lo zSxx=)SC#gWl0boQrJt-7RG1^yG9(FG0+PEJ04~8|R#pl7m?^-vHVvO{33DG+_yaj^ zHc5CM5m7Wkxo|GRpK_`bv%OI?i3t|cs;0KZeH{m`FlZN)^46`BVzg-oHtgKCxwjsZ zKP+)4867CKj|g8IE<9k!w|(c;9ibeYfv`j)sljHx00QTFSG^K?v09MtYa}@0@$QR4LrB4h&pizh>R}Y*z@-CFT&UK>TNgAyOpEP zRk_nljgygS~4bl(&Nt+gF-QU4K=hF!Q@;v|4Kn$*~2z2gt=6 zbKdPAtFTx0=U_n6Un{z2Y{&*Ql%IqZ&0|uIt%R6;?Alw8dmBkhM7EcZo@(&Vf(|DN zc7L>Zb=gjyR}_1_{jpgTQLNkK3o--ol`w+4OT}~22j^NA3eo{MatN6)Ni|AT8vE$*GXaoX&H~X8*k2C&n zJM^p1&W}8h64+osN5IJoel1y&dUmpheUU?kITU+=C9cTDM(-*6B*Q^orgEjKIBz?DvNU<= zVP2IBh-fUVkb_Y}d)ujmHvWS*Ciz`Z>${scCTVO~d@kJ}r9=ziSX;Qu?5c4Gx+bI4 z1~M7YwL!z8!JY*y!;lbZC5NP}x}{|{&3d}6Uh-U0US+S|q1lT3dry&khyzHpkj*K+ zJ0Mmck^apl6C>-d8MgHF)7~9eb@h~@Uo)9-Q0RBIyYQCk=(*6rvsxWX^QfLArd7&K ztF*El7zdFGXd6?yI!aCMzZ$0C)DV^23Ynr>)$NXTSf(4m+SU8g7fdm_SVG(*GZjQk zBvgqjIgGhPJ{)fWKv15x^iow7BglegzfS?e+ zAOpEtruq(5ob1l8rn%(A8hYZu3`f)~aN;B{`UamAE!{mdx4rLXbnIwJ?jzy`i&6;& zvXy#CRr@-DH7)c5!Rbm-c6;PBvcN)AD26eHUeVlQ4s9I^jYTw-2o{(qP;|oPBNU$At79u&;IhCM8A(xd{BJQsy~pIc$FJ$3A7TDm;y!2`R!8_4R4(%dtmP!KIEy;Q9>Qyfjd?z_YLSi09~F~BdgxZm z1y8>Khgx3_ZD;~6KSiTXoRA)p7tzXb1b(9|!5I51)F%;QmRpVBx5+P)J!vBb(LP3n zDrqrPq)6~6IX~9lpcYk74lT^o>>;lN-O-{m*}m@b#wSbj=k?mTjkW3%B*vIxL1h7HQEV0}E^rnZz^=!9!#BIIC7;(ZoC z8yG`9KJQAA(Q&f*_w;UX-sIctQv1+B6O!~GVY($s^>MwJsS+5V6mGR%ZY#r&+2Y5B znI`x_hnCzl(X7FOB}IJ-5p1vuve{wwE5EF_pD8`8D8` z-O64f3huLqmm!sVeKaCmTz3>$7?&UZYWt z+Wxi_qcE}O-2^QeJ}A@0>%DsSd`;zdvUi?4$X57Ef37M=qaPFq;xsAm!N!U0`iV`? zGhESMazCGtkT9=!5o9pwaLu;A-nnb`E8$I$Z)fQ@3rdADGHJv80AIH5z{Lfq4v7Ky zarlqDKU$d@)KPYk9VwU@C$pCpVHLB0vBDkA>FDvA(zUv#TGuBqS+CtjQ-bazxZ=ns zjH&_Hh0wnO1&CVBo7YVkU!T*rD72?5VeEmUuBaBhzjsCJxrI|Lml zt^pXL1`ee|UL^(u68lNtw(3IGdiRViO`zBYinM=tsgp5gS75cax;Zo0LWakWNH;U3 zOoqmxDBy&LnNY`im5Tu zfs)~&+So#Jcxs@JqF$J5%!=HrZUgwivkLGRB>|rf4EFRPnb=ugO zy)^Y^JvlvVt4Krz36Fuge{F@c4kLAop@2ssE_xK)g)B|=V&10o76@QHe$F!^?#E&x zO(G6}L8G!)9^4H{wv+gDHGI{Uuwriyiw6X8M48@MSr_BN-yU$TqEBef?Pyi(z8J&_ z&n$ICgV94n7l>T^Xd9EZd5VYXF9vvRGbz1lTV53(WXp2K{ZOLoBgnin34wg{p_ z>pCSeY`bbDw`?D%j>yA!`TLu)29%{T=2!AoS)*1mzV2VIt?k(W4G21Owm>kc_+n87 zSI(emSp;}lpRdWm76jQ}oh(D@PtN7D*3V1U9<26yh?M9cQw=y`wCG{oONV4)o>#f) zc=iY!8yA_bE@fh2ME1pDGg}J@5I=Uf7Xr{k(OCQAt3KVkUEYt|SsSR*5_)T_L570B zhXwnCkfJg&;Lo0JO-Qhg_b_Z#hCzz!fHyn<_=DilbZCl#jF7YLJaKyVZswvJvH;iO zi#mfml)^z<&%r(*YU{Z|iFoo~5@oR(<~CJN_#O=!vke-1!Vo}_hyaDMEQ*_+fe=oD zo3LQtv>}Z=Z`hmhT+-|EMH{!QG zzZG{=>dNv(3`dSdF6d+v4#Qu%nnEeub?k0}(TJc-m2F$;P&0d^Rz9|nq!d|1G>`Ns zh?^u+F141HGOpZQjr@~`;%};!C_jR8?A>!irB;5mk(ioMgE*Z?3gMAwQ)wEq)GX!U zR+7r_P6ae9-Xhcgs(4gWGmTOVh!%1Iun^}H=xB-Q!0104e|H=U?i6923n}k~)JU2B z@?7s~bSA8#gZwD{dD4~U4IFcuybh3SX-a72_LJ?Z^_a|iQZvrjH@h7PQGdR@82xz3 z`RduyZJ*nES?XF}cPjSiyj}6j9gmGo`I)2;5aOfE3ArSt&u1f>GBG&<1%{2EQ=pA-6S+pB2z`LOnu>#J>Nmtyw!d(w&Hbl?wF zg(|@#&(X@%1Y!+Z*}^Tw%j&WLmlx@ydnmQ=QT_esL702}Hy@ZWmYZxt$A+yuChHa( z%T4wui?zX}Z8)wY^H{Wcq;wSWh`OSoK8Cb(CR_Jrn2dt*;c{~6O=hmeo_cgDrObP* zf~nUXSBEoKPuA`#>@6i|{i+ca=HvR%BUAC8DIxcg$~f|6G;#8(OdnG84vga3joQ|% z_2fb=F;h-|o6iXA{|^9aK$X8oQ$g!0(a3gOiDYunZ9|#^lWyp9pKn@^ZBv zp+R(srpp(cMUgL(g(;K*zdr;g;j(p=BHjW_KrKnrEDH;yAxU*6891$KloXzpYcG_d zrb^JgoY3Q;x2Azbu(rN2iP_Q~`~sn-sU!^^p9sGL z*_oVXOq^743hUBbl8{{sNm8MxePK?-K*I+#?g`X~If8fRX;nf~9^xFc}j8IN=0c$C7V~YXi(I4(7)Em^<4E$nPFZhOs3`syQ-U*0L2#Ghv+| zB6J(3N}mW0&#be|v#gaNti^LN@%L77W~G&u6Qd+K?29kEmfi@!2Ju8F>u#wVq_!=6 ze~xu~x3n8M=);?oEtY(_hs=tIVgpAZ-i6OB9h9cc>I%BQ6_zT9KoT=d6onhhC=KC6 zywFw%EfC~h3=I{EoUm=}+ywsj)ZKhPDXbG?~eT z(k0oT2t98(6BUguN^$W(s-68tkS~JOfD8pj)4EjGw0 zDiA%QGoULmd=i;t*@TNcmt2DiFr(F5BJ!3!K z31P1;REjPPAj9rix`KQd;N75ct$^?f#~**iQVjVWw8|=IBJ#P&467wkF!vE|GVZSf zS&HC&sz|wpIJz%#IP?u{6A?fX(Ye?0d>OBnF;bcqb-&i5dyB&yUOXwHtVPjuOVqZtkSMg`3X*#Wl^(83&Xe@wwx zh{{qyme>dZ7^^60!jxoS)~#hB>e6mJw{4-w(%Q8cUp{TW0mG$im)=*Xc2n&(*|Z_9 zC7V08Hg%P0U+%k-VX%@+qjtBXOd>FBPz+YHc-54!K@(H0fKdvi=wVvXxCI)u4BTFE zd!6pNL*S!D;3Ma7KZe`Ytk%wT-#;0*Ar!F4)0N>KgZqmr$HN?A9ON-U&pLnc%S5lU?6 z&(b*2pbtfqQ3PpgB3M7?Ifu}@vWYlXE_&ALIgX|}qG%()FOBif6s=qV1F<#Qyj+Vg zT9)O54F;G8*2V*aJkZCMms=8WqZBqXax(Al9L@ zL?mAfdbJ>WpMhez-xB8>IBV>hAb%Y{@ zS9FusX&58^R>Hu4f_G1Z{vXIWTmtO>kk$o(Bd zGN%2w$U&b_61;&FMY2rAL~uOBgn-RE6~)mN`ArYp1u#=b{uhJg4S_?%InKz}{)1W4 zpLwc{+pdp12)db=Ds}){b!}UAg=oe#B1o^7(c&~XJM@kY{rSNrn=CaZ#ElJa&aH(9 zOz?lOJKwaHFP+!gdtDZZm+LM3r7J!A1J`KD5QF{Xv|{%fVS=EOV?*1qgsII93<4bm zz_~i8>RZ$$#e(76SkelSM-TPB>>GD>jNLuh{Xz&Jm?`QD zjjIsS3A%Me;D$=N$Ot^8QIBWy`L^A!Ydyss518D*DPp&-_J}mGHFw7`_WSZ)UmV%2 zp|k?7~zPk45||Z%Y#eF`@#Hqow|1&0zVCb8_wZg z*J#|>5PHCk3E{%?7)DI9c=dF(eng@(Y$IiJYVlarrqkYJ7giGxaSR9S1k(#532;b) zg?vXTtc?(AjS0iXwi{Dzc1X4%F*iRGqD_{cdKI^-VbNo=d@V^l)TttvU##eeRe+fK z2w5ILU_cjaME{l*$-(#Wo; z_cYz23;{Os*uEmrLB%%>?5MRW)`V1#Ldbm##o+mkK%oBHWQ~;|^+c0XV`!>$qSGOr zM6Zt5&4*X#d?c@LQC09Z*iF(sd4)(|kh$mt;i`sXu|F|w#7%WawlzkmGQ?r0u6|l2 zBZeSfc}Z%XWu{bEB(q4kCZ1j{;299s;=0xp62tRU5_lcy0rZB+6yH}s=OCD?D~W7) zG9@_+UrP{ln5C<>ty{3CvAx|i#Aq3gL(g!-R`ff_Sb zT25DaI?pOOB`|@KsjWJqgV9tdGQ3X&L$Q&vfu_nsbZmHz(99Dw%r=cAx%EN-DSL5D zzZO!Qh|_T+Dh#7@oFR#Zfy8@*I=a6M?_cpS*BgF7WLH;AS5pQ$ynz^3CK`+ux4)&8!-)*12%@1$dQU{Z!RA1R;!+KZp8k&!_R)GbGl5Nf&%@^snu7h3(A)>9W& z)EQ|?7{p|@Dh3;#9wfGXLmOGr$P!!)z5YzLy1oA(`bE|U#1t@K2qD2XL1GQuo(M~a z_%nD0hr_M6ZR57R-E)V)$BMu&gu{LBcEbm_T@i*791hWOvp7l-Tpyvdyg!r3C*=eg zR#nqBV3=mPHlj_yV8S$rt`Yfir-^`dh?vSXW(>ojOxH3~PF>;+F>x0cY%-Qb%bAM0*8)<8A;*#c%P6eKn9Z@{$HG=4KvV0!fhNK@1kvz09judT;L-{? zy&61H%!qLAz;uYlY&CP+Y4CeEUE8=s`PW%^OFLhlx-W~z%l7e|s<$kT;)ee$4Y~?5 z92X90j$&GEI$gy?a8MXa)8EnTPedsdO1Bkc+Ko-lM^*!v%*(KNDuSpIyvosQ+Osxn za3Fhyfs7=`l7PMiZL1MV0x=&N2HGRo>WCt8Ng&R!qMIBv~$pQiMJkCNcCg?vQBcoRM4XmBu98e;0ErOV##*l{=G~yw=DgS@= z-n>bUBs=hn`^@{O8&`MFG-rm~T~g$(QLELOQIeTtrVp(@w5yfT*0f7<$hrDJ1E@Ri zBQkFNo<~-p3XMhs4WJuc2uC6-G9ujl`Sa`fJ88P8{DCuZ2&a8zBzr^?8^=it2e~*` zb2$RxDO<#I@Vl`vMq?0b1I1EBhf2*J9w-$-n!6p&-;2jvFMthbI9zQf7J;3Db_;*q zzZ^p|EGOgH@#!iwrX6jHJ#E_{+ARh_iC%a(!$XgDF<(vQvx~*_WH~)v%ugq{$PleW zrYy~tUR87!J-Z#Bw<`X@Kcp~K@b>!$7xZQsW%RM|l?EhEqjVdt6Y#&~S}n(D>at|r zqTyJWuq<{N^nqogRlre)GmgtRY&6lu#J)?2ryJ84)+HW2898{F!529OL8VRPJBA88 zS7JX1)vjnKb?gz?Z)ARXQ{;7(g{t(hG#W6i35kdnU(2_ft^olD*f+qSL`+*y&_n(q z@xC}Y5X%Kiek+CTDr>APobS3rUk-eIx5mI*W8jX^aNoiM6C2GIq2bt!hy}RG(1?rY zXY=_;++wBtrcm4f03ZNKL_t)`Hd{qAbo!V(b(}^U&uS1)gg>}0iK)(JjrRk1YA$ga z8x6P7(ZW5=LFcMSPd4~DN*l}ca79Q>3-?(tfzZxI3l}A2SYNVlUGF#)ruH#WzGpX5 zOQCG-)P_+a^hDtrrDwv#w;=l>1EB(xXTnB^m)IzcJQRqtg|)Px?AGS$LYt1V%lVv; ztn+NL*2QN!#S!rtI0$$o?t)nht4%<)NNTFm^Q)^n@9J+=^3CsCE3iCx-O727U42V% zZMR2hj(nPN)^_2_&RYaCm9$aqN;?2rr3kv<`6L%tR=V*n0Z&$;No_g5N`;W@+j8UOZ4o- zqjG{cpJt=yNfe4{wzd~jZG4$EymcD+_(=p6jJ|9`VA9d@d^8(RCe!(39!=NU3|A7% zOSCt=b<$Yp&8X~pT5M^#@?xTpg?kBFRk{2Ytbv3Ig)+i_NFp~mL5QpCc;auuG)#{U z#5Zhn9(}W(U7VghNk@&%Ni)XrTeN62iFt2`zN3+>v*mn!Ii6i6a9}5!;VgTQ>EpoB z_AEhyk1Z)3RFQE%1h((XL+Af!NtoDTek)F zW8pYXf@m=obp{~8>)=i6oJ?8huI2*GsJfBqQ@QhG*6_4|5|#+CEM2Ov>G3K(pDE9; zzJ&_~vRWUwV%#XOJ(%snvGuTC!G=Zk{SB}wPA$M3uf#7=HEm0j>98T;oXqh-));}d zPBjL81{iowv-}KzeFG4ggq)-xZ3r+3dukKob}}oLn{c7e))?wf+W2p1;f}vyv|Adi zeGJv0t;lvGv7Qkz+``c-{Fk6Xj4jPuB0vx>0l~w^n`B9-W$Y6n6k$Thek-R)mH<_b ze5ngr9jom~9+9qA9U%bN_1ZH)tlpx!MkwxWvlFX82VQtJHweb_;_^#!^wUL~nmwfe?dSP?W=Z+y?+ zE%~F?Qo$(&@-4kuRdb`RswDMHeaUL>T6)WMJ7}tTn0Dg3LFK<$CpW}5v~4N&Xy|&l zm)qdx6n+2wgC=8#0ffuvP%9bZ zY8#fe6QM$MISDB*EV7avT50Yn{(~xY3j%ImlLF?X77=VVlhEwB1LdOmGhI8y)onDM zojz$fMPo7ck*YynvPUyPa&Wkqtx* zK`ty#s=bv2Q`LVVg5Nf6X$km)AhFTy8}|-yHe7~hYj-wDkH=q}ohK)HembA@J80;; zsLthXp5P)Gg9T!)uOk+bUWEJ<=4NBj$sqqrXgE5FqN_>1{eD|ef4cnDSQQT^5JNDg zY+01>f(#d0Y^sF{rm^%!^`Z=D8HETE49GrNJz=BrGDIB(7%vT4L2fk-)O40BsZKB` zv#a`i8v)z;zWw0l9ZC?VFA`?ag`fm%WUHdt2iSbB=8FR0I-is*E3mYf_z*IOg>MX1 zsWgcNvO3im_`os1@V=`Fh72?uzDZH+iBn=2Y1%RmSLF=7pQ0q7fsGmj>Ng?sQ*_|a z+qjzO&oLr#U*wpH#K7}0Pb7SzX0By!>o6>c^G)>wqHSWF;WU8ll!L z^@OdBB-ACq@sY<;mI|kU2#PflKk}PpIz3yTjHk2z!QxNVya*tJIH`q^_&__$v+PkGdn!$kVIG)z6oH%PVS0 zm$SLbcSZYM{jCb!7I~u#!Mq~f$wpv|5V;PrAVTdzRY>&q+lh`6au@9<1fb|zIe6jq zG$Z0b4vmi@ScY9dnW55!XE5cLP<5f|RPsT^NIwb*r8-*ZdW79vs|cW~71Ca5a`mbD z>~BgIlZoL8>EvOWuS=EId))9E3$f@ksT0N z7vjz5B$VS)szpo*w~W^y;e=(vu(626D;O8d(yT2FhlDGGj4KJZ4tF?B5hJr@ z7g{j2t&JC7MkDX(#+jYDUj`U|AXgUdskA$npjI)w;>lTbN-%qOWThur5jG{BC1K2% zHKM~VbLR?6-+_*Iahpa#-y%&9%~r?OcP%H+ZGD;Lqxtmm*~ydtlzcPopFZw8O+1vr z$jo^T?T_lc-tK8(TX%i4h0(~yjxL-i^~B>Z%8*dOYM{Rh5Pz~VzzxHRq`v8S8A#Ce zu+`u$D`E(>#EO99Eg2tGQe#umUd%59l<=cv?O_CRtl$;ieEzazsVWf4O-?3i97Iip+=a5o^4N~A!UoK<}gqjHqyY|hDs zW#|~1*acYTpNJ~-GAspi-fz^V_PdnF;51_xMqp`t9 zJdLWuS~b6~C$6kDP(&Q-2nB!WL#;VGi zM@6U~3CV!!H_}!iMv!K(iK{JKz*ozV`m1XGQNE<!7~sc5Zd%4 zk)%V=NH>ZEF&Flmd=$bNPO5#X54lpHN|6=JsvEDKRgISluh4O;(%1YU)lr11!9-8X z#8n*4fmJI~D0!;t;R~3BlK=t7x7tM2CY(Dsq9dA>uu>dYbxJ!DUb8x?uydMds?iW| z1BQ^mG6Sw#RTWR=!dpf^j!&AymJ)Z`+@U+%Tq42i%z*{6)a6uiuro^+a3%C5(+1fZ zMCZVKI5h8NJMnrqOWf9>i**T7Zp$FdD8!}AJjwIE6XSy881A46Sl5FLjZ1S7n+N#b z%k5x)G+8In@_e;07Adh%c-3}VP2XujITJ?H^;{>$cByGD8hSi~%@S9TV% zRQ#;dbr&$Wz>YyAU|~^#H6hEGe=l#H5Nk3<)d&>50;nh9&`GWeB{vYmr6S;}X+x%} z((7r$of|V%z5Vi9Uv^i)pm>0C+!f}Uz+8cG1BUCYQoI6D;E+bE86_}Xfp`6CeNkiJ zgT=rdrr}r~*hz%p1y4Y%0HEeJM2CbshvJj0Pi%&84LEbk<}EsGSF6tR?h=(YKCIG0 zAc?-4?3gmk*~-xPM&(;LaKe8flUfL>W$C(z>q5D|WzpPv1JDR_9?j$BXf-}xUS7<` zm($5)G+SRz%+WfW<7O^!JAJR)^O_Abm`XF-=*fbhO6dl#cdO~#n{13Zif}+B^qrDW zRLjsc6z}We#yTj=AuR*vq)aW(+)+r8KMIfA8|wGBzbf=dBOr;w90g@$U1|0D!^D=16m7hZKmeN}`(u0NJn9b}RQ~Lt@ZEt*NjC zGI(VS^GGo}_^U3N0z$G@h!O-V42WJQB}%F412#a0;va($jw`5q!-KRn$4&ia9>N~< zny!VXVMLG|iez?(Ivu?R8I86(uK&?q7Xtt@i_Idp3FF#7vUgiv+iv?hQO80iV71fU zZ2tSW7OdbYsDT-yt=o+v=ojtO+ztAl9?n+Fbxg!v6dsBvWSzMk7iPu<)zZ<9)#V6O<{#OGtYeGV;xAt4N$%u<($iMqf3)wgYQR zz-=cKH|3S9RE;V5C6(}zl4Z_SFQ_tazJHbPwf;&*U}ls{*(}}iNA9Wb7r%J3f^CU9~4wTM~PVT&P*fmrUa-y?mZm@2PJ!$(8t%pK7s? zKh?ai7J%zD;AY)w0jY9GG4hV8o*(idXP#!u)#mbIc?sj~cyw}d{_Ny$NUzi#~f&SG!8^LCe2 z)d=7#a0H~6K(hr`{ZVhod$a@$2vIG_kQ?fwh*de9$dN?CQz}8K@Zsn?rUNfZs_;+| zbJCYGte8I=Q4+<>(`hIEZrtoS-R5r3JM25dwuNI3SZ};2=NigKGfK>=VRVWx8_uy% za;V6MJ*(YuIymz;T7lN0Fx911z3=mz_Q7rq;icnM&qGQ|NcLvbKiq60{D7kanPL%? zqgjPRTA1_1b8t%6&m9*l?`S<)k7D!WsTJo|65|j9OAtdBwRuIxy>v{z$Hqtxzpqpc z6se(_)LWANYG~Z3c6&<=PBqAQ6-+=!RruifOLD0KWSNp@SO2S0tC4!6hBv;ffLH6! zD{&<6$-ou>whi3Y^<5CZF(`G08Uybg19zK-lgTOS8L~0SxsWIa<^q0$DB$s-@-Y|j7q4wRa6DKJBTiMM-^PLM%0=0R<`Yr?GkcxCD|@f*HvK*fiP-y zdOH2;52L4lJU@GSd^vwS^}dYvFTFjdeS~-Y;bHs!gVz0nVA!)7xTxKXjk%qV)8%;8 zu-r6uPiIE5#Qs(Yk(-sh0;ZQedZT+)3;1TOR@_Np6|qC+;29PW(im_j%6}sG6{U5M zT(Nq-yP7SjAhB^X!bK_vDMAaN5Qdq8Y?hc%iVq555=cv_{zl?eZYcz)>O?^i(@J=0 z`~)1;Z#UjoYq-#hA{;Npc7Zb0V4?i&=lScm>w13wg{$ww{|m2w-%n`~aLO>e64$FC zd7Y2?(#);GkFL=_0vZaF73WHLFDu#C}xfFkR z7AqpUtr?BN?q}Yv)9Ung8%GEJ;f~YBA2R;;D!zAxZ#6@t2uh<1s1*sT8{#703DH5+ ziHp_{Vo6oUq*;>jK|!H_Cy((iI*4Lx96Q#^(g~Ya zE)Zjn^f`t_xxGou7+VnZ|2#sxN(=&7U*37DdS4YQxVx1Eqwg^>;G}d}{ZzkwtGX8f zk|C%@TSdSLc?Pb(Gs)WwFd1>*D*8J!ecuBCKnA*6&gIY5=Vet${Q|T!^v3hszrL@= z`gx6kpBMvom4>Sp5Y`YH2-`sPz2OkxJp^Atj<&*onn;E)4?s~4&2<#F@$W&V>hru) zWCbI!wBvMC6x~;FPt>Ip{+clcz#=-@JInOe@f|>EW zY~XQ;Es~WB8cqqZfUi;-Q_G}OUM;3-*}MhMHDEvJqCzhul^B5`wG_3gRzgrj%pG-| zjZy84;`%(#V>Uh{SZaa&uF6Ei4@L-zf!QX;AB?zeFH#jC6Gn}Pk2cg2sY5x#s|ukt z(jfkW+AEPMi!xe*Y{_ws78hikl8Mqa>b2*!3sIzqb3C*2CUwaxP#p{zOg}}?F8Xqc zs=L|@RBoxVN`g|lQziL(dqcW&0SyfDWc!p~YTMj? z1FtlX|B8lVoQSPP_4+mkw4DfCA!`&`Hrav zK#KoGRs;rx&~VDiLzFO6K~Nc~_wvBT|D92!cmTr94?zeBrYkzNl{W3WY=Xi*P+li4 zs~(Y2Ea}Fx>c+eIrdTzI6=zDiPjGWJe7>^^UXwRfX2td0EK=A9^qbM=&Wd;={_2wm8$Fzh30cS%;NzMLzw{`} zNT5ZQJ&}%X*F1O@*bfTxLC*te!!Ss=ccvhYI-70RHM~uGFp6nGXDo{zR_k6 zjBz*Y4LjYVL3eMrv)}FPhTXo`X`#yl1-BtkQf!z)7nN(P7#n(D)=v>%RQg>)hbYog zEvwt5bvxqT(Hd1XLd6k>MTQ>&ouFYj@6_BmzAfNoAe!1Ni8DEiSIs1{J$!)T!9u(g zyd(h>nu^OjTg7OWNAYZt%_iDnOxGlI7`|lT)!-x9#Ayf{KFyQW_NpoTl_cyUKC5(WZQ*cgd0Y9!X9617Twsz#}Z z3st8g4*9J#)kJhiFUw>3sQ1)``Yo_-;Z)Dw^h*kn{QR#9zg-yjxN-a5wuQi8Z`noPl2wF%Qdd)@2A0;skLrSo_!g|vv6Anl=X1#u+8MYhEo{Mu4z*M9$ z)TZ+Kn}dJOL0X1KWQEV1 zJ~41tX*gCAmSc#D1O^VCx;gC=@48{n378#}+T2^f9$IYj*{WpHVIv6eF>m*w(35=|*M_3@}dLNgU2Cm15u4Us`73L6zTf)xZTev$ceyq2uETQ3~ z2NgF%ZT3M$3|Ai^%jA#(sfcSm&R(NVL7Yk-a!M#Mt68H=o}8)JDv8>ZM99p zefg9ZyQyv>Y}8}OPvV?MSXT(xQm*VsvLT6JlW(c{YSEW#flU4i0B?~fmk)ynw_={c zuw$1FszJE35Gt7s#w1SohWM@a#X7K81Kmj+FAo6m+X;w>?7qffY>~~PcsW^a&PVCl zu{QZqi-OeXX)Rp36XZf+8x<>It2uZFQY8+iUK&* zcJ#dR0IBYWIzeT_=#p|p1y2r66^gdi zzrR1g1g8yejzq+TNvq|Cms?>$*Z{@b4TP~S5Py{&0eO&fJ*%RHW~vtXXJX8FJ^V_VPgHOgN9 zJV7oYT#4f>S;xtIMXYaaMci%#Pebd7xaPjy^~K9S7Ho!@V={QXPqUDzC4{d^Tb_+k zxEpHCRTPK30P8LxqN{jO)yj?GM23Sqd@{gt8v0(h6SjKIX0H`?f}o}QfwaWV>;eJ~ zRtWsNXq~O5IrIz7&<&|z9T>&kC%6}5@E7q~1>O@-(s*H$4I&P*V#%05{*CH+@qAEUb7AYQf z$RoZ_2+@Fi6ndy4J1iUL6gm2H_{D*sgB6FlK%r-dr?Hn+mJx&m6J*{@l+2w9ob3g7 zs;d`Xy_xMbuYq8bVbHx=Am5L*Ft&>XKW`F?Rl|E8nt4#5w}uw9-g>Pq=J{kCk4DjC zww@KsGDrdshmkFxXdrsSl>mK=$+w5^b~6M^(i>a)TK-5C>PB^NsqVb%dT&GV7Qdt$ z>r`W)#=!f;z+I)`gqBfBxJvCZF}SB zDZ0*hp_fGwQVJ`z)g~g;>Kca5beT-Y+TucsLf9=0Xsn%vv)i(}LCTB+j6a4`N-3&jY-8q28rbyzql3%3$JwSgohk0baJq zw_aYQl|m(>9R<(IB2uRp_Wb4pjfn655}l%Wvsyj5SmfHYr#0rTv1}L1cD^ENg7_m- zog*6&ge)AmnHGoIIb9H;X>mMGpF$)^oQ++}ZZ`t2+pzZbOi+b}qgW_GQA^CzO$t4qKs}jJ;iMAGRVVf=whwUPcu!DSfvzl! zlu@hr>)}2hHI1O!B`N0emjCpMCcW&DV23hM(afS{GP~Sx!YG?dTVLa?$I}~u#q6;I z%_Iixb$(>Q385rb`D7*4~fk9RR zwVZ5OXX?*&`8O}Bez@)~x7*7&A0rqOh|IKP-UjZx3`r+PNg z{V@?VH6sg3$3klxQfa+e&F15a$#guL!yJl7r`m;Pjq}fvyf<+Dy|%mGwRgH^t3gn5 z@kWMO1PiRJuhoAnB1M>+aQaU-gkoH*Cewv$HCkTi7oN%XCcf{9MVa7rD^2wrf=`mj z_>lL+alGCn(JEORF;4OIA~H95JjM6@d@&o%7o*v7JkJ(OeUq>OLC0;hbd|>ha9u&2 ztde+*?i^5XpBVtjEionA!IsivK3`{m}d zp+0B_jf0MN*t2#zMk~|_0WX54YNNRRX0Mb!$B!H+kF;c3c#U{!EvLq0YFtd<(*+IR zaLvptg&DJ+Ud|??$!s=SZpLeUI@jh9Cb5L$?gZ9u%j&nyW&_9ZKq?kOM5>^+#Jwfv zorUtLFxa2dOs|JgMNT2%P4j3K&qnL%VxBE0W;yo|pr`E?Vt!c3c=pxNpB2MuIr>ET zW)?4}@nk$dA6=d;E-&5DdawxhJ7K@=`CZ5A5t!E!{Q<>&1aAau!^D+?8rC$Bx@M*V zKCwCgIowQ!hN9_i0WHXeBolg1#x;rcL!G=tkx=ZW8F-99h&LRxsIVdGbRY*!K4g(6 z{QcmE*R5vPKWYrr757f6)kS;kT3h9)B}gb;C>78TiwZEkvg?jvqY zGpiF?O$X0X0Xf#_z~TKA;*l^$sZ+{kP#EcYvo>e5#%MgX>`A(w##(co==R!5X6w=E z=;HY7{QTlj;nfAoTS?ZHz{`gWej+4}0deO)qKXY0T$n&YhZ6`3csI*C( z=~&Md*w9hS@#V$Yvu96EN6$v<)tUB-r`k~--*0QfmfzZMd-wYGUeD;YG~ZP*j+Dr6 z%QTXgTnn>>Z>#2b|$UOWVks#O-(0RL;=FD(Ad}oWbYy(+U)VOv&+kq+4^K@ z9v9k)W=|J>r*!(EIqaBy=*70-2Zfj_C{71F5TVWmmlX{c>8|i|7vu5x_`+d>UZqyE zWrSglWI4-C1mmL6uQZqhPw$L&eivQ$?K0G86=Ffrs$9@+F~CG=tY_#OeVv;DnqX($ z^dFD3%Nb)fwou|cA6ZY&9QMe|=6b(lesQ$s{Bh(5G_t`IJ4J3u7Vs1R^+>=M&%#J_k}7MDiMUYUZ<`Zq-@;s4-Au;ErJ6jh4X=qs_7yzkWiw-*zy8LvJN+7sdWO`!~(ydHB?@ zzL;tM-#3%E7;h#Qakr6mH7LB5-$c8mjO=)-uSUu8ay~vkJAU@`^7xyj|5W?S|E}#v z<-w8P?={1nwhI$(&@rH1eB3Vx4W~}Ze6#t%bioG*J588q=4v&aTz+$==Vm_Nl>Ls@ z?-ZS|0hbV}Y-42wdf}!w(1NvwKT*?5351$lPF(xU+hom!bGgX9R+KplJ)OHzW(vVnA$H+DXqMnfnvo5lx>&C^Z<*d9kqs-KaiBHKU051y3tZdC{yH-0P zKV(Dv;R&-`^ojINpr@ToyWdI6=HxQ)qc}zL@UeCgW$lYgt6?OK$vA8`;X&G1mlD_z z*Y{9asJ-W#d9t3x>-l_oJ{mncAAd8`cQ@MoT)VI7M?quQ>okWQ4{Q(HddC6V!f{hO z9{U|hS_ZkHYYp8Xusc-nLW?qM5oMFr=xk#L>8jXtra{{-+WM-I&&*`BL1=q^`Q-HU z$s~RjHJ_H*Gp!%$Jpk-C++ka2xOU6%e1dUNI6cc2%7sXAapX@WwzjsMbsWv6XXmEv zmQh|zqOzY9ogxl{rk?xooi+tLQnDzs2jLF$6LyV2tWeH(b#dJRy!@?u&u{T;+jrL) zZeQFkO^|f^`OAK7Uy;>75fT#oiUw0_VdGANNbg-3x_UFU8}q>L3~c{_c4BGnDAqT# z;`q!k19KH;gHhNI%f6Si%(Y#vk|pfx@x?J@GVJVpJa!jp+d9}eK$D=;c88glmspEK zigV7(9>++f}&I`uC<#^wCG~JJfDt_pB*1R{qpR| z3}E7M-y%jNGmllmr-a zO#-GB%GCJ^PU#;76>LXwYa&)8B80}l4xa=WJ*TB7pXa}g4Gno;|5V>8v;piZLI$zpd#iJEahxsAX0yvvq9#m=XU5_A{e5$@ z|M|h*r=RWa58Fey0}UIJ0(KgDVGwK*6N*UD)aSTi3r?Ne_&Qs4l9jQXPezI1KxY;6 z#8_uWMzlp|6FOdEnkW%DLD1{xk9OA0$IZDHrnwz2m*de)55i5_T$!zP;DXDPBtGUCQ9V%$eXTuqs7XZ`?+U!Hfdv%JH&W}_)xZ! zL~Ivz8a8S9lK+(?=M5I^Yg~&{23qVIVXI8{8+q1>g28z7$)oB2w3uBy9a+v4-p%sV zUrw7}KWU<37I+qF;Gu&wnZTQf&!Tv-T%&KeiZ2Ayv%*Us74TIuW3PA@>xgRB`@zB){csATRP&jryxg$+>-?`V!?z`F*X~#?@5@Zu98*FRGIP z0P(^7jfm_dtJJD4gtt@b1T}4HXtD)suQBKjcMpI0NRKbvi^q+QH_7bT$bs)T-{hCA zV6R!Mf_2wfxEgHi>FDg@+1F>!{*Ju&uzQ|(?85Th{(pMJ?IaFMpoG-sVcAkv-=Z${j-eC8B zZ*SP{HnHz8nfoYAR`?V=VvC!sH_MCF=6GEt&W9-41=o3%~Flf3TO~5 znzHdmsT9I0FX8%vkqnK9<%*LWlVi2XQMZ7m!@erZ@q!a=HtXeT>APi_*hyj7sfL$3 zh9>qn64x2GN9vFRSqWgz`}$RQ5}KXoLbwYTRt*D%WVw7v5%RK1Jg&0Juj{+jvzO$n zUJ-U74{5#BRwWsBSDz}Qj7))FfKeq^&nvLDx#SNI*#__frJ$CL;$;=z$e=EzRr00` zLn(u6X$v~csw4rXPBjKTd<^_7XgHbcB3@wSU$4_~iYu?RhCa^d(n!vQ(Y zzhwTxAag=J99U(Fi{EC}-`$*j9-p|@`TgGH;&?p&#$6`8r`i;0PkT;?Yu!(l-s0Gv zeO1h#&gZMsMf*v-cj+C4zd87~{iB23y`u-id-n%BZNKZ9Eggvvj?Ykm@(Rs2P3&Ls zmgV+#t<5jZji*f((`aO!%wy|(HES)`N9z>FZaV(bnw!jS=|ui^8=Y48@X&4kMcQp$ zK0ObfTRqO+Pe)|OR9g}3kz%6m3q;O2a4Uz}Lyj}w&p7MJ>} z`Axid+P1ulzSS9IR=DxZEZyjfjS)v>zAUp*j{A~HGFe8L<)NiCLE^*FUhrG3fh@0Y zn2-@V4Ko-zW$tNVXVX7iU8Kob%R4=PcsyG@nXXqCBl9e6&W)yKy0&AsT&?LAo&^mD z-y)9YQ8Zeo=na}fV{p_u+UxHe>~s#>ox?$Y=kTz9|ERZvd9G`P1UGul_E1ayb6{D3 zBR$8e1CqmQHd}458iX)cayG+BQOsawY|?Br-y9p8rwF{xCz~%W@-LUeXXU-*H-}K> zdyn?^j*j-aot?lPI7ZLZ8sdZ>wL;G*>0E^2UZ>|q4|N38%L|0$=NECgx?BY(i}}uS zv%vG1pWE$3_Y&7Ak__!T-Dkas{yl<0UHM zX|~ih7g$=%G<%cp@P4tzD|I2Kn5#Up`>0KqU{Fbst+UN!9X%f58yU{l5(WHh4dsJY zRYj_)nzkFHY>aqHgQpW^3;e5K8yt@*rp+kyb{#91)0Ju)Oonq=yxFIF_OMHUpN|Ja1_P!@#^gH>|%Y9 z6i@BeMXRxsJ3FD1_MnEV#9y!rb1hCb>(zX(FqR4r3N>x0+6)fsY^f+#Ws~#d*tXC!vI@ zfj7%nz0EwG=y{tl#hHUgFD?qe4=)4e5tK_c;hBQEev^ak#vP0z*brds3fi~)7e4X$ySM++9{kFBV?B(HTVVWHL5T9 za-GbQshKrm(Eye)U%hqf`+E3O5G>cE*v!(2g+uWbRyk=IBfM5erdyfnA2kL(R1Cbq z9RI1Bz&w;W%T9!Y&E+^7MX(Ddns%~8+_IiVSt~1#n0?jW_I!F*@N&Ac#NI62F>n-N zbectXFAw$HXt~+2v3e4oJ^ua0SHBBS(viEoZ1zxR*L|!R=TF_qAIkY+G4G#jdQXP? zJG%$>KE40o;r)AuySuyH{T;lv1s%)8;T$U8cq-*D`<-R3d_QP+{AAxvBlbn@_z#Qa z*XxOOp=+mO>wG*~%ynlK^w!>HUHC`7(S~U^z>mn`rduW$yG3RA_;==GI8vL7_H@$g zEqm^|o6mY#uVV%L?+?xXUO#A>d2UZGp;7esc=PNV?=)Yg))(XRKe%qv-d}lp^Srgj z+e5htH*0^jFyhNHUgt|~{WOQR(!x!?pPds#2ou?ArB=B9)_VQwp6AF9qYA%pyOx3W zd$*Hx4pSIyPS-cj`o*)e_190vqp!8g7208Eloe?@)&ua%(EO33FT23DhPIX-X!d=# zz2Dm1zjyE8;9!4mcepcX@ASiAJ7_^5UlVXLDz=|(ZjmQJr^X`iY&RNv`;Eb%5d`?0 zb&`z}jm>CYM3eFIEMGiH7RSp)b~e?%p47dAdS=nKEgIXlQL$}172CEdwo$3rwr%qd zE4FRhI;p+(y5BvIa2}wK(b4Ao#+;+K-mHc_B;H)St}vevE(KOQT3TQWiq_U{KJsiE zZz#eg%sW?v2e6=%aSch25!c_dqhwNS@nKC$dKDinC1* zG>sG7p<(=tOC?OkA_dS3-rcnB{1zS?Si*bG_$=2TTnHg=Ndi|mu*;Xi zaP=wV=eLai>^sbggG0**ha+mQFK=~iI`^YXgVB-EDxe;4HSUBu#Zu=b294Z*H0ax7 zHCypjkF<>j6~-jgVr=pBM{o?cYJ^LFd*(1kf9k6dZ>@s1z15~*CzkzhTaNr6cLKWj z2IY$r3(gzvD9A9#oGX03(+eivjwoN$5_b(p9c7MQu_Hrsg5k)Q*Yk=Mj=iy@ucnT> zh1xfoxv$PER21!w;iZ z4Du5Ke3~DIU7TC8v~DJ6M;Tm{Xu{p;{lQ~!OVA&my#mXKZs+2bEfL_6BN$%D$Tz-z z`N+R>fBrU6t0XK*@O8%dDjVU>pks2DW94IrOlCxi&;k_79Ibc`^^exfQTKo;;6Mf6IkMN))fFL|HclfGpQ}iCmNE7rR9Chj#sK^w zVgT{Tvid&aFkT1DIV`9cbvyQHjq1;r1UvSjj8HS6P{c8ff_|#%GsF|+lkY}Sx{x;h zF;i#{s&tN&14R2!vdJPC9g@76-3;y8?T80KK<{r54H9)V_tC~abRw9RIW9>F}&)H#gY|yUgn_NFOZd*SGk16dU z@uvBv$;TkDKNoA8wCmqT5Tbh;PM;imeam7TYx$u)@FaBIu8@3%%{)KNJnt-NU6m2O z@0L1_d>S^oseV!Q^hq~rK zZIGM!hTAJEt_f+x_T5XO@BUqRuzc`j@tli`=8oOjfq^AMq~BkD0~6lzB!Ss`PQ$NC_`b#YIk-1=bU^Z)Z+-?rdcD1M`IgK; zXBAt66$h-J9>%Kr=%EXlj&hT`iqU5(2WGzU?%SNm?hNOgzW5*T= z3{wSTi;W{GqIS3`3eF`zbI$MYUrM;M(+kkbOHz&^7{MI!N=8&5sDuo#i*3}3*pqA% z3Lo3ckr#-}0Ev`v_8$5OAoZHbEHClqofX~99%Kn*LZRajqaZcXmu8F{ttEO$o|+RR z9_q8nZNj?OiteyT#y40bE|Io%a@ALJ!Wq41s5ymla&&HY4k+X=0e)0?3c49Q-QR3oWmS_62^=@lczTP4m zG5k+JKw@!?6HP* z_?xnNl`Re@W!AF4R9pRXT4))-bn6%t9QF@WF4*WW`{#W?HK(z#{P9n(P30iu2j-*x z`N{yNx94`>hh!%RqzJZ^VvV|5#!M3}+kg~xJy|8VFRBz%$<&=$e;BBjZur5-w6m@6 zvj`sC`Z|&TUTHP4VX>2_$qD)wy6f^2f-mZ4)B;~5A$sw`y0$kn-0P>F#U-aO|M#S> z0&}*Gnn(rk+S|Q6K^v7W5|%lr5NlBk9N@hPp>w|Zw;pVwYZ)Xb9QWW6gK=K>;OO*& zX7~zwt`7*3La&Y|gs`W_H!rU&g8NYG#(8RJU?){lNS}4%m*_R9)gThzkutApKTiva z9!t}{6{{@z8_x5m&_#ZPmhsStPJp1#AnbeY0?Y+3U_-^|5#a%#{~F=TjmMx~obA3W z%9>DUsew<4)w*L>!6r&o#&I0_Y_{Eq|1Qj?BW;S@k)NOV6t#josR{^RL zR0qP6^K0=3u7q{yiOCYpCpmzlvKC>tLs#%TEj zeujg09rygo#m#LMA54@YE-Dwo6!y!tJo&*2aXQftW9>$+#}@ewI=w=+<2tIb&9vG0 zYT6p%Z+04QV?=NpA|jwa*s$km+v3{5?Z~0EL&EuwIOLH7Jw_!2n;!We}|PfJ#_MdcEi;f8Ou zeZPF~?s`tV8m3Sv$FI<1LV580!kK;;t1{T4Gs>SZC`Q^p(^WPjAMXJ%N{@?~NLX4l z`_AavMmvGB@Bik6GX2ToYuX-QADZy zQswX8;1ZPDno2}?|2fg)lPka^F=L25v>|cv^D&w`N_V(D+p#euS>P(OLjEhj$D`{w&CZBdQ4%1X~0QQjS{Nd!{l!2`a+Wea;%s9ON5`HTMEtsUXk z$dFylUw|Q!YN{8&&pWTF5tym&_OU=4K;b-XhyOmf z?YH;0G8h@&e5=z$fw0V6ulUTx%R#|O)Kzq+{W+Sq{b)!`lLsZ?S1CLi^#bLL>p zKs?XGG<)OuxZGk3B)-ee8vs%d+P4~4pH4_vV;wcSw3=M1aZ; zRXJFDlT7OPtY*3nFx)m51TbyEK^2HItbeIE$2|FkP)n+__?$)%IY>M0*1_%6mb~=H4T(1E8h2#7J;@wUnsXK*A$*>vebKpXKm$}Rs6i59Q44wcw_z*aYNesuxY zexX*dlOeh-AGIH={A(81vmyldXk-0V$KE~@6yBC^>Zsre zXA{4X+S7;yBF;uGBGU{rPBjv0OU9jU>3j5BpyATBh7u16enZ#%8O2WbivE*hvg{8p z@_AG@=-+1r(I?KSU=SiXkPv?iIcRGRz%SHFqHeZo<-%b~EOC<%beewoNo=gEksyKj zm#QJJj$QvcMk;|Ws6WyUNv^U z42lmc=D6ZVS~m)uhx!<~SWr@MJ=gk*gLrSCP%RXNqE>J265qpe(~5VUXSG-L z{3t4{Y3vSk6dF$i6p$BSqV{4hlxt%r|rz;D@4SD4ZzHh$)_sU?a1)T$z455y#BaneNkuE*-Cfqf4vIA zy1!5|q31An7*#^YQm@W}!m14s``;ozDfmSj)A9<#S>@zRV9{Lgdl1vME?q<4F1iS^ zD8TR-6{c;mu-r(hn%0ZAdY0b7&In9)H+t72)A7lMgPNlrn%(0nU$rJlE#~rO7ok_* zg2Fr>d(N9bb@8~=PPy8Gs(+jj973m3JA^Dy8LQe51ly*oW%nfku2PeRTvu5Qr-Oa_ zL!pd)+GlMO*>!W%5o?YM z%1d(93|4U7;on?ROHKufvyKzi&PltsfHPOb^bGgZy4JJvxsW;5!0rP`d4DKnz;C3m z6NYMbjv0DuPVYGSwa6>ij%V7761l*YNMM0ZXL6|0)|dKP=m=f!d>{KJ4 zkBSIlI={zCYW&5z&<4Y+Af^{LU~t|(c$2TNAwML7E!btGy4jVvN#)&%5NRCFY`!dj z8a}kTzi!$=O#z-@6YW|SKKJ-|M&e=R742KXOatP-h+rFi*%bP@Ha{>Hk{MG=(VIM3mbmJ77%gc zpogw;p+FM1sc~gTa@kbVml0yn7{oYyr**+N<0XZPIBc;Pr28@~F-qGkHrrEQ5z$rI zmprw4k)ABE42gHkF6^FXGyqh-;8g<@BrL{SVWOHlrgx+l581wn08&BNh`8>%Ccg|j z^R8{){;U<(ADqh+42!jDI0m2Y3??m(q{_&t-wXcMfZ$bAw};YB91Js+*_GczMS24y zA%R>I7&S51m3-;4EsWerGXKXc4^lJt=!uS2J)tuT=WO=xF=~mim)y_%Kc{s1yWOxD z3UT(%8yga)(=3l1^6`6>rcrTK$oYE%E4>dJq4qAg+QvG&iYu0fhv_6DQnEq`eB- z?WjrO`9&w9#k-<)RQ2{k9+^>o`#J@dCl`7DyXoCPX&iAa}U0FfM(n;&`T)d|Oaig_*Eb>wj6+7dIJ%N$WD&FzaF=d^_-o zu3ST&s5o0sDL$P*Uz0zVc1~RoLDmFL?eCW@O!U-Y9Te%}9JDQPKBtjgLFFDyqXM3y zfe#lTGs-=NS539qcN+0CVpoGI>FgLx9VM_Rjf5w)j*GShR7aJ>nx9YG!jkgVlvISx z4ns09brVg{J&0ioUwMb2WYDdD5Nc*|p`a?m7%Y#QlILbu6zx!leQOHF8?d;35FEUD ztc0=DAg)9bbj9iXnM>kFf z3n%q!ltkS5F2uJyTkm+NEcGR(giWji1D2yytK#q_iLq^L63P`#*tSEy78b!Fg=1|| z?PL`RYA$Ch@>TK@!~E%6lFJ%K-<3r8n2w#AFOys83j*e2w45Pk$o;RIRj9kf)rp)6 zg_SuSywT#ERoVTBS@3T_O96jv_g(eC+Sp_J8f+61hmr#vBx+%fgwI{b5tL^Z#^xEu zh#1~VHHeW=9dW&xbnxe-Ny?%1g^+;Yht;g!#I1*^ZeyRsA_=)jg(bo!Vxs#)f{jr( zDsj9_fD-jax5CPt{N1gTY1H7zAKHo?tTR|HP{)bo`aD!rd#tb;OBSXF#SBBtGnBf}poEc__YKS=T%|KrvjxS7zx;m~0T6 z*bu3vgic0(nX)vcQXORf3Rda-CO|1o`?6w7KB0|FI)|x0?jot9ZDLc@v1Y#rqIQ6`weQJB>a zP^c-Z65gp?twxgtS<Fe!r*L;zc?~0C@%q#1TWv#B$jAzw4&+R zVnr8E)OJMp7K(%Z2%S4MoC<_ef+_h>fGQuc)pe?B7pkow=mbN})d>!=VI$D{tMC+n zONOO-tdgce9O$Z=yq@o6uzDLkKm6uh5na@UqI};@Q&dL=V~$#PZ5%ip6(3j7n~1N= zS3R94U4oL7H(;gqoH7kfPCOFZT2V)f9!D+sTpho@6R^5u@kc?7y{oP$xg1Za+P~&7 z5e{BT_2IOJgdjxTbUlx*#4}-7_@Q*M?5U<3>@RgqyOGsMO*z|w&uXP!a=_o}2pQ9% zo<$_PDr$wkG4#U3M6q`I>+*PxHyYDX2{n-68)9H!S`UrM!9{j!vPV^8Aqshw^DMU4KS3(ZptDEmv$-LBj=028`GEj#Ko2v%`*Z zcPnI0@I<BXC+R=K>buGzHB z?uRD)c9@Qqd|zN|O8c;uJe#@bwvf~@7|Otda|UK{oz>KFSO4aVTz`LB&N-liEOe?W zm#3>O(7xx5R28#rDZ%O6tFN}?#fFG)t8_-vKn%6$zdhnNU3$MgKCQ>;FchvOZF+j2 z4a2T%?)((;HxMfKT6+HAXI#P1xHWXcejeN|Md(AgQki_2#84Fk&w2@pKV7%l?^GMB zzNsP$t5GHoq2W?Yx?_u_BvQP_SgTPEF(|bP@1)adUK$L|FC|MjAK34-+q6z0O4CnP zaYUo4Qa^DK^3*ik`gH|kC4M|X;r+9E$+NP!`_&U%t?vuHO(c)C4dj@w`Ps`56RJ1l z&n(S(CoSTBw6pg^JY3$Xtz8$MJGP}2za4DCE_u)2_O7RRfm3wVR-G>X{5;6$fTd+p{%ef+?FB3atY!MA%BA>uhUi5n@|OL99-qSOo|PrU zsTZ?{cMJO-v7k+ov$2dPSKYf?W`W}lr-5e&AaB5D;-+V|-G{qxM|b(ZRz!*Z-l;&u zC(@n#>O=Fe09@@nytNl9B@A-cgdX9hh;mY?vT)r9gU!|~j#H1l&MOW8uE^sMwq4(s zYnwf~%f}w+eh3uK79@3z<0}NFJ;-oN`>7YRAJN4-!H+E<&4-Hb|COw8^WLq9RT5V2 zzE}|3h~ZD`f6YtYg$krqbVZlMBwN=Uy?r(CU8-?F*&zP43zsro?-=AB0(>}q6h&@_ zu>8eQxud4}!LZ~?Gz-b++|jF`!83*iav8{8^KOB^FaqZ}Ldz!=zR4bfQ33xJ@dbmM zXI+o+-*5l@6bekdO*Ch;ON}v>2vl%Za~tVi2|i}TfNTF(EI&|#Mz>YV?WoH|MAzXz zw%<1`U|6M=tvNs^E=gB#pcd_dLlJ%@sX(>l0m4hGfw?&}fySs@ao6qC6v59NW#1H~ z|N7mk{LheQA&XGmm}uK!j|dnQSKQs=$q(DxiOV6#FN@#3nZ+Sgug37w6T&d-8lRd1 zkr31MbaC-J;tvbT0XA+By2mB}AGrJ{jE^GgrHzuRp;vN1Wy&h8AP-ycGwTdfZYWTNRA<-YpuEl;z z++c?DcpxXuAO5YEY7Aue+=>a8k{xGe|8oAH`~fr8M#(Gf@gW|I<5GCL9;I4$ zHVG4XmnFP9^F5(eVnM75QJuK#`xnyx+VBJhq=$Bp;%jp3xiI>B#jV!wxFD(aDF1gw zLVgG<^gP6#7%M32{{hDTw->{|K=lSBKi&|GbItlUb|$Mb)c#`@1OM&DQ{996#^!%3 z^FLj;^akH1m%9w@{|Cecn{fCu~=|~sc z|CH{3Bl~~IdBj}IF+`wHH`x9S>zR(hpSoI60iMzG|0zPtkiTB?gVyb$#``7+*jPJ# zjq36u+rysM-9{4T-|`LC?Bo*Z~LoKb6 zg#yX^G5@xZkofw-e|Yc|BfU36^)6El@6(SyYfM>6Zr>_LiR&>ShPqP!-g^Rw zaO=c>cY*{tD}EJQ;>u}b98k+A&&_-@HH@5)tCGbsYi~NtNTFUYhWc`CA2Xuy(~D~? zy|;%ppbtbDfq+xcv@(mv7L54$x-zpTlII>gl?N0AqRm-XCmpcPFGF=Juv6u9t;;4x zy{9TATm4Pb%_=Op`6G)l#}1581`9+pFeis>&@ZQ9APko;n*+0|>*f~mJ6Izt*JoJS_zcI+>3#BRD;LMjs{I@uQ&G=E;vVn=+QzxuCxc>j z(TjI0EAF1$X)!5hKi2j=tvo!Jogjh?%qh>)3<35lT4VF$N?u#P_N&$&e$fez)}a%2 zW%mj!@xM#%R9m*XZQkJ3WDvFll#HE;+lnfb;da8mUKI zjWT5Ufjrz7($3@EN(Wz(a+aSeQIRm(P?P=nSaHWNDUmQi_AB4PfX{vY#ih=e148fD zabgh_FT~fd26qTzN{!jg!QS!7m6`cg=AR?|$-g!6i$zaA&U^?ECN=`zMyt)2O=;$F zY|>2ihR(cQQ3O6p4n{G}F;n#A>IKB2P~Y^{Ka(1a;ld^h@JpsIrkAw(cnD<#azIWV z2@4?gafy3HUULOK5qQ3*u>tfHX?GW1?(c6JwtPJv!^|(^^FaK$Trj1B^l{==QfSrcR$zGfQ6{w+Ex@`c_|- ztj}*BJ-_@`?EUKB1dhuH55uI={}md*dO6~8 z^tgAEKjat-YLpclly})5ZsRmQ%oDHs;Y_6~(xxyDuXr6_VsTOtL35{KDWvu{mRZq* zBIimnaIsTKLd7%5=y(2M$9XXNS1_!h5nxhm3dIM!+}$N@OK~B_Ekpc9D$*&T`0nA{ z{q;cr?`!L8YUzvdqn|DNIETmYPJrL)L4b$X$Urk>kycCqlx#Cjv?i%dmR?@fDwT2R zSnfsXXI|Bo>$|Vi&e7q%V&h3?i~qgaq|UktZvk zbVp6o#75OPH7x#&TGwshPFp=@e(SU?wMY_KRoF4P4_cP_v8C3Rh6mfE#W-~=#NH;+8qV^jsoO*Y`Y+T9+h?bR+5TghE@_t~e!9hbOvmc$HJ92GY{izg-6! z1q}4`;0cjDvYa+jL3`Tb&f6zb&!j@YqTs%e_9!FEq=Cvz3ft6lh?$!7Pq1E_-E#8H zSHqLuM?9W-wnpKTre=4wEvSe?XhTE8?D;7rpNzfdOhtEI_C0!Gl9^vc8aZ@&z(sen zokzl^`Ke$2rL9D)U=Y#t08x88b%mYTGpFNj>emeX%sts0(x0f@eo`?c)thxBI^9{u znSFp+FXi4&o1ga8F)fbxA9GbSF&I#W<$!+A+P>$-KtWteSz^Su60!)!@59XsqbXLX zG)N`!%`0fe2Y{QzN@2kix}fd{je&-ynkY=_v_kXp30R`+T0#CuXtf6SA8d3@2;CqX zTn3|y&6xpnK95)b&I3~nvkvgWr^rk`b8EJ!^Ge=oEiCeRx|wgr^7coKulf*4C9IlB zEi1XT;BsoRoKihpvwx~kEzXep4ng*%z~?d4tY>j|UIMo)p)&D=nsghfYyd`Ru>+{t-QasMDD;i$ z6WLkJpTWFE5SgD!O5w|LGs-icHM@%yMqrh%za!E@b(J+ar-|XM=8)fAM z@2fp!Ak({b#18xkyWQ0h0a*-kto?phB_R;SSvKrQnZt}-!i%1q&+DZv&?_QD$W01! zmqjzF!>~_ws6aBQyT-~~_aD8l(*eI398nf-SYQN}4G88=VCX@&SDm$+xYIY7=+>H@ zf2NowP^DhKH$HUz!I2jB@cvS#op``&Hm?Mv?v0Y3w$S4m(t6G{t_* zMaTiwKY$P99ZN=M`!htcm1F;LKrdcxngwe_B|TC@A4lgZsrt+2uJlM2qM14)-UJ$i zY~8OH(>EjZBP0j-PRLwdiU|$BVfsmsLUd`qDVpSr(<6R4w6nOU$MhD{j`2odd#4^h zQq86F=bkgZ>RoD;GUKmpHLH_sMUWcNaa1+BBdyI+40EIK0^ms1JM@diRewv+31=_e zHd`CvwA=L^Qb+khbiRTLT+NtB&Un3Ha%l2qS`kg# zgyHa=<{reB?v!NQu=wmv3d`^%*b@E7G%%;{h=UFTE5)dnBYM`b2X#lP$fDkAgnOj2 zLOa?NJG;Uw&;r+AAUEw!`4#!8w_^xkxQJO9@vS6v!@pV0(K5DR)3x#P#)|lxn%Xie z1s$Qr*#JNc8Rozp5o-{v!%MC?Cp&x}x_G^e->CN!FFA19FcjoVLTU5;U>HTrp928f zkF6QFm4BIbnLx1i`3Jsv9JgP1CD;P#Gi{)6yIbUy!0J(9UjG$UE^pC)NCEK&Qk1g` zuFkntuV>Z%O7l)xecx30H^#bG<((0kafOt?k(4%)yOKQmxew#`8(AdqkK-%Caj5p( zF%8dpAz%XN;l}#_3xRZxuUhx54DGw6*pHcUD_4CPLQZm0sxulWFy4g9=L~%Q_E`d+ zFgyy-1l82FHufBtQz<$6HW(SY8<%LLiH(IwW2sUhR*#;WQv^@ejt0vncSj+3*ZKAI>;&G>E=?m%yeWBM*7r6Q-xSHUl6Sda^qYUc@ak+f z8fI6&A55IjMGiKs;RiznB9%KBqd4s{N=5&K34P;6j&kqvd=BXWE1N8&PF$O1^(vV0 z`*Jq2*vsZ|kace|mVHR+q`IVph{nx1pZlOq-2y**u*0PPusJaA;EZ8$e_U!UuyB7# zML?<(+6YEvUM*j1pIhjvTfy(>YTvSs77Rxd3N=7r1gbe_u2Fv|u@?v>a1WEjyD@g5 zO0?7lb0Cm)Ju~21ldUnIsyT9-8>H6=J>aeS$OoYYv!s$5BU(i2iOULrbE5p+PUiM! zv(W3&!kPY$lZR3SC%dJQ^=@Hc0gULGv+yec3QQmDA_kr;*fHvOeX8O}d6^XTz-BA} zS<|<#-v?3kl|OQOg0k*MXz0BTN^n8r&kQw*g&X)T5XLybq%&5G09ojkkB=fmC^&Q% z;wnWXOwg;Ci;A2&vnE*xPfCo_1fe8t`446>ba;uAKh8ofiI>wOtEmrSVF4D8a^b3c zGES^Q2`}^9jRzeyFFe_#Ta6>Wxs_*)lCCrFFNZxJsOR_g);H2E#r17d1I!jhl4rah z+kdEjh88q1u`l6xTF4k)%N5soR6W&J7@{(0WHhU46lOU1G(F$zSAU-Dt%C!Zg|S_2ZArZtdvp!GggbT7&glDfYh?u;nvP^vV{3la4T6 z18|7NW77Leu6Wjq?X}nxUVp1X%Ph%Vo3_2q_UP#PX}$aDQ{+@dwl`ix?6u(t957=W z{L!I(f}AKJu$0neu`jn?c^6~3jwcE5Ze~8)!{fG?Rhr}5?)c65gt_y;(9+Z0+VwWP zMKyFldQ`A_`ZO`jWy#~w4=HB=6Q2|*UD+DC9K;76 zeMII`vaav&D@=!zLwvlXx6!C%s7KlQF#4;D+B;RtW1&nt{^%xepT!@Bd=r_S5`7%; z-Mt4|tzqNc$%{j`%FXTU-c_6SgLhme{#XhGo6^A&&-@(yos8MWrh0~lIkbq)Xd`jZ zlr$WEeSfqNXjeD0fXitv#-+jI_{Fav=L!ljtPC#BeMQAF>0HJy>pc0>v#q^Dd(zI# zZOWd(&(E!?L$8(r_HZjEaZ*RbmMB($HV{ygS~P5vilF?`IkT%3JR)BXqgqNN8YZR*aI}m|<3N7Mz zcAiIe670ro9|CcC;2~9VfmV55P?#}L0>Jgno+odODA!+8S@Fn`E5{rY7t394<+P>d ze(YXp>gy2HHs|EK-3^|I#pzj>fObjmOmJVkoo-Q77@!WbV?!-~qjPS+Cz4#)t!E5c z2FuJWHb>Q8Cq1cec!o{;YjLv{arx{1c=Y>&cgqoN1qHmocD|YTcElyQ6Iag?i;Gb3 zO%pnv9*Z)G4^6jZjwPC>yh4W2o`^@5?3j40Qp)i8b?siAu=-&)=fX3Sy_TJ{VJRhT z7sf5EBpy^pG7Z5KiLBH6wOj8QIL!!X zusHG5FSuHN$*b$q5fLXffW!sM^2Ko+D58*Ot&?i!4c#=35HiC6xNdOb_0f!{LmT3O zcjj9{;?7=40Gd?0iRCB4n&xAK3l9T#(?vFty23uT=Bh@(b%D+TNWq+wpQme~W zaex?v8ZMzXX!xGz4)lEx-?w?!4Tp@$x3?29igM1EW}HvAHf%8NUYwY8X)j$rS!c}- zMrCJwWUe?pYpfsILrsBRnax-9!ZA#pGYV}mIXKR2n^&v;aIuyurH|a`(UM4>5dlSt z7Hrj5Pu3D^Gw39D4l-erkP+wfrgEs zqc8tZnN{);k53@VeKh{@OFGY?Wd3C}N)ijiZJ) zunL_=%rY2Em)^fEZe@RkeO-=9T**=RV9r6#YF1b}3lowcT(Y^I@r%4m(-VRvRxbx~ z8@vjJIjl2<(mNBTLZjE2q-Gc-1q`lmsZGL-#c#EUDF}OiL2_m z#JISg^1gN;X7jTB^t#HyB9u)`=-$;TloG7IW*78)Igiwu0ncDi3@m`+oXuYpVmiso{DK&bq^ zQw6E}OBeRq*FWaxIprtwBZpz)Rs4!o&V`7;LzpIG^bT1A4F$kyor983?vYjF5sBkn z`PK2E7jY7h$HSDwG-vpMmqc#1ZRPW;>3zRpfqc8eb*{6MDwyZM#BcxjXmQG=Vh@Bw zJ#3=5NN7(#++!qODk;;%u@>=n0K_4QypXyfekHPJVvP2l{$+XIramEMX zrC7}Oh9%>j#v3;V`P!Q4!x>E(7+nP0R=~E}pwM)zPDd%;gvZEyEfn6B}DA{Xw4>>NqoM;x)A)6pjEh!N?9h^nT^qHYv zz>m&n+=pR9weH8cijDh=YQJN>ywN}rM789;L;^-I6)fj01b+A1{0m9CY@w#0Xv%A<~O9>B@66L%)NrjRY zv4V9x{+n_-y0}$I>bG;%vpx{%n&UmsZO6|)73y$hiI(Mp#@>tLX$;sthX~u>8AwEh zlyy$O@b2FF>3Inei5}@U5RU}jTP&8-XxrHqzlDXjE}Y%)t}aO+t#)9WwG9f2#jO#)uAh-UuzFbkd<99^Md$E^VO@YN7e5tkqLzF@~Y z0@X6bfbVq4MjjNvqolv(jx_6W~f(#E|MRio>0x|UTaRqDmOA(hQZbpt_;)+Q+phTPOBqTUJ_s!h@T39 z5d}jWqvyE;p;^FXDJV)C&k8Fob$Z5u<6je|pF1q1ASyxNoMbn3{)lf6l!N-r3LhrD zR3tmi2SmJk5@mB~?P@I{JTs$w8)YrZR5vaNZay6 z^_n^Ei!04go56AZQCT%)AsvdI!I%JTRM_R)$z4qcGoZ6<9k#Pz=X!tCYnYO2E3Q;|!7~Q}w*e3;Xq!C|W z{7ied{-&r$F1UXdf2Yhz$bwWI8@hu<*umF=GKi5)l#~M?!9biiIYI-B7PUe=Dlc1R zjFILBya*XnSew5Tv6;-5L=?Flq{9n+N9<-Uvb<9f>0-y zYBGpmRk$&=nYW)h@E{xpI~1ao9vfMp$0|g*%fMRYNGz-G5R*tZ+|2K)`0)IJb^cz?hD)?!1d^6&zA_4=6k7x$CQ{PWwrjQQl^f^+l@pPQb znv>dSuqFt<05BAUULSZSb~X(|AURVDr+`Ng>+(P~0%^{3_-p1~qB0yZ)>}!SZpEx;aR~slrNYcU7b`)ql6V7%*%XO5WF7Xld^pKAPC#i(`zkeaF)XdU zG*ICTjYPNxXPhFaQcV{R~LNU45sBsiw^so-j+q*kwX=xy5%SaC*f@5yPC zKiu<8rC)@l!q*OB)$2sZ{!JvIx%3Eav*SfJGYe#C`E?X}7>=Ci}CP za5h?PgVmHEy1$Eo|GXnrC%nE^p=?aPCH*u=?@6coK{4Qj)Hup`Y?)fDBCKd4v}%{2 z5UH$7&s$Kf5)Jz(?Ko0gsgd5Ks1f2NS|xJ60~au3<=d8u;Uk!}kDB`W8!K5vm;ejxvVR=#mao0-B?RSM9$d#C z9&&l*^Ab?P6wfMmie?wgI90=F2s0ibf~?^l)p?ADDCW@GB(mOoq~fj*$YYkx zozRx^1S?xDnfpmQw62vdwjgdFOQzS}ekybCKrt;r_nrOsI3g~%%Sgj6yuZ0je6rv; z`mYape){tP%=;&Us85JQq`;dN-}5;^v-@$*-H4zYRB(sTVUpo8Ey|`WW8CRBj>ZKL zmXPE@PlLTzrga_Uk+ha8!0AYqz{Z?;?owLD){fjTKi8jszDWqA{G6H`4jKkr?&Wb@ zuILV5(>g=H00MjkQ+_`)i5|IB&9lx+x3oLlHgCTnt`i6*dhuYzXVBc~*BrRDK4Ni~1mfefBtq6vt;6uS*FPj&KQ6+#-shD6_j^ zK}wbk>5g4(WD^y17Sg_dIC=Ie&7U|*Gw(Ji1I~(;O!E1$K;w}4IOFK;S6iwHk#MF=kU1+j3$ikw*8Sii7@^M%_ zP*cWQSJ0JKPoscB!h+^}rW%GtfQKzhiNo1INrT{bqN=?vXy920Z5unxpk`~g3fEjv z#25)4WiZ&C=#Qgjk(;vfd76tJ@ancK@u8>Hz!q_;0!*3_-5k?fEa@;E{1(`%tW#?j zPI_svV^wS4MX?rZp_Fan4MJputgcBk(a;E%EgY2`qwKaosu}k+IR#<&tzQm(Bg!<7 zD&b~a#8AQv+u`f^CWU~$AR~p>>y{N#<(j}vVBoxwsxV!B9AX!(Y{)dVhJ-N0wypN_ znQ$XK;rxGpRP&z#261&my1tsKUyAh+8bUFm7JhU0s?VJKw#2@WR^GmgRrl2yF|1r} zBL%%^s#g5DOQWA0ieF#4%32dx>aRQHn?r=@I(D63`v*lgdfVSmhfjUN}wCaDtn$h%9Gr$)L)vMTn5Rp_N8E>QI{y!VTNtV zi%sY;I@@0g@4+PD@kqR2tU7DD?Q129IEGO`8wkK_P|1M>c*YRB>*m(=1wmCck+fUM zj!+?(?qrHCsp+$;+hj+1Que<2w5?~eGez?8oZ*jeKcFH2A!Z1aG^zf;-q61yu<#i( z2NruVVA5f&f;P@L!~6aZ0B%5$zY6OBiLh7RutG#pOn6RH zKOV;*9I}ard;Pgrzuo_d=nc^wOdHNw3J@>@XeZsY;k;hort7QSdOF|aQ~mEZ@BKpl z+0W0K1wmgPfAHlCReFr|ZAg9{8boIGW`+QUJD_m=A8b zjH7oQ%J!N)x?!VG?49Y|zBd?6#z&{eqoL>NstwC%@NiWN-yoc}+@OTu}G+cO9967_v?MppR8rL`*0pf{J>i9$;g2G=9>zf!H{7r?x zpF|_jqNY0r)=iPqrjFL(?q;{U38F2K#LH3?utC+5T??L|bzz}9I_jT+G;?VW%!Xhm z{-S`F^Nvo1IWch2vPAUk{cg4LZjMrea|~O z>71eskibBk_>xJ&C7N*PrMxbIKe>wLxBl&&zuvB+beT8n5_n22WIy0NA@h=4M?W6v zgP{r^DzK{CTnr%>1V<{%kk4_(jq1#4F8=`iE`%l&t5uV&04TG%-hA=J{`qCO^{rIs zs!nfk-ar3v{21F1aM*x)&NCtL0*EMqh3*ePQ(&`J%kcAM`MIyp%YLaG>&^sR|A*%g zKB2urd8pPXLWks$w}{A23lBGR6VK;fe&L>yRZA9uHfN-(U9%x zWS#lj{c1P8+w;MR5!9dr&mhUQpjU$1${ippdmVW)5JrR6wZSwMusvAL_lsG)-5YT= zDfMH$Gdi7AA3km&gmW!G%D>hzZ}Wv_J|;bQMzXW_R)5+y_s!hD}NvEv)nK1s6dKjQV|hcz;sMaIS#39j+c(SO&llTh9?^ zb-vl|LvXohNLB%A83JPhRK}7PC0mq$)!{i^Z*Vf0)Z$3hKumA~M(1Dyc-iN?=Sfr< zg~|NJzxZT7yI29RxeyjoC$gWmgO9AS(Hl6Eu6o`I5XD)TA#Xg-GkKdgSpDNbfwL*v zPfQAhMS%zyGGkN^;wzjacy{PztolEhh(&sBfxLS6;Sb0Kub1hM+z#48EFQ?V2sV!7 zjmn{7q!!@5T0};OOniks+{@fwUeWpSp>E*el9&anW_wjGF4E=Ye!ZL~z3cu{;r!|N z@spFY)APRf$W_j4X$Y4p3JgO{rwq_UQe>cRFbSe6;*HEt@=jx^wOQvz5}2hfPqTEk z-6ydxE_OY4e=P2+54qQpJR*Bfy2;nj<+mGV!*>;7DBrA%K9^ED1e&6EJD3bpbl!CtePlN?M3<+ z^5u@L^flIz8$)Uz#O#>=VBiqgG_7(!3)kWHZae#OefexTySv-O*Sq4zFV+!)27$av zOvW~&!+OI2uq*KA9+jh59Ti$qk!&6y*$DANcM*Shc)h3|t#wQeE@QzbIoOp+o$tdG zr??{9E;iHWv#a0SF1}bpK#*!eZMcPelu2izeL6gy{0+?I3{7<{L{=9-Q26qDot%L+ zE7~G%l6~Q?qV3&&xmesTE-x0(FJa@l_n|uyz$J(o4(e4QE0$`VN$ykGO}$}tep2U% zPhcaS0)f{Zg3x1Tpa@PZ2>SBgbi-c-%j?a>r>kerrt|Hk-+Y<2x4>gYAgxB%M+4`* zGcT^(M6eypX0;u}c96>`6GA2#kx>EO`J#93M-RGfn1%>7M6l#v?9ZQF{pQ(!{XgSh z2je^To~=ImAO1J?KX$Cnsn=}~VHSD_+Aw)StNDpGS+@Jn%f+Yu#q-P0g8%EY@jr!s z%Q_8vr2Wr7({;l<8AC2b^bTdyP~4BteJ-zl;2ye#h$T-Tk40Jq(9Hj9{QUpjT>kRn ziv8EU_%GW3x_GP&o(}%~nfH%U<+;KUG?LiG2`M1VgSmhC`}zOqOc^6QW^6 zP6BttCkZbCjioihAaTq5ELcbDo6Qy0$d|L};%ZkuONHyC!Xj)6Mf1A$hv%L5pLWvL z>A?<=IZrh%b@Aop-E_9b zIkXV=x!^-4SLLKT)q1UPRI{;VkB=T7J;kGj(OPD!VM39MBl-{xc|lCj?Zgq}X|bKh z&wjQ0`2V>3vi@w(ervMpv;STEs5dIlJ`~{Q4`e4c0@=nK76?ti;)B>n^&hqD=u zmn4QJ2{0!CthbOw@YcoF7=l;IKBz*bI*FTPX@zCKpULK|dXX7w2ASNVzH1wgo zc9bnB#Wxc}zFHGdEeK}Q#@`g1+id+jUI9tCoRw#H?&Q&62p@){;b79Uhqll+SVtuz zimy+wkdUG8h)ij(!V0d)2AlQJ`rKRF>+R#4T_WyEQEMQ7?ZX1;O>^+C&;F-7-q{}b zp?SC$tA*R1!YqgMa@aSa$~>S$A@~WtRnV2zaaJWcP`o9Q#=NuHov=@UEke3zq|V4cA7}}=t%9N zhO(K(>i$mKe=I=KXo!1H@>7BvvO3J}LI4CEn^c-=2YyW8Oi8nZUV>m00A0I$zl?YD z-Db9&T}{9Im)l?Mo=5E01OSHQu%mNu$uOQ_QB<$CWdh4I$WdVM5ve%>70R*TW+ZB& z_?VP8bqaduK)=z7lAw~v$afj9e{~yVckBscy{H&ouZw?|!cvaFm=0_O%DX3uV9WqC z1-WMBH~D21eG%WW`R^8pOUr;Y#fsTU9pC#^@JCP}UN4{L*MH&-TLN(td6RGJ=(1X{ z_`k86v{?Y2OiS0p@;YrkDu8Uu5A9SZY=N+*%peucZ$FPIw%D(K1L^1|b@rZM9x+kN zElX-4FWEyp3aKeHv%(fw)~kBPfJB9aCHW7}OC{^|o9>7i9r&@Sl?CvO0%v}(4s;Jn zLd{^R6z8xk^LO!bxtZOrZm*U%Uray$&8NR^enYnb!~jV3B|D1FCYUj4ZK?5&ZDlZi zFw?@sySlypf4=w?`}GPXKW4Y-+%0$DLkK{d+D&rtTfvS!296bYD39kd#wZ0 zAu}1QUrU%(TDKnF5;4 zDg*;p_-npK0E=`pd;Xj0$Jdt&_It)YI*5+Vm>&(dqY}*rO8r#7%bG6JrzpZ>IEnG$E!nBL{)TV2RmLi)X)PgVNyS(cm zimRaq-b}Mk>hKxE%>2(Ndw0$3yQro8~=g- z{1*AraC}8V_AmPKEB^cyKOLT+I?+EOSHw?}IuxH@S;kjn^5UnL4-iFxZA+dN5u(xo zxq2CF${-l+#ADGJ3|#jJabsL{=m?0sW>r7z)!4wyJ7$H=nyz$!hA1q{nOY9HEzj@-+&vUbWgYY|jYX<=237!B|W zP!W0_dk0iE1(y`C>`r_1Zv{C0kQyP4jW0XtRSpY+dW97Q6(Iib|>n3 z)b93CvRG!)Y+do2OY&C2JADf0g1lrqAT^&@(SdGZ9GM zKbQ(&?MVA0Ya@fLi7zB`9k9i81||yn@~MY0r;a_4SA^gIvH+6}W=MTjYs`ZZ3BD#| z&W&d|DdtbnBi}foQg{~h5dfL8E9g6E+?p%MfQ?dz5jK$H$o-AyPdZ)!K4@qHBsGW& z(t!`OZ!%-T1h8budbgS~tqm4gs1c0cD{%*C|T+LTEx6|8;#q`U4)Fck^8x2&0<)NTccD_8&6`c02qRUwE=Y*c=)%b z_UPY(APJL4Cjj3<#~DXeaLzFd!!*=V7II7U&A@0wEzj*37zq&bK7^>pqQ+^bfn^~! zx-<*nx3@`xZJQ)oDVT7)~-+z>5-R=4fWXPBTUJ>24J+ zFned~`Q7~P_HK4_yItQ#of~EJ?068_=F^^hJ{HCvu$r+VQ~zkY_;R_v*=4_*rlPQK z({&x&LV++IutJJT1j&{9w$L@;zJtOVmZxpJse*a7okz>N&1yDX@2*zG&BDB9+Lq~E z&$Z8w#^+7s#Hn-uK#&~bcuqAL$aBXW`ABnA_XB``UGEebl6)W*ES-+p%RQvT>0 znoBvqwnXvet>@rZ74Vjs;P%54)O4N>Rinz`|9yWw{Oy~6Vz$7_8h$lNoRT&l`_H!P z=clD}GVBbF9qZUqj%~4TG6(wGx2QC>L)h8aOc+43p~Q{Z4ZGdlL@P00R_ieCgrTNq z2CFTlMbG8ffoT0f>(sn2@Dp&6I^c+nupYmQ11U1i@~$ z*ffS!Rx-k?z@AohVQwtKQ<_ev_p1$)SZ5YFgzHtKG>t`eY}oEV39$#0PdDt7n_8+4yc}z-xJL<HmNYdb{T_KXSZK#$j1E#7o&^k;UCK52yx=yup z;@xx;Zx>Crwy;lcFD_R_qKgUhSTiwL-hr}2R%tJXbywQLN3-6?P^T@IH(yR?pIyvv zzF35qBuCn#vm@uR&^hY5orxu%nZ=P2ca&HX;4WK)5o|vZPACnbjtiODF(=*8(y&7q z9}dV|UBNg|m+_%_qeH&QSL$2z25@E+7N|IKz@i5sxec|jOw-U)MJjD0uxh1A^t|ZB zRfd*8OAZYWcy{Pt5xW6e>8b(?mjSYLzs&r#6NzR?9)VIv7HlOmP4E~oFAnLOblGp#xUXYcgyM3_59P%7Z(@v#rig6Yh{v;AGv3b-N_i&OqWYyYhr-Yt|{) zV~I`AOf1H}CiZcHp#XmlNp7w?>yA4d z*n2p8rRy+of=t&-SE#`r!X5=wWQ6U@y(}3j<#|GMQh&=$Jgbc_bs9U)d z8r1zjY4{P3-;5;)Nk$Ejsw_@$K%ZXhFMheXeYRL8SG(SC%i}NX<3a!E{eJhXYo7Jw zlMd_Kl`7A81&-W`rRt^)+3x{bEAnifKl3VXi|R~6K%(z*~} zj{%vX_i{Y=G$x`5t^{qt*g#|kQy5Lg9O$>Bz>Js*R3i%v91KzT*pZ0m<<7y$5T3#o*4b)7EYrd$&&YmvyV zF1ljfm!h5G>=ijg(9tMd?{4Pn#X>g=>C{o;BR#8a7>z-Qci7H>O;G1sM4*Bn$n@Ft z`9H%3!1tZg^z=+0J$gKOEIQfD4YuEFrO2fF%t}H>ifVJRZCbfARH3 zAZCOd4lA|@#uCwPvAupi|M*w4Pkw*nCztx;EBAEYJ9Y=hWDoHTBfQ= zqu8{4S@!kDAKq?ezhB>ILGcflt$EuHdQBfk9r&zvO>tnuH5Srv2n{nh66LGwWcK@T zdc9thm%jGTrMsSZZm<8rq>FWIvRy95 z6tsf2*u&&L%y#U`f>;RP;p(8ww~@bE-dbKKh|8h1TEXME0AD)=K)A8hZd_3uU zW>2eoqUS=CQ*>aTrwFq;nbzC6a3y?sB|YoP`=2*((;g1=!PvGNg3ILMPZ~EcaNy!) zyQ57WTfg{^-R|lvssB+KKN`L-K6c!`@(5noI$3AJmhL-DG-IgQoKSyf`QGzehoyi2 z<*hQO?E_Wf&+zwelgY~xAyojJ5Ui5RB!dtj_qSpYI*IIwcE>gy$Ix9khA9j--x!g% z>|pHXfHHu!N-Z0aYIvpzY{q?nFc8~l=SG=hl_Ld4nJvY(`zifMdf*-5;WQ@Lf&^Rp z%+!OvXP=#F;@^qd7wt4#Pj_Ct?Cf+qh1#mFv??*GNGt$vXu?CEkl6tg2Re?O6mUL- zzysd`g!v`^ArAZbuo4dBker}EcypX+i!zK8A3i~w%g@%I-7TkVUJb5JJI-kQ^zqsG z(eSiupMkhL0%)S%q=@C3XvUB-QbO^f9^ z^uOH3OoOLK@{~-7w5rw+?um*Z7bnO7w)t+6Zl}TOV!41hz4DBmvyYydk57PpKAiMU z#@(Zl>6xO@rfQ}N1EuhFgbgp$7KkkV=5lfI=y|nXv2I`0-|t9CuBHvKT(Ir}s$Lqx zCTS5bK91%Ke`##P4^q3!`j3G6(wX!}lg{yxjr_Zgq=BPNmD;QF0U*Sy)kcB{Bgm^J zE}{?$a<%iYaPCRrVY|nV2U1C>czpLFvtc5<#6nRbD#~wp`Tmd-{{y}d{Rc~LlGqJG z$^#qCGoj7l<=x-->QLU{_CsB$KE!IG1YBbApt=uP69a(_kp9CsfF>faL}400qsC+> z6YUB-lGA@HJJr*c;ox=lG?{#H^KNqjk56EWM{+f>CxHA_+&EZc|BX{skJ9( zW)n4O5SkmczJnDc5O??0dRwzC;+9YdMl^=ko?41VeVY#BPtwEug4>B<>ci#>R-B4t z+KS!LP4H-Gh8bn{_EGF6c>;cNpVoTPf+2R)su9sjm5wcZF ziTh$!3y_G40J5t{H5PB+wf#vr|I%LwTmOT#GG>p@^rI6O0GFfA@v(Cz$r+8Qa2Q{Ej1@x?Kktq=a9n5 zSM2GodFq+xXWf&>!=uv)oL5YkQaMs;HeIpkr&dp@^>$z4n5GIhV0jo<23V zad)4WR!LeoOk7xJpii;#0BvLR#`e=+)Q;D@AIN#z7vY!sNa&6rChGLXq02m=y5On9 zD&^z-HgETn?BydgWzb@7ZU(V79lPiSKIU-i%R9f-)x(VZ+K1mL4MRZn;>SzuFX$U2 z8kujEPz!`^0HAXTXhMJ#Ld~57a+;ZW7hxq3#R1Oz8vO4ym}z{QhNHvyN)SsRIjO3N z*bva1n^LBQNwZ7yJ_55E&_9t-@mn?Er-$Cb9{9m|IIITL0b*lXva(V+W-#eG!XsV9 zDOzX#=bPPggM0kFZ-pPYU4&;A(;Wp{ZC5rfpbI z+{hp_kok4>&dlVCGg}cbS2Tt&`Os^sA(}YoslHLE-A)pgyzuBVZ+^6C5aS}u-IQN_q=U_ zX0Nl+NH#s~=#g{u2(c%Qde-Bva^#9#wNfhJqReu;O^j%-ujh(_jR5yhH{s$FsBH>66}gG#I-_W8-|F^lhl3a6=EM6mFk&AIrm2cjQ4H^XSAn9sqb9;UE(+lQ17M zzum(A4eH0Fy}L<_9Xuj@RynM230S$UQUHU3F+rf7wh+rmh;~A>YU6d82XRv^7WvzR~Q~72Dh{TNOMU zd<-Csj0ljh0LqWPwII0bfQN$|4!79(-mYZ~wX-llWZH};)X%12Dy>MEfjw9x(g)gy) zc~Wdygh5+W@e;-sYt_7APd;WJNZltV-IJri(c{tN$A*fU_@zmmjaWf)^kFD!HI;kO%R1c4sP3%BN2h2jviakc2~lwpj3*( zfJWrlz=wkjwZlbm{4n!p&K2%Dg{Be%NAMz)aK#z8Z_C4B2Gb0Dc>jhqd@=72Yk+?x zTUe?NUJwmHGH`(T_=Y*YejBI-b{|9mhSLx*ra2CDsTLw|i44Uw0Y^@<2QmqNxoy0j z!=Lzu_Tq&x#IcERd=*g#o}F$rx{+yVE~JEUUlYa@0N9C~{gi$jJ@Ah3aCG8=3}04C z%k-PBCmr?0>QN>NE1~TC4nP?fY+o&^z2Un}#|j;DW~soW)(8j^!3*%lad!bgj_ATB zocKJu+wZ`}-nAW5hH14$II`X7*oAs1RErQk(fT$L;#ZIur#@J`fOidWP zv7lgjDs>Yvg$EpL;2@LdIRdW$n6AqpzMH3a7un{jDP(!%K^DVKCob5x!JyYe$YgB@ zHqn6-AC|X5M$M@V=*%#Pz23TKMk5x6({*Fm(urCoYFe3bt80KH!~O)E(zxW!SyK%L-p4m~5Rlv)`f7Rpozb`)L!=JN|Za_iD&03d;p z?$?kHPI?dY_s>{j2ZHt@gFidJ7dgFpfgw1Q!x=c@;SK^6z88|i3B?fD!Z#y=dQx_~ z#^~6bJc8tMNMcLbrcsN9e z*P%VEOBv?y;f4iE7O$H!z?jGCX`3B0HW3YI%}1l*==5ZCbkyxSx(zT6gqH)RNE|wn zY-N!ZcB~j)3@jJeDn)F87KlCvhLogO-%;Y6FHl2p-T?R;1P!W5c7TWnBARh*bOU({ ztmu3jmgc_3X5)z!0CFlD?G16@sVfL@l1zhS4Ebo7BUhJC@=@#N9@7(84D(JTc( z&0&3)Qf3M^G@5Z?N4B2AGin(N_EOYtOZHozHLJ>$i*uM($XP8T{8t6@rm76m%HKEs zoe(@{)arfx7J9GylebST04ztL^7)A$(THb~}aOk8Yu2xq`r`|=Nmj`$3yq1q$Z(?^w{mx+Gb&tE`ykrAO z8tf0O=W|zuY|+o^&Z6mbm>1yq2DYx2?F?4GMUISA@NfVp#7P=BXfOo;maGN4L*RdR z%&K&&x(gg{u&VH0RUkJu>7*xloOW%XyLLzX=2hs(FjlSYBp z3{#8hMQ_9*ee3F>9ylr?wjy3|vqysFUTf%lqal6lzM$s;Fx4p9JWx1|W6YF-+UEoQ~)9hNHox z=@nwX6yO!m;6!S`mkkCSIPT#j1k8?*HGW+KCO_V$ z!TW4r0dxfNki=5shn&P{BTYcOB|VWRf!WlF8|q!CLSeS<36OCiXb~x%MlwuktR5k>L#NY0 z0NZ}o?O6z+;5q~u1p`H1ue>!2QA8Mk)V24nF56OJvu(j#^b9E3;q|8}^`HJL@8Eam z4GlVsIciNT+@bQMX~UTp(oTfsGHlc9TK&G+A2=QYV@RGOxCVL{d|ApMERzWm-*E%@ zEgJC50cIYHl{$v>jbdU6q)5dJ7GQ|x)TXHr@7rkICab#i!4zb=!vc60dx}ZnLtN-< zJI(-S86BXW+`pF@4WgFivEsQBM6!FWNZFc@g(>h4AUODT#h2rE)UQycVhI462ALvI zhiXe&2OdW_#7IUM8SsHs$>+uQD`Q_7W-$^Wz|CthYUB_CB!q0q5iy{sfSkAOSYp>U zx(L+4^d!?t9dRcRT0tpGz{A~R4IPe5=ua>}@KuGcXo4TglbPq)4sJ@78@a1T8JV)KNqzk*G&D5Ue4xL`l?e@C;{%9Oe5QL`2)(5u*I4(F<$1WYXRm+yz7%J>&a*}A2 zQCs4WpTMMgU5Zz%cg;GF6-!oh1V*xSyl}sOn0=tQpjl-4p~ExVli?#MD2_TDn0e7{ z>_SVWfv9(M8*HT9Giq!z5Dd_|r5y~*{2)o>fNB5$AOJ~3K~#An^XCd)-4cQmBSMqj zbu_P|fb}uqO9X*EiU706SD9uuIc&J$5SP^*B7Hjaq;6Dzu6kYMP>6_g1Rhgh8-x1P8SP(?c!C?vT zSwYxVoHY!*;CHQR3;|?`-9UfBDG9j0`i*}^jh9MPFlmY8^>*L^(CrVk! zaV{IEfzKf(TxoB@Gr z>olz+zlt|SvVwoKUwSde%xfJNB0*ENY*lwuxE#YXoVt(JAU1y*7z4hcVq>eBt2)5d z@(}_C7KD(8<5h-ckvmzFKvkKNrmalcGU9Apw9KIb&(Nd5G*&dnP+jOAZ4G*44a*#% zN{HfuobbUyWIN4PH5)N64HYbI>%dl!Z|y`n4n5D=7l_@_oF+|K49&s51o%)@hp@55 z3V&O}`463*#>C1Mom0^%nTF-~zdJt!rK z-Q-l^(1@?8@}Y4Y%7BhvIr!#H6L{&q-J8)8ik?c&ERsYbH?EbWO7T%TWX$V>I#K5# z9v8Y1k~IQd&@AxAFk@~CGH4a(NDreA7y(dO?l40=?MZj z)CFKU@Y-*8-Ok#b_ik7HWw1=bbrELkyL>%Zo5%8__@v0AvdoG!1?<{>6ZQMSVC41S z-*0wdNo>>|DF?c{kVA(Rw`GFUU((83Wo_oXi1U!Ka4D>t*(?id4Y}8aPDjD|U@7SJH zM#M?WAR4H^H!VJOm!DfR$rmL3_I;&hDfgR>cOwB$5gE?YyFR#mz~fr0twSmgbMBKQ zY@V*>>Dt?79lbEtmA=|rb25%ps|87)Nb0&l=H8zEMeVIl53VMZ}y~Z1r+0kbj&NB|q>-HR^W53LG;aINCQH z>WgIf-Wp+KkQCYNpmoJ@7koxpBReEAwZf(MXpVZPq3BRZ@2aP4O7jw3JSa;Ho4wJD z8-hTxcj#bAgpLeHQcd)?d9hsXuHWMGx+Js81^ZrsoqUzOEw)=5b8%Y5 z>3-}rC24%A5e(t6E88BCE>)Y@Cmr-7d=wE!jh4ALa0$bpvADaw?3{hs*!xv_70qlC zER*vdTccPucp}sD-Hf`g0Wt~uB(?|`cVpcfn?70o;wUhNiKKI7wup2m;*7N7Pw)SR zhTQi%>(df)wLCZcsCMhczV|kBJ1c9Qbij(Y#C|U@A+jrd@8Em;VsnARjRWN;ZCf~S z`Ug%j#9JG0Ry1E+?ab)DxmP8DDt-uxo zKG^b_gjrwCZVA@&&xBB8rOPY5$TjS2*ivbHK^lS`YCTO)H8CcY8jvGmYw>L0L2l08@;5ftXT~G<{Fvwtdo|%RK?vSbS@+stM3Ah--q5_~cL)_znpMT$H4d z7ZQbhi^WB!Twe0)kAA3;p|!*oh0Ma(h)pbeooI~;H?$YgjHK^?+-=qv`3$qrSLJ?D z+4C}+mCHr3+TaOT1g@KS5vhZFUSQzfg;?YuW}$}zkoO^Wr9~II=bvaes&Y?Kc=l+; zHZQYHO|Rh+J94zXhbg&3KvV~hPs0^9%bI#RZKyM1oC6w4?GbenV21?ME!Oi1IZDwY z>S>KngIXh7QCP89M1Lk*6w_kkRA|1E;S4fe`c#)S!jtgdyPxx_Pn0&UY1{@s4_WXe zrDw!J_Z6dgdj*3b)e>1jY;P=1;^|9pcS|6OIwW`1f;k=-cCpCr{I`|6o4IdL8nR~+ z?+MXT;lp0-zy`Y6UA@_!U9G>w>sR|D`03>=sev+T(i|f#aCO?A2SZUFAN-?Fg!c-h zS?!Czdqeba*cx_!{A$@0*VzSO9tgE))im)sM*L7($+l__6;$KZ0GG=)r@(j5Og=V8 zzi~!Bt{q1MNf_w4vJY=kA39DyPh@$WN6!^Do+QXECA2@=k@y~iLvfSNFV=6eb)jE$r_pcz zZBzl#)jMdsyzWGG-VgYTc8MIF8S5XPgo;#iV%=~WV8+O|4F3}ah9Jx-UDV?$Wx#S-S{* z{9&1)27uFIQsZ66I3hpNJwjX5J6k;Jz#$3;wxTtOEhM)^O0E3A)IUAYNGm)td&|R`6e};h&i0$ z3L1^4g@LXS-3qY&Qdvr-imH~pQ99j558)w=QV|S4t_1B5r6$M)a+SJLn)KHW!t$i? z9|#wmc~XLWdw4#(%s=_@uRY_({QbU%RqaINR^8Iz{EZKdiGtU`5fSi0=?Ed&s;|_= zLT5Xqs7K0XD+96R62v%+r_vx89WUIXB$!3WJ&GBF46S6_DzUU;Rhx(mhm#H{%iW4ZG3S>3+ zm!2EK&+vK6fwNw0QqKWFG1Zspevvw}N;2TBN%H}WVx9Pdh&Uk9w%d`}c$H<*Sb-K{ zi$wMS;eM)@9w)ghJsf|qLKALKWkr>*aEcW`GjeCdATSdLWdl6~I0llgOi8sOo-~*5 z$N@GRK88+5CPt;Km5G)bfOWH7((+OyAC?C3^QcROl(0JTy<}Mcbgw~O$jk?-vRxxQ ziNT;NShBK;IgqYd`m82N&L_BtYK^dURIXMsbL28v=kjU=cQoyFAp~o`H<0TzCv&ab z-Y?6oakta&E)|JT>s-wnZl+^Is*sTnKCb`O$$oYIYtPisA zPoSfL&M+Zyeny5%wXd)L@#0N2un@DwW-DFw7Y^*pfH8f!-&FSKh5K2@{rEUG%0u#@ z1=<9%g(XDf7N!3Div_*^3@+t`;*J5TwDsp&zG%#w&C0uO^fMIX;%+EcwCnc$dpWDp zeM5=$UXe}jE{^hY^4Y(ye&6eKUj*F>G6+J6xo4^3BtsaVfiPY}vEg=#mbV<^8)oE( z5Bl@pmG*#YNakC@CWv5zsx%UZqOiLKzv<6Z{?C{B%P&9M9{735aMDfp1c)^PYAaai zhyfi`QX)KAfbPd`1U6aP%d*{$oO2~6k^-n5@~I-lfs?Vk+08Fkx3{<1*XLh0Z?2cG z+ygs)Irf&B;U{|wh_=$m_D(K>k1#{lOsn1yu|t-Y0sw-7+;3gff~GbP$Gs@TBIcwb z%MZWEU(P+g_^OsU3`SbTwUA0W9LW$L8W^?Emv?HCmL%|<9iaRyz6`h~Qxnl`abTE} zYwp>VIwjOy^l^}AK^+o_lr!8(rgxWA0sO&H)@1QYsFIK~*_nY441XaXwn>i?7D8%l zt+bloTcoJ+C-Gq1J8<1TFhwFUV8f|&NCA2uUP9GmA1VDq{UtC>JTGV-%~mboFNbP| zYKb5|DYp#T&#V2pZL7Lcz4872RQvn8UpW6r$lI$vyq7yv|H{NsJWH;Byi}chQrzF{ zBM(;fk-CUjk5na8lq}=Q?%YmuMo+k|^onW&JtbYsZn^2OPiWb>64fb&HgTOAa!v__ zhM5CWz)M6kcJ%0(7;KT>_$Qi#ceCrU-J+L}*@a`eAStp*Akc@R+d1}O=RV%{Jw8y` zR=+2{o2a1#tfmv;8Dv(LV4ww-3vS*-MxV}oS7S+9!i+Dh}p z1z0<3}V<%=dtGSP}zv`I}>nDo>eyYS=G;%e|1BE8qb%S7QywXH!Vs$Bf z%b)a8J0hGGZKjf_nd2;7z=u&z6JDa6(UoU9Yfb>e@xDXHESnysD3$%4S|iMSykixh zARYhY;0r~vl0i$n?2a`yhhwifChHQXY+)1sQ@!0}0DL31p(ay+aDNTs3|w+Rdv zcYEd!BmtpcNQuQtC-w{NtT?P11^F!s0AQ{0u|39z*XlY3{wt`gptXO)LzZa!KoK1w zPKHirzaf`>drAa{R}U$CtbVe_}IyPwh0=0OBpj+ zFY;3V?+f{Mf8-{K1k;APH#MAzN35}tS@vI-+v<9?B0AD=KGH~(ofdXlV(OIW zAOn}uR{7OZPQ^ZXV~Hy1O3|3_LfN>M*E>3_SZ>=S7Ck%_cE+#m1yXQ(gIpos)S(Ix z??%}6wRWW`gre-I=@LVA7A=sP-a6>v;M zE|?h^uc-@MO!790zCi5;OA#CFzz&c69+)hMcG!ks1p8(qeT1P_9O@wt)K&R5^CoKv0duWHWTz+``k;@;yiipDf;svbEzW4U<9JZG!O~F!!B^UM;plA>rBd=qx>$rM8mfjz3G?)wIykSG#*>?buM1?T z_}tw$SEiV{P>qVb{pleW$Uo&I#3e8zCByx9=`rdiwqI2XG*JOb*LA~3MF=;HIw3Ps zO9(zC!Q6bl0Y;^nb>_F)qH##Bklo^;zl_r?iEMx9+rv;&l%gw!BoHH>qlcr*l&Wa4(N|7zXL(_Dg%6{CZ!$dWtM3bxNI;mGTXT?9ODvrkr5OV}bS?@SmjehNL&Q zd(oup9CWRIgvlP+oHZ`Q5pe+FP&_R05D~EzeOn2wAactLhn-%}?z=%OiC-fXlf(so zljDK1G2@Y~s}9KnP}Yy1^*((*et9$)BtF?$h$jQlT1Cf8rP@)Zg6f0(6-8_mKg^HqE2k1ZI+XYiom#wZTW7gTwBiJLoxw-Odnv z90DhlL4nEDmfKDRRZBHJ_#`j@UQMX6&Tb_f9VUn5?68A?2?PXuW2MAj=(y{oCQg;> zT`4IE@ogq#l`TQr)^AhS53b=x5#Ue*R%7>Yl{Tv7Hw&?wfXZAC&aV z*MA{B@H3a;=tW2syA_OgwP%GL98}nhsvW85Ay}iVA@~@(m<>)L!45Qfdv)8nv-QJ5 zdwXm7bW5FuR!Xt5JX@Om9Y^!Y z3Ve@tBb6x;hKGv=P$&y4{@v9)QN84<`wv|5Kp&KV0xJ-8f?UUch`%slQLNB}J*h6( z*r&SZK%lMuu|GWMb;D`YnFjj6$M;Uja3b&VP;P2hGdp1Rx=4bEh zpbzjU!J!prBZ?OKM1`fs^2UD|jJ5&tojSE&r03*&`;QA-@r3(6*kra%ksCkkW5q1n z0_7p?qwg!Au6?*kJxCcF5#Qu4ic;r3Eb(vp{jmmZ3x#MR3z!I+5A0q2@K{%_zdtx7 z;~^@dc8J>&DNj?y$f|kQnW_@;hLM`z89o#fM4Ll+glr-UQbx41x|3pl7n!+*laYKI+$h1oh0OwlrANqZUn%9D7y1XRNywD zPOiY<<1F-0-|zL5UvBz&^}x?ThErUGWuqqU&UitH^}QzA9ch5Wz2fR(}v z%c$?t5f z4({Vx(LA+9r~=fN`wQC=RoU9%7V=O9O_}5#QGyL9Qu7HB(za?5!hrW56FeM{fPlsqy?@QIf)+xXb68Kwv9?8 z6=AzAQJx^^CqhsN`4Cwf6Hu#=5qhc~D(~-(`y1O6@(?A&QVv=i96Lm8=fBikNp^ft z#c;hM&pZN<6=F3(O;v_P)e%zHpXg{un9<;%$ucYq6Ed07$lcfLN$uL}+saDIqzhCN zpIoELQopoJnRzQSmoQ}UJ7nCk;n2$r35mZ3wyvg6s2qmPPowT!e`2XG0axzSgog#t#qF0|`) zaJTE`PJiG=iS2eBvuo>wikE~MSq)gyObEao+Q}fj#_?oHvNI!$0ET6VUnwFC%4S4L z;=752kclgkO5|86fuVW-p#MmuRqhA0EqPED%*04C8x+H=%xzsncF8)e=FBP7^9fqr{&| z&?3|v!HYL&I*|o*9&nALV07U1V+o+4Cn~jO_#h9IB_(Bw22B2=fcGW%-NQ2yGY5z* z5}om|gypIKJ&$&(Q6XuZB-Jf2^8uRJBLiLUU^*Ce`<~r3aPK8eSPfy~1{y3aP&d7+ zn+zbQpnzaU6ES3?lEaXS!z2l)+sCxK<@*~s)t-G<#5aCYG8_ZzB~3WcfelRty+Jsc zIMbm;W-Qd26i5d%#NT+ooMciE-6i;Z09KJd+47l(0@OwwJDzQ!Z1j7TzuVgQYZ8K; z1z8bEo?!MV;yo;6>{}mHLj*XLr*2c1KxToToo`cfS9^IkFuOi!s1?wY_~(4Jd=DDvYIdiwG0l>QQt}u{!;=2KNsIgx+1gVzv zheY>_0`7q_2y=pS>tGWTJSjL`sP$W4@0RmO6F%sM`~IRBX_Mkm!v0lniF0RiMiEHE$m;-#$Zuo&;1kCP%`65-|4~iHELTU|apy_#4;^|{;)7b>&l4KQkb$wm0oqAq4 zpsmF>TmP;}^04PP=!F5=A>vx_cfq_N{X1lU&JtObI;uWt`>uUQutMBH=>VcGDJV@l z!H=A!p)K*}%Nku?HEA=pTsB+(0M>|_>DO%b|9`;QYVSMC9SY%!dn zC-=j~MO3719n2r!MV^eNTx~p+1)MMtlIK-8HA;d(FN(rm7$eDk!$74M4o^5I)S* zj>aU8TL&|;`X=jbu*`aktV#;Weumosvz_Vm!6N>qhqdRF3H?H{)+ST6j{*~$+ zZ@gNyiEPi=nd8VVgcy$&_yqEB15B#9Xh5$+LM3S)y?sNF7hMrGG^B>D zlC$lCw!@G{!PdyZ5nHXZ^<9;NX(CYsqVnQ+zXLz$6+p=M7U(7T>^2)jFha1f(eX~O0m6mNaXR#2uL7Ao)+0=oo{AE zx~wX15BCf?N-i2C2VtCBWl#5KyyV?MVI3V*$Z6_os1jId- zx!(cErvoM&N;I^=a{%0S)d0IyiX1Be*M?3W4d~#LF=B2)G;%%LYS~K55h&AqUoSQ1 zPV;wQZ4)`5kst|RM9Y^z61IDw5n*@}jkVhF>_}0MD4hz99GNO5ezium!c*=aLrv+K z!q}%I;MmtV`BvLPZ`VjAo^Eoqqt6G?ToMrD0IY&aN78eN0+1y#6J5}Lz-OcvG@ByE z`Q5;C*sO^RCu%?eSpkv+oE4nh31yZfGe8g{Scq!{ejjPhbAsD}Zv;t#0((Fh@q?hO!oR8UU%cT%F5$XKU zt_E(pYvODc?8xdH+^oZk?P!qh$rxNaT8}-of&_WL{6F`N+T!B9ZL(RjzN%Lj+3I4s zyt&yqw_SgCK-%vx7>72{cCLi!{JBzoIsY%N2YzB1PB=MOoZERZf1AxO!GOD3X6Nhu zO=hp_q|}EUXNWOydf0sdaRhlDh0_yS`(AgmK#%|s+m2veE_MF?OQ?35Q8xo%ds z>0)ua$lomL>z%zehP6Af1A!4f8euge6Og!TQyp2hR9M~rUTy#C7jF5+yW~72LI@~c z%xNg$i1@I<3J%FpBw9g8H;4ykrSY;P1eF~lI1(WD(=94X4u)eiSWa_0_*xn{%huDaLTN)(hb0i8{5lUcv z+(`*?Al#L8zAe(lE}hN6QYVkN4ETdSL|ybVd6TBLukTbLStHz_IF@{1YY&$~d=u?7 zamAr&rZ5RzoS`%AHr3WCf!av&IjSSc0V=!s`|c7gtqwy^H~Dt8+^*ha8^Qkv#JyD? zeW!|ji;Ed6MvNB7fFs5!v=bC2D1g{&IY!UdH(^+1ez~$X=$3Q2?~5|S`irW++nXSOmNrK_&ttNc|910`GsAg|E1*unT+i?CPtEifP; zSl_M6U4*`42QaVJfLu)n&`bAX4E(!5qq)geNe zjNZCvA3GUn0p@3k^t}_ zN{ZtG8=4Z^1Dn^32u1Ad)!sx8M}9T=RU!(Iopj6)508?^{jCp#Q8s-{Y-ll6M`4A6 z5cn5yC_{$2$M|h)sQTlZDkv0Ds0S@+iSFY?kGDL2c>L+ld}yZ;6IM_W*yr;i5sk#~ z6E91SZv3-Vo(8&7+!p@QA#+O6Y3g@>9gCHq9@0bZhhdrr@pc~Ep4&Ix`6_<3nucIv zZT2++oCg%C;LJbV_m>_czd_8ijlRCwFW!{%*W20a+5G0B`uI8t^-(yOgwZGMZ@QSJA7@X==XkhWYVH^}DL^ z0G5i`LI7zcJ4PWuiqa{OKp31}Np4h;U*rpy$Oec@n6SdqyPXck81e!-9ti5}mr>vw z_Rh9F-?96h{Z^Y9B$TYSH8#0C2U>KKl|kW#HrsSzL>-iH`A*v{>&1CCd$qbczrL7# zd0zhWvUg#9T=qN3$U7}ji|C^sB2ptBwL(_p^@FFwdt`zyv=AeYkBa{M+C4l}=87>xiPY4QkOT}>55$UZY5puN?H9=_+tPABJdn&-E!g0lI zNq9#GIXCb+#Ic2!r6-200=W3~t_Zi;Zk?wti7(93C07q9sxTk`Lw8V|N%B0eKnruLqehT7JtNfMC~d7mkO`mAyaO9X4^ zJ9)sUe3liPZN6IW*7M~Kr3t8ibWkh9(eXdL?^Tz5zqcn3e;7fA;^E`BEmOvumxyx0 zcEv9Wy(@zR6gEpDF`e8dHC-zc+>j=1L${-7+m;0yBocRVH~Jy^9F+_x|64L;ZqoH? zvstCPokEVKqO6i9)%~`0`zzPoSG}#cfZE7o#GeaR==!dig!^qmnk9FWSz2B%x0mZn zyEoVB=IgkQ8a3)9~ zJG8w5c3- zP;5*_TBqLysM78Dp2%zjRNCy@aB~lxPC79CFElbdbo6pv73;JJH`#WR?>swqJB6#^ z+hqYpDQg5^;BAHB0T}awe9+aF3>6ySM-TcMhFn^uBob+kB#_brV)Of!w{xs{Eaq8Y zdh`7nX96+Z;@-y|td`xX94vF`Z)AJ5(s4Mbxv(;GV2H$+W&&h^ULI)#rCxiBy07g` zp&IH?or>fvv{a~w`tu-=aN7@@HtnF%`w#+XJDO$N5@Uc;juJQjn0}hNJ*lLs=?8BQ zmHI(7;u-Ro_75G1D$W#B1Qr`&MbH&eL-bhm0<#6A-j=3*P?Hb7s*4}b$qo#Z$S~aZ zhe3WAuBy1Z=-pjjY=#%Bb-dbkmO0|A-;f$a#5hbv>Uh6dZC^Z`6Q|3ZjES(P>uh$m zo1L#0^Tjs0bq2d$e9#X@eQ(gSf(V1N(Ar;)|KIe$Pc6dS{JKS zz(Ke0rv)mYeLa?3o>O1%2SyMYWYfZr7uPxx8+Fpq@&}H&buhSIlT_fYUTxCNe7#N< zt6;VQUtdyQk^Kk{aWH{5i+XvN-(9V*U)`Sn;p(feUVoncm)lSN>!^Qc6n~#+?%sT% z8KrpiwQEZV0iPad%eR|0E%o^DT3NF43nWje`4xE=ss}KX3xdezz$;Fj^q;In*37U2 zZ?$k!B&&35?{@koZPqCoIj}^y8QW(r{!rWPZn56ZX>plpPHt5Q0tC7R4PG)ZS+IFA zdjTzF2VjB5W`59YHk!LY69SC#4hU9%zF3pXvQw^2XN5?--}`2!?*Ut1RNL8dMncvF zqQW|_*0r_PdOMYh@}BHSgH+ehJJ3SJ3=FoY{0xj!MrWV-+P0^;wzfeO*=@^3WHJXBq1g?*rzCr@B>AFQw7lz!ur)q>ZP7>|BV@711a3q zgRvP@wp#|)Zmi$E(6s-i`B~=gc0TNKzD(jd#%R4F8@xdVlE$tJBV9I|MYUM&n7?%CPlit1$4gIq`tTh^iDX!%)hR-=80))FH17(5q*`QYXSI}&y*yo!iO$cf2F{OpSf2?(=(%y|*&Ey)O^#o9*g)b8~ZbH6(zPdFbQ8O}0YM1QlCA z)kPVhyoFUWM7~GF7DX$LCRpP)dlFjDuCm$1`sU*HdVYs?FFRRktr` zKu9XX{jLQM_4?)SztA4|iDfvMjm%BFV?x)>V`nq8{9}E8RpW`GeL>m)?F;LRgZej@ z>6`A>Hj>@i4@kFengHyu1D7qK(l1DyL9uDJH~ZCPzPw%D%`PskudlvX-MP8`QhVk! zM`1ne>S4Fz1QxkcP-N`g%1D!rKXUc5kKq*4-PNo*yxh#SyIcFB%UW%=TZ7O!&mfPy zSMRjVU2%K0y|`LmoL{~D^6ZN*&;OJ5>MptaU&t(FlloM$2?^~aD4q3&;-uxfEdhBl zL%s_wd2-vka@vBCQgDT{$+I{}igMytYj06>%_VZni z{6n6_jO>o|aJpWvW~;?)z21-xHOM;s$}Qcv8cqUOtm_ipz>d+wSvvCVDmKf>epP5_ zf5_bUcDagft|RQn`*K#tH?`;0R+W;GATx_uy}iiS=d0P3jKa-)mKICHUTO(m;W6V4 z7Z=j}KvE&sBrZS+Vv&W)6O$C-75T=~&STB0^e&!<%lYi~>J~%BewS9hSz2|fllXj7 z%x{)gmv>i-yPM+n*1ke0fxE+MP5lZw)l%*D&o}nDI{B_!zZ;c*sQ;4@l8;Xx^t9*? zF8g+0r7-#bIFaD!>^{g_6=-!=D>nLUhiu^1H&J?{xSq97JL{!=edBp?2mZSu!-1s* zMrBvoe!JS;T+XkrZ?D(4H}%b(cDvGcD3Qn>f$93km*G^++c&!6UaO;cHDUm}F%mBn zJ!?~RyqffP9^psKjjb5ZQ8dO~-yLHBifp3ld`yYV7&NeLS3H#bUU$nQM5Yt15+Mgc z;}7(rFv7<6c2VbY(%in#u8~+S^Ui#2U)=@qg-33)VwY`wyK8HqzPAZaPZ#y-GP}E8 z-`vcuug~AUxjp}CeRGlnzwxrCy&O9y6M|*&(jyxxP8le}P=}M3p^40T)S&3|Z1K|k zYQ1)EZ(KL-X$|{}+r|%pTFo@Slbh9|*I$}fce_a!MrWzT8`gkuYUo-k z^D?28MbGy4oq@^y>D~;;+VPCddj_du|=m(LS8e_ z-5y#%$B%svv|AEhX#Twyth)0DyxlLFAG}`vLKVJa*`oGV2UWYToy@%-<-h}xI0<6} zhp}qQTkE~S_Q?CE^5Tn&*WI;izaa!{)wB9Rr{@)s(*?j8kY-Tb2d)<2{`lWzrT-{h z^GhM$xd(n?8IIL~m_X7^D@mOFgbi{qtm08MvHz1@oY$l_bUWTIn_bSovUxq8M(BtP1x1N%Ki#hDn%o4VPWF;fM@Jy3?Ll zj^lZC8|a}q+iQ38wRUz_HiO;bHj9EH)C+fwqF&4O?Oi!P$H8%ZcX@L+Kfl@DX5}WZ za@VV|n8|)t+JE0(X{;#W8dOCcUzwzkusft$!&HQO*VgP8`R(1^pz|ic!z)ev)YmdQ zi_L8ZezaW7&#$hp=Qp$cY?aJ*^A&b2U5`xdO3WP(-P?Y6{QBhi(uYs>(tn0XbEkT> ze`O@34N_kFuBf}Zw;k?|XUTP-EkBYY=!!dc8XZvnb zWO}@_{kzEQOv|^aWMtRvQd`XRvx`n;>{qL8Kd^dE-R)%FeoOGs_V(`P?CkP#b#rab zXTe>uO%dR9%_qR|;lb}!>U(^ls;UDhj|D1CP$kQxRWcT5und~M+exBM-m@0D?O|!$ zBw)$QzD;v#3;i!SQvE>?4FlB7W`L%MkV>2sS_5O-S4gz?UDU1$RlbOE06OfApmF-- zKsLh4@fSoDjxBQjUSk16k-OPgZ!d95>J(YE>Mci6XXtCOMbu8(q^opsy}mkMUS7|x zt}b4`y7+^(z1>^C4czCw@v~v~Xz2HQt``vF=K|A;brj>j=QLfn97pSN84%rl+i1&G zW?bA<#wcAUX%tmqr|@=h$0Z0VThxn-eD-EHySSar&k(6M`N}r)*vIWo_O%ZJoU|5G z1<7H}u(5godF#8%@>hWy4?6iX?KK2&N%X~9ySg!)wSX-+iB&pu;v+6YTI{gIoIjWh~%~ZM$-P?MzyEt2&f3>{3oUOOFr9RVw zop)e#U%CUEgrtL5nZr<_CjU_1S-F{%GZ8b-kG9 zlW{ZY*OT6=9}VLG#u9@K;-CnH5nw5}$o$X-eE+8Za_5iU13$G4$6_E5+;MdsSt)a~ zIvnmt$NP`|ol*b4ujazmo@-RukZ?#u2b9FISG5gznM zqwt{TA0!SDskWUWm*R#Z7?47Y+I3G*96k$*0tiV^)JlyfpWi#)kuPqg0QOtzl6y*W zPfz43={?M9zFEPhcm7s-Z!p$g9G=(d#rCT7;^o$vo!RTc+}-71Om@0W7H4h)@HR(^Wx_|7Q$of(ZL`-~XATFh41QVGb?qo^5<^8C}JN#fc}3d6s= z!})BpT+bKXEBI2Clw~*%(M1%^YEj-^=2w@y*)4|D+Z3w|;#m_1co```6Cs3hsBWMi z5O$EfM20glu#pQr4hP3!aR)QsB4(_B`n&4pmAze*>(EX8I`g-QwYHjdzTM33HrKn| zjb<&~WK;JutWIEKOPK+z2}FK^ZI<1*lDxhEVNf|%z0W}->imyXX55VdlMRKtz54c>+74v z0e_?MQJRdLQfm=q*CAE<71PgwyLL84@nc+L$y!@JLD)dx?t*un~tB^ z-UdwRf!<6-|D+owxt1I^a+0ps>wA8}PL8;9P-CPV)%~g`r1$x+R@vh-WI*_W(*XLV zM0ab`&-`fE8;mE-Q#frbT*-rFjTSvOCCxvMuMOo46^i-cyGshl4W{> z^X1*wueYo1VsUf5iJU6g7ZI|gW7f@PRo+|`SJ$i&wBdIt-nXP)Y9caOtfcFtN$MqY zi00s2V0wEE_eavh7j87~_C^OMhxih>H=kcy)+*Hz%5leC?5bhnk3((jXMKC^Xe<2r zW;eIzuddI(THRh4msjyTANT&=`1PpYleM`gB7^zW9!vq?NUC(3-CV3c|BrW{|37DE zn=g0Gm)78V_&fc%-+MMbuJ#|X#4O3~AWP=^OdM`fhuA9B{xRd=Q_{^lCz=^VP zlQbUPj!$3ix0~wqAM!7AX2spzQD=2nrH8BCVA;*DJ_O_JrNMR}r+Px>aDAB;*racd^N}*-CSkRUT#~5ZZ#v261>G zQtbydLDWaY14yfY4y*~eT;0SH#g^B3z6aGxAfJSLaXZbxwySL?q&zdn04FE6iVcd?%K%YAQG^rFZ^ zfq>U|IWLQaUv7I9=pw};sMdgtm`!@Eb+*g#2Hiin4uHmqLE-J*Z-3?sCxFbt#N zK{xxmq3OOhH(6cXE?2>I>R!*AyKpxOon6!rQ`q-*xHAH(l@hTUn9iZ+g9~`)WH&jW zFWa04mq>erQ{d2}=_JpIDXi746_Nu$%&F1gf?!8%TF0WU^#Ms9B z{V*;17>MyCYlt>lWaYvtH%ZC%u~>N3YOn|DjWx30Rq>`Q1^YyBn59Vq>L$$zbaKsl zd6+3JGa%&i=6Zzy8m=|DwIpwEsiXE>0Y+ z{=394{(ZO)L+uc3NY&ThX^K2tEaXdsDyAZ{1W9s>S0sqa_-x8nN-YgFnI3NV0e?j% zWLfR&Y!6Hfkv`Ub$5IBlxrKO^dCtsgkB13N#Xw_$l;IdyNMJl@Q|%t_;mRNKm&b`N z43mWElwL9vZ+J%l03ZNKL_t(c4o>%JuDh{5zR)k0*8KJLO;(!aEx>sd+6l5a1ohU- z0z0tK@dB8K443VTZN8oF78jfO+r|9)_7?k){igV27EO)b^A`uxgM-oOVee=h5*HMa zPWQp<@rP9XmwSJ1J@8Y@aI&2tC@NqxE9kOer#CXo5-VP3c;H`Mgcp}=EdFzPrR^8m zQix1^tp#5W{b^r3iV4RvYe*n~Wym_a*2APzp)`UmF8baXt4r{n4I;q>@meApci zy;0X0#Q+YpOBTRTGHH)Oc9nr&#N%D?HZxy0`5SLBd-G=Ys)+tDa*CO;okNvLCQiWL z?)vQR;>+#r)ztd+i^)gNdnZTJvP0Kf36KsU z$|wdo)c|olv4#hN9N}5@pjV&2e)H<9ufKY8cOf$FHkb_)*BaKl@$MpCe`&1$h~M_v z#jpR6K41O%`R{-A?9-2?&rc>t)BedM>c?WR#!dooiSCoAYg`)zxJa=Uk2hbxxL7vE z7ok?X$>O)Ky0f%5FGgs5y6{`lZ<|PRU8CRagqgFxjc28?{yLjo85npq?MtmsVwpHA zy5xen7vuCA6@(TL*W%jSZqhA}`et$9Ca2n)KP+Eu_OJeUmKK_~dK-=>LAdBSi+%^^ z#%^eb}}bswNY`E?U!W(&RQ!7$XZn-9RW_OHchr9U?4l!X0NxKWT6c(Ljp&;Phmr{ zS(i&iFoQGUr$R99VxBd?sc2XE{OkD!&OGEa2q|dTl%(^(Hy&<;MpT3!h1GbucpQSGf(+U-?(b(Mn=Bk3)a-cmJ0HRZd}D%DePW|S5Z z{kW@U^Zbfgi`HxQYP)r_0wxWbt*D|g2P2UHO>FmcCo`=%dic0r^{WLweV>!0JF`Zm zbzqqZ9zl^AM?_@OShZ}~z#t&P_0v^x1(J$3*RH=}oYVX`zb{`j-~weLPht8xfX0p{ z3<^MTg0g?{)_nWge0|nA+i0`x8*_Lbd8@vbPVz-JcWnZxw{zSjzFwXEG1*Oy{1?Xu zC(|Aa{q*?wcsM->#(isqdpO`CZIy5@0x_&Vw2d6^(ZGm?yH}?-f2i!weXV?5`mZl~ zcbjNlOzt+51PRM0tO`g!pz~uZD6GvaTmWNvSuVe@=Y*wb|CHl=yAIN#S3o|Ryxk`d zl!6+C=nFgVc8B|CCoV3yL*IRy7;j#cuU>Co=d+vQb$59cdF!y=M%#ryca0^swyTTT ztAB1jU%oJY_wx6jeR}fp^yK8=`N?oP8Vo$QYv>@R0DGd;GcuE9lZa@wU00V9?F=f= z1(V^5?kbo09uErq6=nDJy=6F=EZk2t{BBg_h2X;*It@&Bffmz9RQT@~&7cbkbH zTrD(J%}Lhp?Cb2+r^~Q-)vXT@9c4Ex>-OPLPs4hR*v!5uy~}Sdw&~x^r`0lR`UrIr z;VX0Noyhla=XdXzKs(R@mj#e(15hSkh;3st!~z7zQ~eWYmiNc-FSR3IXIdeOh?1w?UsiA3|NP4QhgbX86a8>vT8^<^RbT$|w$WZ#&qtrU zd^Y*?$Uelq&3~3*c0R^+F5@Kr6Q>hxk#* zn6UkKhkQ{&27A#x86#G8G0Q&e)j3n zv(w3B)E&fL4`N|S3MYZeWv^}w$@7QzPgv-K**5FFZTf}z7PHIhdcV58etmmwX~&;- zpB+yR4ts}jJRHKQ{CycOz0g=$OPo9e9KUZ5WY ztr#&$UXN-9xm=ER7Qj%bDwBgLawozcC>Yvy33y7O^l&m~1Oh`zC-7XsqYu4IA8ze> z<5~qNHL|9@xvl)!ov+>PZobeodTeyn9~>QZUyQ@kiFJzX+uz3_at_Xfm0m|{GBvD7 zbs?hVJx6JW9i6Z`#|xD6N09^nxH3{jEIOrQNDD$D%x@EBy-OW|@`g){| zPkJwoq8A_aklhX^)5GDjgZ|0T8+J{UaI!#D+>C_iI4JxEkO;?i!n!k9=IQxa(<>2E zo89iV*`2So`G5GQFJ65bowB7o?i~ftBUH|ykVG5wjG43j8ilUKOhlY&=4a`ypO=TC z{&`pK6fLj|$6GMwmSWtGq<@N4>-aVsIK za)gBX8XojO9FfKARET%`(|3JWYf9ANBF1g^#R`Cz1v+cAt1^O*!}op5e^MXjo6H(U zoPE&xrn<~fX}vz#|2^4uzF7ukeTocAt*X0M#dWdy&Hfe*z06%Xo|OGVKGJjaATTAQ zrXY1ghg>9HzfV$CPXOx8eP?Dl=j3%DCRlRDi9ei*X&`)s?cG{X zoIBkzJgj@iVQ(@X4xdf>&klOie%z0}$Yb+ju$%eCg!)4U>}PxLPcFmJGfXXLA-Mod z(>3rB20zIM1l_^a#45o9D{v8>h9|ozM7u(rfgMGdvT%dehQ;KRn=Jz*S|@OlVK_RD z$4A}K(P%iDPI}W}H0=jNF~0(*{^-$(xrV^Y7!(b_M@TZ7XH~gDR4O|Jb@jK_^AeNB z99%N+GmFl?0o>94_1{6HA%3UhkB-NuN8_Wz>99W`k6zas1!iR9=POb2$}WL)Mfsr} z9K_H5&FMAn=0Vjt;@d&&;5p`~U~$#>2U4X(OX3tX!v%)Qv`+2bU0Z6x(*Q~#xr|}sb5`Z#QLusVG|6q{ZR#X5PTgy8u8LA$mZb$+Kx6ke z(<3_=8vfx?aP+Ie^!Q}_G5hl4;qhT_I0$3M#KWDAQu{W7pj^%Eg4$;9^M(gTm ziELsd8*OLss2ju^u|eXNXC;=`jk~?!C{E(2DZ7yonH#&Am$c-1hu+JmpBKYu0`RJN zKNh&%iC`j-OTeFpWQR5eBU9Vfy1A&L>Gt-cYSmd}Y}f%u0e!!;gEH#s$)H0Pv98`9 zmg8aWSliHdu&Xa#g#T()eZ8KfXc7;sX>9sru#g-#4{K1s)m+0Hn(=S#qnlW-pY4YG zXjqNhZrGVRdLRIK_kkC34}h4$|9fWqD|_-L{C#=44omZBJ0&TQb?Ou}u zJYWCb4tC{{cbFV9VLCElP*;7%>JQ;NlV{e3F-qT|ovBhbRnn_=QJ8fEl9Z$U9{wDrBJFXHvo;o<%d*q`u5OW^& z4a_WAO_m4rp-NFHIR5OPN^EHE2RsduwA_ zG#PFX;3)2y`EECeyMtlU>&3d(-8!+Ay7~fGS7)%UW&j*cmZL7v`Z10SP$@h{Te(IY z?7`ph%^(JroCH~ry}ULyslMG|n+9ntn^b$0_3Hwyt+C6G$|k7s`aO;R^{<93?`Iz$ zKYM<7bb2rz55R00_(tN0Z1&h=syRS65pfdPqi61~E@JQcC^i09tA9N@(ocgp={(1` z0_<5ltJGvsUGQDrm}&GF{w5Fr3uF4!>h|N8$!`t_JF!9n?HS;Et{~(6T_@rPRiTb8 z0+SU@oM-5 zD}<}02_HP?`+a4-fU*PB4nU!GYI1ibqHN(d1yLnJuTC?-Y&-Y!;okoY-ZC;WJlKeH|^&&g*cY8HiPcTlwhV7W3VqhT+ zdu}}T;!%|J2K{i9_`u&ru7RotYIVN=Wi1jPICO8UQ6r&YIX6z4Uj*Agy0y z*fP)`u#XcdlPh6SmJ_&1FBl(1lW~ImuosSEFY!A8R8+tiichH?4^$+_9r?6AtJwz7*8CA8qZw{>o0K!X(WCE(FZAPcgCJPH~TI=hg!MQ|Z zGiiJ1HQj?lZ#4EI&@ZYW?)ZS&_x$q3JX+<^);uwj`rsru?DZmNU>iNmQ15zxXCTGn zYBUKnJ=S{T<~Z$a?dk5>mh4_ND1uD~ANBs;>XQMp8+pg*N+xx`SAw7FyOXAw*4iiG zu>MWj-E|I)s6QSDr$Z<7RH7ROM+7*Kna0lKw}kEu`pb_$*6b?zm3>@yCq9Dy&^n0? zKoj1nSU|`3`}Xitk063*cKRISR`AlC=J5GJ1jNK~V)hU|5#wYYG6RVp8wIDI4gQ_e zyj(gN*`(rZaO^xAcl%^4L|vmrhLkV9gL3ePS&7UXT5y)A96sy3_@9oV)8gY4fiu@|aAFxqNY_6TfV zv`=dJsrzuDNZjsG_hnHT)6TEUZt8vJ4a^tQ!9jQGIX%nrdp@u)(;2cf^)cgBa_c;Y3AZ)!nm zhd?qCmw)^WJjZZrND9rAy;ESOORy*!+qOAzCbsQ~ZQIU-e{9>fZF6E96WhF5YwdII z*^l@6>#wS-3tin+UDX%o#&t}&)yw*ZFCd>tM(Gsn5eQ3rRscbP3S6bNk`NwSVc}fQ zIUb=Cm6J#jCL1Iqv<>tnaPG9~8M3Ks=g7!)ju_%lGazICEn4IT$MK#e9je7y&iD z-J}bLNNTshG?w@(fzj>tB7a5-f``|4K_D{ob@As#r3=?3a!g!tW9TUbzeKa893GCu z3B_CLy%4p>ZSL)OJ5S&`{Y0=v55YmRfr1!Uh-!5H*6-OYdqN1Q*46Ri+z4(FJU-nZ zFJj#}B&}s!ytGb{e8)F1S=XI#l!U~LbihA(%ICXYm&5i&^v4!eK=}jsf*aCX*i4Av zxi-IJlAn8lANB6Y8SwDaUnhZFiF3GwfDu7BA8)zGOMm{!VNSVb2|F1>CX(hY@kwdd zl2d&#VStE7J|mWmg10O?i7Xp5m}JHpe3rt6`MS^DK+q8brTYj-1iQwC(I}Hoi5Hg1 zQ0^d-BoHtkQt8I(bMAxTRl#op4Yq@6SX9dMz;?lLBAz>>ozO-#z_k$^;=&MwkYy~M zk297lbS_$uGxG>m_tk^_pA*q)*i^HOmFH}l2^?V}*^o~YrDdW05;RVcPplToxLiVZpWLUmT;7z!{-^i?# zV;iiDQOMaMgwIBoB0-;^fBF5%a>2afS7;m^K?P_stcm+ZT`s}GPD}-m@%x6Yu2#dv z$_dRNzHR2l{NgF+D0Hh=i0$^Y?|(mz=9_EiXSgT5MGYW6g5&ayd5I5_P*N`~MTip9 zI?m{#t{2f|{t-qf z04Mx_wbS|=qZAVlHMe2rM}$B7-YBkHm^lP`((OsnlDwWu2INf?c4YM$)SFxZxvL|J zSv)ubqA0p9spM|bRy3aplb{4wfZ16HesKk+mo5^tk{W;rC{2oxfJaCIS_Aw%H;f|m zHi)B8Q3K)Bl+sG1(!(+?l?!{WzZG@3R5up9D*gtZt0Tzf?>QEANK87g@aiV7=dB_^Y^Bd0t>j-W5){HuOFaKm*wD^?Ui<5zvH zvE_jOikIZ4ks(1Z8f68U5txPqnIs5GW<6Bfk(QY72HsqGRx_*_aZHeaMn1NF zF~N97q+RM3Gd~ga!VOZ7X*Iq)=R|2&fdx3I!nVhG(~^T6wr!Ni(417KFi>s~c|#11 zFx>H)US$^-ZgglbFr-q1MeuN8w${PQg82kvtmM@wS(aD#grRDBC#KlI&GG3=Ye5;6@p{lej0&+46}f~t=t&&No>$Oo zy2|?dGan@<-fSZt6ydgtN}~9kStC#ebzL%w`LPQyG)s=8?i!r$n&Cppk%tsJ=kY+o zgFYL(yW5sP>(IhByz61@A9-i!mi;?)xvmJ#)YbX!01uK-M;C4}=2T_|39y5UzU8|y zF%|zm^C!P}B`-qJ;4#Hekayvyke4yIoy^WAWPh?pTK_F{%R09Lr9)5=M+{j1xd~<- zRLB%}m|wm;UDfkG9n36gjs#0`+=lOlB>zW0f(%Z@m}~F%g^@M=;+U8Th=-AY3!-| z@}<&_Hf3ZWXtG$X^WBQHlb1HT z!p-rF#wa$+)nSdMpNq{dtv-P{w3n&v1z#DZ#NU0R=d+r(*K{O2*rnS*`)~vtLFfu2i*OvMWlZ{5UvU_&BlMB}4qc za#tantqHeoN6Rs*+1d=AxrQDv!#Z|NI7tCJtdPUf)|Qu0lcTsyv>Q&EgTm zd)ke(#b!?B7guFz^x05cFfE!fY5jd*T$|-Hxok?Eb3`^8jdo@P@)-h?Qk=GoQp9<+ zZ`&VML6OEyN2owqM<~qU^V^xIjK*K#Mazy|33e_jqnnP~uKls72njuXqnnE^0}qO} zj!OD!8c)DAp{hfFa8TyUBUkhVLReRzGCESf^CmRyT@t=)8`(^FHv$`=n^qL}!fYYQ zDpu7sGX&yngDkjM2;_zvgu@^U{=6Nj>LQ1KzSCNUO&=8&qfjgX;43MQiC3rKwbDE9 z?@T+wX#Ug8F2UoO9d<3)YnVUX9MYJX0C;cFkSKk)PrDm# zUtD|J`BT8#aElT)U7WQ(+pI))Xq)~R5hVm(aJ$7+j56UJTzXGArH$POS5@rR$6i`e zOWCGGk(R~zDFxzb`z+`X)$mY-bn(TgW6Rg7$;QiX9Ht=IWjJdzdVf~q;vqa@LaxRF z4MB;nLT5WK@+^;5Kew=4>Cl|O6e{>OJwuo=leRqTK6<^8Hqyo680^Lpk81Kyo!igp zY(FSmqYQ4Gy*2J`))YC{r$GaZkz=Qt)Q6q9J1-B|^^DPUP&-m=-rBJT%Bml>JI8vy zu!-&GIkX|Gyw%#vsArl9y%Rj>zd8sfdn&nn{TL$7y6jXU@hw?m%s8-pG-Hgy}}?jp{-(4-qn+Z8s^Hr;CWyE{jW1!*$pJp9794(vG0I^e({ z!|WMHG1;pnt7Pnx=VsI^#~CZ2Tv0jFr_E5wU(JW(%tO05ZCfh#9uv)kw)V!qyPd2* z0}zY!*`u#Gw6?m+m)&i+kJ^t;-~8K2p(gw+5_p)rLu+14{EBYY4UR)@=mI@flPA2k zzG@P8ieX%C+~2amq4ds7vD{jh-rH)f-7ubb`N2>vuHD!lypk{M-qJ|ye{ycnL9Bl? zcALF1^whs6>BT>-b<6u)sr1FT-jKuaYXuaT0|M+EIj*J=zG|CMocJpO%}yoa**O8ZjYWw+z${jFKS^7*={)9dW> zw%I!drF(?#YKzpbNPGvBXoqtH66m2T!uw<5=JmJ>wr!F9z3K7o4Gu?$17rr-1D+k5 zc_ORJ^Y!*HrQ|QARee#sN8<0rm5F`XKtd9VbI=VtW^SqVMB|g8@>X|FeqSaZ2j5Tc zms`o}2f}~`jXz$oTN(ER=MM;vUi+IolttIFu#|_Cqld~j651p;!k-^)4xfZ5xOz)b zV!ckSxV${w4g~)Efs!*fj&)jYGyDQaynl}KU$NU!G@j<@EyYxZ6TlNC;bp+EFSUGPEY91dp$N-Blet>*C ztawQ|Q9Ie1i1AzZ(%$q?C%CxUz^8l)xoB^}rm(W!PX5Krkk#$p#^b}9ZujZp!HDy5 z)VCSK=j-111%$vZcX>a+B$!1WKO(;+yAAY>v6plQ`I#P!2k=Jk{BB0zp8){6W!gY_ z1IrEFb7j;f#NUMV?KnSd|2q3(c)fqwyGzOO+tRMgNO&=IZ+yBTON%-6@Z*(QD%W73 z#~5Ka()&`(sfnJ`iy!>x+a2(F;&1yp-pY!B(w1#+&R!$-T+Rvd?egGw#_#~+0U_}3 zas3CiRSt0XR|`BanLQX0Stq7H&w~5LFyfh_q1)qaUsl%j&Yk0}9{bnH`xArjcC7u) z<(Bie&wu*%F-1lPqQk&|!O(4)o$1r~wjQYGeS5P@8FR(!EkN<-P23BSVxR+**&oKA z)Nv)pEw;<1_vO*^U(`9+AlU;SlMoATf6Zcmxa3NtFPM$s*WPOhP~;e5Y?9Fp>})98 zzaB1MhU$5!Cq0>d-JxzGxgw~|Gun-lX5IU}A-W+F^0&T?eGJ49Y;}mM(x6B_AOc>1 z!;HhO&Ulo%emt1*CwN2he$%-g{rj(diw~I3+m|IWl3ekeB1b9oD7UIZ+3T5p%QOMp zB?CcCMg5|297z)0_hC^S%HdOZ&Hx#}KjAs>xi$NI-1j`)cM1IBDDD*xov0iKHt|(4 zH1D?Y&Aw}>4d(Av>@Wd6$~Z%Ij{-{LY<>w=ks?{Kf0FZLUtQl=RgO1My;X?2NSQPW z3J`0iutjRksF&O;{rnx&vufVLe~t)g{6gVEUl+z3=oMAXe+%XQy+-VKzyyD`xhv1% zuj1{p#dm5Tfa87G{L4V*ez3`EEM~+nCr9=eH@IYw-P&^|NVM-jrAx0J-qN%O^;qQa zh#XK%?NiB*EwAX@s|DsL?>(bl0P>o%GNo_XG||o&gBVa zV;jS|GLX2&&g3*151ir&(SLGq-e>&#>o=V!|IkSedYEy7{A>$GWVNk@{F<`ywpn$X z^Dk3@V|pQld->h`7|nh92>c0A{jX$q%n$S&uVPlkzcUK|k@jmSZQ*YXx6*$W{vY|j z{+HSoFyNV8+cAL`)BgeWpK`Q@d~IP(Mf~RZKl=RN@c(<-|Ka*i*>S#M=ztCo{%?Hb|16L+#Q1mY?Vrj1!-((Uy{!L` z4iUotzwt>UecOv;%%Jk{quhj{{M5K1-vg- z2)7u)Dw`TdA>MamjXY>F_VjWSy>{(((%b0Y&RElBx^HM`{KEc6^#v!kyeFst_&8qB$^m9g9hHcVG=;_VCe&Q4ST!N zP7vs|xT)G|v({?fvO-oL#Dw@ru@q~EEP9_2A&?eM>CujQBr=`WQ;U7ShKWt-j))ZQEmC z?q8H`&{mlcZCYy-H3=VHVthcd%@*6I#V^;Dk5X#F1xp~3=wIhd+vZc?A6nI7=JbUz z_s;r<^%ufBf4@bO+`qwJ@Klv`v0h!Cect22a@DtsdFJ$*Om5sj{-V53?*2yNgDv#U zf${SNao@|k$#x(V)*ltd%w?XXLdC(cosYL-I_07f3WI`ne(>`Cwxf?K^r_(3mn{4) z8wTPEII!p525KW@?55fE&FMc8rGwhY9@-A0I|k+wLx|D>Ivh<4zL6WWF(jri;}{f) z?Bc#j_WdF&X{Qf;NytZik1L2Rra_ltH}`3o=X%k`)oEI6Uz?nWhG3nD+iladRj-5H+nyawFS4d!dYAUF0P9$4 zVLW(i;`MCs@LJ_eUo6}3;1>4zuueBXl73A1B9-*%qBSzp*sdggDvWk|wb^1-85q9| zUytCoufGJ@rF4!iPr%n@_GAKG;(YD>D=M}>L0W4Xg!)OZSriY0#)5(_j%vN z0OR=zktF0|BB`NaN$NP8sX5l^Qkad+Vw2elX-QR<>P{r+6pEPurWz@@?3DpJNOVKW z33)0ur6QFQid1N?B{a7`DH!5XPc8mff8JR_U+^m;ZGqthiz$Q129u1k(s+RK3a z&iz<)zWyUxXw;UUFvmrQz`YmM$A~vVG{z;1ni=Iw8By8vYbC$luFi0Oq4r^uMUIP>(~1YjAZ~J2fLEnjDO*NLstFy=#Wuw0u+tB}Luu$OKOYvt_H%}NnD4UuuOo^^PW`sxh zMSkY+)bNoI6XBePkiMjBqu4y(#H#_>pn5_(PGR9ki668}q*_3P<;p@2wi#4)M1`4o^M;>t8Qn~9gE0;30nxAFsys8chk0D9v zoH`ZPLewaD+zCn2?8{FO9e=}tE%pP_l^u+K%MBDKtdgo9Ab99t52cY3R`CcD{Fyiq zt?)|hiwnXtW)t997Qy;udS}>NnF5~4VH+qj^ijEudkOc55VD`;n-d3asTQ6AZAUYc zbN8Qxhx8}3%X*Dq^e;Q1D3$6c8#nRcv+Lnl&Y`~_gK#OB5=x>_WRy5_(7OT~P<*~& ze9jPD_^VxM8&u^-8DyoYm3#rQY3}4`z{lO>N7e5bFGsM_5j*Ycy@O_g6e@Bc@FTha z4Y{A_43`ydZ(!xUTuCFfaln}+{OWN4kcKmoi5xbo_i;3+iP7a~by~j0G9Ph@II@UY zGNoYLCJm^A*qmJCLr;1lW z2LI;Kx{P?qw%b`ElOLV0uLkaePL@5~<%1P#bEnKo$;RkvSyMioJ?^lX3k>@Y3*~#`7YzqT@|l5aZ%2yL3>8d$96W z^zbZwT`s)O3yn05K{w6DERxkA_6t$E$-eW0@9Gv_7UjvIdmmF+=l>KQ`L)D)8{pyd zDCPI5a^1sF@S^kPp7{QOtgT#%E5xyZXn=?W9Tee^Yz#UN+wYKwS0s`PiRBi>ie_D|`G`ehDHJyEaclSWC7Q!Qt`!wD37*=F~6Ss>(V$m}==+eGK6Y zzA7B2SU&ZT^ldv;Atv8i{>eHM{D)Kc1mZUOax9Gxy<;oKBq}#de;~&~9S&)7U@zx1 z@ZwqZ=XF*43u`DiIIdh*vu4Iar$%-)ee~a&>>NIy*8{!o&%OE6yM~UEsgb_I_aQ&4 zDWaSW+zayv@zEmk{;8`1yZ`pT=R2qk7=*u-Nx?v@XD&_Ziwzzhj@A4b={2*BEheJ08H?vo@ zX%&602A-0yLl~Cdk@afn5%Q=r+*mx#Dq6QXTs0UYXR9~w=PxG{lni>_kKZi|-d?|; zJ|jMJN`O%obsDtZFUrfb3dr{iM?MY~Wx>-BeE&zuF#Xs2b{m_pXzkc842w!CTvbzD zjEYX`H;6-o=#C1%8=9)xq&M3zSc(}G2+QO_g?o`IQ??Re{Zch;TAE1Gu#;IG4UZ&V z5P{1Z%=0Ixo7a5ux95m#RXX3)#I;zT?*op&jNsDj;}GIZ<+`IDjLjaf>G3D2ax|2z zTn46ⅆ18tA-t2`mLUL8wu7hR7aNV=)aC4f;ky1qvM;zsbBO3jp23BRIRJ|@cpGhES~}D>y+^%wDFeeP=%G+6ralz9NM=?UA#LnuX2v9KYNa#z z<}L19DrGsgr)8H4udfeX{(Awxj8DbCFk*iHIIr(K|5y_QUMNpxj!aIijD)Bx zHgf{u4C1!%MAm^(!^({y%i!c9%LW(J>Vo2M2X}Q&(w`wP0pe=gJ}MY@O|~DM3q95& zZ}ONl9Et1Itqbgyc~rt3r>>&a!BO441_--FMPP4d0!2PshLxpPTO##%)=< zCci_HuZJtXS@K3{&P9f+sJ7$sVMxub>kOD2cN6X6G(U}xTCO{_N4t|tu7_i~CpF4_ zqpf9qop3S1Jk?}?aePFS`4qJsVBmthx7CEU+$#<{L=b&tU$v)njV&^~UW4tXQg#>zFnAH)xAy)wvAjN?c(R~7 zL|JXrx;78_Cyqt}BOrx1$s`P9bK2jWZPo^Y=VU+N_Yy=^=1-v(+QIs*P=jRpaMQL* zQ_P1|?VQSU3*^ie*Dz!HT1h%ZZ?y(5c(r*o9}il-eswh zZN}-gIxxy4pAl@scPQuy6jkM&Z1n+B6P2Ypv%~FKy&kcNXA>9A{BsI*i&iJ~7G&D6 zT52HPq4arY@ZHZ4b8 zuu?d7v1$`@w<;iMSdkxP8lWa_1jWv@w7LShH3(~xM58eag7w?fy!^3wKWw($&TfB% z`~{q$jt~81H$kCk!JU-$heSE~Mu<|T(7bF>R>+{by840#&6?XR)XK#1_^83EYhp^u z4Wx2Hzh%8!LVSgOoyt&`Js>iV>iQ-rgt#CPBaI2u=B0XvRAG)bTyzUgq^W`52|n>A z#SU`!$uLcp{wyeg#I31ohDIffQ+rc6qnsYMYPjz*tFy% zE$Mz@C;6RE<=N`rLtU?p;P0GQ+>V=pi8ioIN?K;X(y8-(o?pr}-4$7i828M*9 zuW}oY!s;nuO{J7~>ek1U{Z&^R?TgAwjm=AE`GTg+)gqtbW7kQ^ z^6k<=?a*oh>;rQmM;>87bMm*{rzJ4k>`>$ocZOd}J3+HDDXe4lrmgeS6;9?GZbDkQL?(JdMfho zuC&k(*_MzfN6L0Tc3b}j>#&VlL%GpfuJA;-z0OuY*NOS0&L5th=N`X;inZHjywq#d z)~wfKbW?e1%G`KDSEG(8c(O|^M3cNF*mF?9hwPonHqfq>&jkl{Ing4S&I5}ABaj=| zbC@9+G!asQf_Mymkl|6^^on*ph&gpTxo=$vmm2o}K`Jp~4!2Ich|CCy{nJj)29N9W z{)5rLWSWhP9Nm!qjXVJ)+HN&9sCJ=w11Y06E2X_uK;}8ty-zB}BOQsA)>{_LB)W-- zDW5jnOEQ_<#N=jvZ3ZF63Vn_n5|rB zM6k7tUy3B(ac}`J^F-}k@`W$A_KQ#@nXgNMl_8;R`dyPoBG?{;PAT~S@`ERe zzn#Qjj_$%H!q7&*4P9=}1IyIBDyz)`3s&e>lq-Vk6~AXDA&-JVP5Py6Hk_Jf4Fg0* zv&@v};1CDf>{WDOcawzQ+<$jls$jOvfHMqrU;kNVDWpde8@cfv3-r&v;lCD|6gPo` z8)1uCD?%AaL_=V1bb%>Nn#wm81pqoT5p{p4LV(d;4k4+BDLa@q!{Eb%yJ{94?jCfF z)i0!by&$K09Bq`=(9(WIjHg>;mNAMBSMj2bJT8Gf67M(_Vf6vWI!?YVZy%-Z^hHc7 zAI0^%@$9BP53EVU`O+op`_&3fhVc_3oX9(GvxS_}tn7Be^%!fH9=c%{iwaZCMuw%M z3Qe8TLY&83(7?R*lnOm^F6cAbQe_>?ilal(w-XtD>V?T`!8OXmBSxRj93BMncPx-L zId4aJXn|t2%Hu_6MQ<1FPL8S)t9ZO1b3zq{=%?FrImXEREZWu-DD=yTABaUZXZ7iH zJbp-4G`*pj$ns>-K^l%Cq;+M}!SP3hfif*t+Vh;|D@1#lfAbQf!m z9+u}{jp^RIZGtweR$x;EB#!H)F+iT8=AbmUaZNtz@S_&0qb^rM!~9P(m#^0FTmm)( z5=9Y43au_Ocb3jx4Rwd_AnGHZ8|JV)e;^Mr6*^@0M^c^~18ot?=G!yCBHHV@=C_`? zF!f|$K&!UtE?-XTi-ENIEhoi%W9! z;jS^&(g~t{!Q!)^&-(r7XU|^N%WRQ9dTmrr5~WDjPjzoi*;)qP^?h{Ao7&md@YA*1 z;>LqL8@SJH>QcQ}{UjpXUgv){=HdL!`}ZuN&m$?T)AF2FCgV;7k))$ozEMtMgB$IN z;pV@yB#eK?HtI}|g!}p`yv-E27TQ&*2c(Y__iA5>p~={+LPmd%@!LXUfowS6OTH@TFIqY zG@vr0Cu5gkUql!PV(KAXt{LI3Mil)gfjtCFofMt0lb>ZZlJkJ zDzuLw*xcGVvy37O3lXOyVKkS|$XEDr*?!GfT*q)({q%9ed#?~)A4Yj{wJ}AqRWV&7R(myNYekfn!`%&VJ_E zUa6a{mXoEdN4vH!8{vWv25Wl7fyd^S|0hwO9L^;BUVLn;nscEqOMthYI+{9==$yJx z({9PR%Wwqr&e+tu_&uJWj2&DbLRI}{*mI*$_Blj|Nnb`qhkcd@nNP@XSQ(ALh8N4Dz{=Ss~1Sj6~R>V5f zJ59ZViCUe<;wgXc>mS4<3Tk-Iv4F#|HgeyI@IE2BGx8VKjCwHO5GV8@1u2wdFs(5(WEYndRoE*m>!=t^7bi3x zoR|1kSNy8auQG^&AphtDE2E@NHZ}{ATk^}q6O5d;#FZPM4ihRrT;vIjir zO!RzId^hVxs_hUmHWE5Unqk7yAoBtd+)cMNa5Ew%UAVX;v)jmJvJ<%s)^_bGE1tM@ zUc>u#TEFf(KmMX5FsO~HM*n^8%0O50_TS9J#(g*w_y}Rvnx8-E9WF}^2@diDovRK` zVUf_BqNg(xNJEK~ESpdnVC?-1^WAG~VI%c!n2%wqz1HwCG&SC}(QgLNt(3_ffsLH}bxoJ^Tybh=(Pyy{C&FmF?+_pU&1&1u;p8bsS z_nYHeBgUv@HTyI8@0(OzZ!{FyJiwtrPLPJx0a+jRJTQ{gX2|U&l48ER##PC>Bc)B7 z%jHJ)%16jduK>Tmb9j zLr+D+){w0`aFkY$w+BtXSulA6Adk|`dVc6Bo%6v;>L%Xm`w$mr3w|fsL4j46geHd^ zqRxH$RH}u{=9qYz4zFA$Xhb!LuH#M^s)Qz45!cE`WbEWxO<2kgce}auRp}?OSw;R= z35{FYB6ynDBvCeL3RX@CsC}b{;$Nwg(hvb6LtL6ISfYmx8MAp9w}qgs47RCJ*)7TT zqiXFF(d3S~HTYpJy!YnQJ=Mx5OJsp95<=R%`nB-eI-~$WlgcN!S;~ zSbujZXPM*apJ7qsRDDXPZfGDhM?+{2Qz|(9Hp`YDV8T#Vldh?d3ztNJ_o-(}7RM^R zne4jLPcfxGRH(CO<$kU)DZGW%d#SHscbErYiU1pjs=3+*GC-ZHofxDcXoBY2;MMHg zGbBGZaJu}$X1KI-o6I#-AL;QhF}wv&8oeK9h|7MKeG zg3+9B>4!42@E}=pDe~$G{B4gI;qoo|`yA1XD3OJlO|5_x7gTMew9|%(0ueWsN0MFq zU5tf20V`a)eP60wosMPt=|Xxmi7sy7@=DJ{9F(h*41%lBo`BMwer3o(j&Mz+36Vtp zd{Q5VV*dWprhAX{*eA}v0W|wFq}DRLXhQu-^;~Pwid~{}t9!T9CrAiM0fKmF^kkOF z3thu25zm-L?$Tdgrj*NV=D{)w=_ND`%RhTyUZ4==v!{A&oCl$*OV1)i9bO%s996s0 zzl#h`s(~X!o+?A(8SOmOh+1D#XBo2TtbS|jNjb0Z0t@VC5yJg=Z$1QsDK=SG z<8Um$WPmT%)>ItNp$6LYn&r}Tq=#fK2l8(3n7*DOpa0*XxBdG|jt=fIM948;oLSQI zVTdNMae)f-VJ_)FU|8!@5t?|4#rSK!fw-h+HSA$kHiGm|fh+ijEWtZCN9q91&E@AI z3MZrKN^`|sNhGu|9g@du1zyq#zuEpr4zc)r#T;U%i85MT`Fiod^8}txGT5uZ!SO`{ z4Yj*^9=#b+kdhpERwBPOv9Cx{^BCS&QH)Q-t{n$kw?Eg-p6yE=n6lC4NE)Y;pLvZ1 zMFg0p<~QTLQpjaE8Tw)_X-%z(fPVd zQ8*q^=oijM_e$`^pLG*c;^g6iffoG3-$?1ebG13GKh;6D_rim8F40DM{s4e!j|A{EiR9UIu z<+Sr)PLsx^UqtsXZNohiIb1b}%dVfSuovA%Sziv{qnc?P}=bS0x zE(-}7H&%4d7+*h_iTG<$%YBnHRqcg0SwX;>m zIOm8GNK^R|Y=-JXf>&L6x?1n)tIEespEL8Y-1PbbDu-egr)OHRnP2MiXt!XHFZ7|> z1~pTF{i1sg)h65gqY_5do>wC0Z+ zTRv!!r=d+@Wx#;58w2;o)MeBhAn@Ln;6luDK`DJ4xxuAR(!c?XFiv5Z_LJldJ7ka$ zIi-)W>4)?+QRE%Q6@Pv6S9wW9lutRd-W&x5eN%R`XQ6BW6~g0LAsR_Uz^Vj>Ca?x- zM^U`DNZ|=u|BpV-xo=gm|9P5r~TXeJb@)wV}d%)y3Tk_F#*-v5P4c)V=5 zj!`X@B`9jsfyg3ipCjKE5R!?uJGU~x#z7MJqkUGFYw$(+tWg{$y!W7-HvMF{vyv9= zrJ}rS>|IgBuXMo(T^BBYXfQI9RMgxVQITf4@vNL=c&gKb+3Kr>{9S61BfR)z!1s2H z?A;vY0#Tu-f7f|*2v@~sZ2;^Go4fuf#zq=F!sIOn`VjRj#<8Jzpmd!(os6JTM)1aD z_`%&Sk%vQ}Myb%qVJ+tJLZs(h)Jrn-I;pO>XIt;+2N^pGf+W+O&x!Sof$&_}rIGB3|XY_F$bzI+o5XgZcc& zM$353K2f6+_sIQ4IDvIQ$WZ?9Z0(wC5(43QKj6va1<|&==`jZgRjpPas^KbJoZ9t0 z=JjYVexmZp7cfVBHRtecNnXj;gBGGm))Jl|Zw1J9oUn422Vh-jI6-PC7wIKlq+}cu zqg6Eya7nOCU^CaZ0)E4ExLXa2a%a-?k41x?uMSdV-J^KmT?urGaeNfH0amkg9Vw&S zKnICi%_V?Pm8H(q(JalQhswF2JFxY{VLe9WERrqP2lW}wT&iY zyQL4)V?m4DB99afLlRIF;ILeXNXdx&{LHX^CZ$WrsURvqR-%oE3^GQNZC&ot6>?9T zUUK)?GdTr|zlQg;b-%CsTH)|$&(4dr--dfJ=vwF;9SDl^Qn>{i2x`U0_E^X|GqDq~ z=A9ixIdMN!o`p(4@zc%z{CM9=FroC<&Ac8IoSM=DZWj6XeO*#{>hf%jWfs~7vJ z0b*LBAG!Dy51PB@@sl!jjhG@LRz*{0Vp)j!8fi#`4(9f@Ly7Z@h$LudqQC6Y4d+Q{ zX@8{o%4Ac?LVO(PBW)9oh(C7^3R*g=8wDUGELbkqJl!3Y*7&KEH!yNe6f~C*6kCS2 z$`O#$uVU$OK?eYeG#N8XW1{=NSlN20m816MlV!zDx;~B#7%~S@p1`vBdbswbCvyv6IKAkaDo$*L@!DnTm~2nWRD zJWsk_#N0}~f4(Vb#1kEdWJH(6wFTi7o~fNo0n35#x?q3bpa$D`Hx|i?;`adh%8M3< zHp>b58<;JYAZo$@Enj3mexP@_irlXL%IXu5qJ2HtvNK=2X!w_NfIxTUuLTjP#Zp25 zw}m?cVtV#y=9)r|=0`r0qzQAX2|6B9P;f7$BVb>4^gHihl}L3#G|UK;04L7G;!3=a zLv?`T5*dK7nj@#E*yPUI9Fgx6obO@Mvr%ehb5A0F`7qbBE$io#$iFsoiO>%z>5fI* z=0Cs+6%rlUT`}dMaD|(tzoCAz&1$20YHnoDc7p6Q4FS1*=&;xRPOJ+iw94fQNr3+& zlLE(Hho!Hc{z#?INMKeD_YH-aZkom7=O`k@YK_}m&!+U%#b5`ozIbQQfw)j>l7=(d zZjvE=qDK*&w+`l`TlxXsh4X52;1Odqo52PQC<-Qsr3)w(WVGiFjF2+xSn>p@6yvy9 z!TqJ&YsBWkO)4iMBr~;gygs#Pt9nQvSWMzkrMXvCT0Rn{hjnY&8s`>W>YgG>9tVpp zImHlvF~uvwj2KF%CYd@)6DLB0@t*;hpKaXXgfjVAXlbQkYlBPJeZlPeN9G$_I;Wh~ ztH9yQl^5hIw!SPB{h>}*F1)>rKYz!Y{=gRT;_}Ee-p7YF3tguLF@}Q{8!%>$c0(C@ z|EmaN@cM)~8(j%^xlVug_rEl=8BqU>@!cK5C4EWR=!N+t-=+mKs|a!z z{9t&ce7LWUYjY$7FIoXM70Jk@CL1PPDfRe7F7pfOc1D@*hE{A5(I{@g#zv6|Bv{Wn zh=f@u)9-gxQqV9mCN4!ORX!#IW-GmY9gZwvs|Avu?(ARg{M>lrq-081I|b1d8O&9~ zSnn&jgyjUxEoeWm9vvGqpjPyA1QjbVZm=?$$xAH)bW@LuMGnb(#kbeAx%puu;*i9o zZ*2Lu@QdEMQa8%p1R0xKsqDRKp~)XGAao3S9+JT&4;+yMTO#vSqdXm2Gs;|Y%*|bm zrrK0zs$Nj$z4x5aT`5}-NsE|<<3C|(ti$BRYa(1F1&lh0vOuY!NkA$y#Reaw;t6&= zcJ0|f^e~I;RM#*?uZJk!8^ie!gq~pbt`eu*y8}uIRV0~gI|oEvN(3q{ zg?)ypCHq^KWIl}hing+%4Y=}{yA6v-HiDBT4O0xf{Tv>3YN5oN=0*0$)x&Q050tG> zNHx|+2(u7ZF&}r}jV3!~b|?#Tp~b$QZ1lsL5$)mES&m8gQ}31& z1;L_2Up0>IhK%WBnF%TA`^^2SL$q2qG165A7Q6fkjin1i=(o=#%7cS8Wipf(lK62e z7kM8qu`m_})(L;|N%kvs<;gs(5Qb46mRmbRylit&r1lLVCc%pLQy;)g!^N4YKDAn$ zJhn`K_Rx9Bfd{5(eS2es728BI66An5dqULqlc+lQ8MdR2h(treHlV!CBA-c@vNgPBr1xs3>!@ER-m7rrji>q5elnefjn#<*0k4@gSu3VCqIX7VuFLnQIghA38qtwhfMWMp4MEBLGsO`ZR>68J);yYX2d|?c^1#T>1;LI>}fQh6}sbzQ)D&eL-hzKkRbjRu7q1-SKTJ}Vsgg80*9MB;*NT8PwW@>`zI-Q3IH#r z=nvdyqY&+PBG=Ir3Dy!lb-B;UhlrGpL32^Ps;DKsOQNh#IOkZUX4Y{<2Ix_Fkr>L2 zQG)=^@}w7C0il7hLL$cY-kqswW^jm4KfNb)(dvv#Xa@3$sB(vZsay@5Q^RlJ`&hO6 zX~!E-8<WIVUVtpkJa1}x!$@hXgK0AnD<5ENwS z|3lt8I7ixb-NO@1Y#SXLlT4h6ZQHhOc5Isy+qP}nnb;HkazFR`R=v;nFMPGDx~o=I zchx!9-UruSYwzt@A{frI(qU}%TImp$d~+3NC;8L)@D_V5GJk}b&@r6RAsn(766wi+ z$@2M+9l@OiseJQztr92-wJN*|$GP%L8DeAg?6@{RcI8z$#4|tO8`Lq-GR6manxD3l z+${#i(&=1Z^)$Jxr4g=fC!Gj>6~@1$s8n5< zPTXp#w+wAOdI7eQn_=JXPY<1j3_z8K*@Yg1TppoqSjv%ZK4qgoX=!$zt4hGg9+an4 zKqKp^5KUo&6Ax0TnUFpBC>BxnKtU?)rLA0Q6*m+Yn12&8Ne$jpXi%aRil&@bQ5^g- zjtf+;wG-``^oDJwTO_{|su4{Vx_R|L7W6Oi{&zSF`c-@)=M>FlFD(|bTe$cic*xNx zTDo-9&jQ#j0Avu)sLkFaO>kZAlL6=-b+hKWBtJ0#(uP1fe9>`6_WDU!RM~4owUC3_ z9$cK!OtFc?KGY7D7IGjK{0rWwf<1rn6UyNcPDqvh`lBALW~n-X>$O$V=*Icj0sDD)N zZY+(MmEM8MGF0XUOqikfCCS=JT30P9Gtg!Bcos7Z6h_kK8Nkf2keSbtPzX!E0^(Ge z(9YhQpxGvT4=Nddw@Hr{@fn>{%V@JaWC=M3YW_jypWr&e2Xse9fPR}9P|Jp2{;8P( z<;fZMtq~sN@iuv)&Y+lLv5$&06G~iW8rHF`)>ODguXj2hQvALI(}$S|4Le+*qGV+hE+Ngx)h^kqEzy#6P{782ZUuK4 z?T2H+MTXf&nngQqU)N>R-6FXMjGF~5NED`y=(fxMUM?J-ONI7}3hqSKtl77d0#S{& zV|U{<;0M=!GIT~Hqk^z7O1JwnwdkNsy!*opwak`PDu5^6g_J)EiHX!fi%(*fSl?Sy zxI-tsUzLPpJB8^K!k!f%%(6eSd%YNv$tFAI)&Z9hAt%B}Vr5IpWH}H{M_QR&j7Zus zZ8+L%u?WSVJ^rJYZEqOAX&LL4=Z9bX8x}$o9?R%RrhSV?E-D8`~g#vWe+X7 z!+}wkhvdWh$#pw+!8VLTXZ*jBJZ8-~!w6c)oHjraRM!#!?4|vR;XNBj2%>9~TA)uc z_OHTAz=O&H`~`KRwCVGMRFL zl9%5p1@&mT=QgFJ7=5vynM&}q=!xX!HCGN;hu{iPzl~`dDrAa7RkPd?(Z`fU44g81 zFYu*EO5D#hgws3|?x&*i3PI{!6dybAyP=hRo&Hc2nOgcP@dC80Af@G@Jb0c?fDAX` z$?nVs_L*+Arjhf2oW4y(+1aOd(C~BZHCC+HhB@gBFxS2``c<4+Occy>Ian31&}BhG zi^`3H>tf|`YM280&ESSpj6 z2>%^La*i3?>(XGGcb4R|tJCA>81TZ~rjj^nr@)-w-U_Wo5ec9G}Yz}U3a-xd|d>oY7 zQZ=_H_L(D=vxPVZF`rVDOw4JKxb-dzv2;I&@VwW#>Si3e~mw&$mh4H!K zzQr&_3igeW<~eJL?brTw^-oE|utlmqpG=AF4m$k^AdBql{Q{f>>QbYj$IAUZ&WNWRA+ zQE)>A4?*JDfcvH`o;E2g4vH4Bmo!49yO{3OJSVnBgGwb3iTAia3N$6YvxmJCJ+g*e z0iWZL!Z8d^XathDdoH^Dha&rr0_zt5lr5~^xeg|XAk|2cakVK9XzT)F9{U|uWu~D~ zqw3E~FmH4VktOR3^b_HFX$1LCGLmNpy zPNEIe2rm*WdnAiH5iqWrfBI<_Imd+K6Df--!3G&I2 z^Ql-Sg~2=uy;VX2{BNXhikSplx3$)&E|qGavP#>I2kG&YVF z_9tAvrWW9#NUp;77kr2!LT_Y~@GjZ0s~~7HsB<+{(yds(bM_2z6wmV{^Y;r(0a_7z z=kKAP^FYyJ^A=l60(Zmrd88fnfyIxdCgmy7l+kDL!STTRJeirIDy-sgcjUY zq(ucen7!9HXt0=33&OcdcSA>g)r1Fu1tdz%r;88#*KuxGzI2j%r zGtT!4BcW}Gr8Sa&tlnrapl-&7!)6ga)wBxBsq9G_FR_k>S4h@n>wKh~f8JhLm*cb6 z=-Q_?Yv1CdJ6cj~?^h?}8VMb+@*|3F;@AXrq4+9{a%<*uTHkP!v5#~Rli~`rV7O}U z+F9FK^_Qw$4<58qP;dC);iDW$Bi9PV>(q(4jyO3KNP;3Ga|Kh{?fZDysy=ptq_f>r zk(H*5LrFwzOv)r<6HoVZB0-?_z72RFE8nvR!w9YX`Gmro0%^ckSxJPeSm!*b-g2Q3 zMuw}2j&wFXXlw_ng6vJhCwj?zGQb{HuDdhHLR@AgHt<|fujkN?&E z?4;gk1iD^MRZ;%72zHm8QYaE+G^M?s3cC!#YAzDpKQd77*80hi%nEIdny(iz>hA&L zZz%dWLQCzo$jwPNcJxwgqq=IYONcNTSnJIlHi^LGJ<}2nyta63YkQCQAC2N07SskU z!Zcf?+y8O_D2m@x+ice?)Dy_d*O2N35;IZ41(^pdgDi$Eg&pQ7j!`ng^%@uLCzQlZXff46IPuv?JF^IOd>b_DfdGGXjIq%zCwe9$-T-=r|yKTPTbdgoXy)+O0fXftW zC~hd6Jx|$!3SOOSzk(^jXein9ZeA&2sls{+$>Pxfi$8J74IT~b>@3U~3~E`o2anFo z2Gk|i30tpKE(%zkG|0i-ME*HEHOjoa`Wyb{85Uv?0RzAVw}cV5K_$P9|QCPNX% z9)Bqzj59@#aF^H8)qaeIL~vDY!K+nF`LjW9`&FOXIqx#wrsQBI&CwZCV!N=^Y8_02 zUP)03koC7=pK^m%Q9x=^2$%W;aeGc-s@;8Gr;WT-va}$>Jb;K52HxqRs|c^SWMyKs zK0>j-tx2a{gR}RNysN}?HP|%_P6phN(H8Ty|Hc*uSsu?>6i$u46dOtfkX?@@hXOuD zF1=tWKGmdo9$KUhuqt<_40zkOGkC^d+L>G{mI_2U9m^ra50JT;`*ZItmAVb5ivDy;xB8K8SQDuwtjw<`w<58GG6-IB@qOU~zW7f_ zVyU$@zwbrlP8;rkwTtxQ?>cl2yqDgSUq`Bf`EaOM!Uo?Bn`JB{q3|YAL(VslNttMP z0&?P1&>eQ`r1hF)D|`itABIPfK8B;{kC>!SH@LY7I)uX#1jtEitOWSD04jXP;8o4{ ze`IAjZiQSKhcH!jT@S*4^Q4*w2i?T>8P;kfTsQSyF-RsytYT3^NUmM}C9l5ZQ`^Si zw?XMP!a}$#njmwE1++M#3M#~hgA_ggb_%+|tu$hf)kCfi0g}R>HV55^<4*{iV71a* zlO&1Z_cgGC&@v0Q32el1kxzzb<7SH-fCSx|F}-C@GyD#dg7;5B*xjSXhyiy+CE^+|VTrG7Nj7q= zm-f_1+h6P2$c2sRDj~_A+_(xQT>c`M<*{D^tpmJXxW7dfwBB`KAdh(*`@PsqjdBjsEqHVjegiY4aT zIxQJ4icwky9xLw=Uhl0bjCzZ>q91QYAC!kTiCsHuJG=H!zMFsN?;!0x@03!;wi7#1 zjnAHjmAqZ0rRWacteEA(eN#xOL3l>6{9uO`9cYdS21jvRVb#^^FDJw;sW^ z#LEQ|2v`eIj68u@ier&?VGii&kLAV4kjzSsUDy?c13ik#4s>#q3U_XYzO}ke? z)c#g`G`kIw#02@d{bj*}8C1$PF#e5yQq*61`=2Csj_jFcWg6w;_AE5(^+~+M1XlVf zBfCNlJ$s@xtZ1%N8xjHDHBL?q$1pxB0(p`Z(iQV<7)#$90diqLa;x&bChVniPMuUI z@;^bobpohP#FG7)sw!MQn^Z}fNM;M^Nm%c+FS4+1I~nm1fH)Q-aNWdm=T6E8VRL$o zMkRSc{(f%UGI<8_wQDdTd!S%YY}khINE+(0xUF`7I~F`j;6T`36$7J#S5%`mSI60H z&R5O4mWXQ|##+0SNF#6M#U?l>^F=B^a}W*8zxUYPlw*jIWmnH$__ zxNi~7rdIogh|tJZI#w;5gs~tlwhLMS6{jPhNVI{__B-IL=tKdmkc&i+QXP-N2-TAt zw*nv0#16_Tf(3?UXQHo-OYNh9#+C~#32=Ysp-yj-ahR`zMwiA-L?3(0JejVa^UZHx z|9(WTQI=LmUmC$ljoLN-70hPO$gLqQrVW~eMHbvJ7TMm_Uc)d9D!znT{vb=V5YrV$ z+C(_-JgW3Z2MZYtys?*K;Gf1~YG-*I7h3kyt_8A^U2wiWU3aY>dO6C1>rOh*DoU0H zIpLqg3JkbhRe$=|RMH|C-$bPMR;UqCfK{69R)O_Dy;f6E!_+C}8u2kxf`$ry7h{;0 zw`0AKIS2`C$A!Q*TWg^JvDl5i8Q0X%Bn4^Pjl`-nTg$VU)}xdwZbBfnY_9Eq2kNkv z8#>g{M`df|*)(j}>$-cx64|o63U;27$9X>Q4)Rm_HVY0s&()QmzUATUjz9c|b^ePU zlr5;k;J~H`gP%jNKrUf{D>2(qPc^iP0x5?V!W;`Dp_<9V=TxzimCJu#pj*t)pYcI1b zAk*wtIPb?bV%`FovaMoYtMW?kcfOw_(O z2D*2A!FnV4M7H}@i6L64WUXUp%D36UF?uLi*o;Ym`ItWaAYD-8BZCxyvpP=7Z}&^c z?8s^kNp0nSFdGolWWku212Cde)S4oJ<0eN=^HpFNwh()3@s9Ys(siD3D)Dxa^>yN8 zpaU5^?@O)$nEVQ34_1yoDw$Fl$24CoB%PoegOc)2#;pj#)06EIJ2u5vXzV@tK+YT7 z?D3QUCN~bV2tBv~Ek1;U0E2h(Oa-&d6V4mmDH4>7$1%q8*j+$1Dlr~G7q=-3PksZq z^gh(I3OBiQ@-U>Eudx_!1TK8I7&DI@YId#w1@h&?URE}#pXntr!M=RZu8V=OKCS?V zAJQqg@eRMVj?rc^9*8OGC&Og?!I@>bw55W|^{3GG_Et>d!~_2by8SQ)|t^y7l{z0yotSH%uT%+ zGwRebu&N$&XB9_(9Q|>R)K-xg=rL0L>rs?0PkX&CJXtL=X;N5tXFdoVtdNfk3?xr# zEMKRhoN898_%Oo!bvXD%8~r16MeF3GAnygz5Lwh3s7)dJxo#Zs%LX1b4}C(PPdNMp zzL*G(K>as$Cj$B_3{w&`%6;I7!A98vozKSW5xKYT-NYk43fX~iCrc20!#k4vOa<9Z z>&z~LreeXVPG(s|pC7!3dPjZ7$&^g_UM>q73a{?VC9Z4wU-dRPe_@A^f6V zTOj_%%C9ylL|k_ID2fB2BDNDkl3hnA{xEVl%X#I9sC@`(tq3J&e>_NYZTc{EE>!9j zYVba~R@;lohXjE+VdQWvu@OmS`ty0%yLRR4t6hP8xjr_?8y@t_;Ku|?PC5B>cLHAgqz9X`@3M3mS{4wLcqb23F>c9@G1eow}ay7_% z(n_{g4R~5?6IFI7pq@eq_Etb-jR7Uwg~R+Fz`OQmi&z1i^*5xVBcid8n@;^iE}YPm zs(mc(a4q=489ez*1-!?QXJkX%^ri9{`g}9&upTJ_ZMQxm7#C#*SWrMXsq^d#Dr%;p zlo+^B2S)xTDR=lHf5i~D0OP!#C%Bds)UoC7BpyRFC9xsKfV;5od1f)2L|mu28Ong> z2;^aARD;^k1JE^W|4wcrC|LuCvCFPbX0ZiOa7x3pxsoaQHf9ZhQ{hH&AQylDwFrSk zUSwm@#oOp}&hYFgf?;F!bizEML9`fsUYPd`?c`SQ4#eV9QuFHS=|Ah}HtatRym3X1 z^HX4`9d6@%EpzeFy)=7aH*DYT$#b+o+~t#Tldz~An?9j;L?in{z4nDaImh-%^$lN` z6NJoyzW5BwFiT+=cQz0Gi-4pgPXYBJL!Gb`B%=&Ok|Q{JM|*uU+cZ%ZtSN{7bxW(* z2+o>@*?X#iKaC!yQm(?FI1xEQ63ry9SuutUZllwk&Z;|oPk7w>yRnTZc@4o_VG#=d ztFRAX5+zNdnKIl;;D9#&yNGQ>PmGi@ZaCdq#COh>l5OfQ0$qNPpaHo9$4!GVy1|Q3 z)6f@brLTi&&g(L~+eZNk>?Os_(kz5n{cIX%G&?PKzkK12l?~X-GeZh#sBj{=lc?24=&T76 zNuUsAFh-IIUl+&E7H263W}Cx8Gfhp05jL#*Mrou)EL?gN&$an!PbB%yb&-+f5iO#C zsxm6fK?j}bJr-zI zZIO^1FN_w@C@B*#5cWF~0h04HkFhK!@ns~nTAoZ9L|rInB9O9?cF#29#LwtDjv$Vz z!M}g;81ryk{mx;FR~sV7y3ddHFHvgErm{?eakR=rh((2T_5 zX8zw{>%w;aiqvm%aW(u|90*TDQI~_e_^Db99m}c-T9GAH$RgCoJaBn~>-rS1SC7hE zD7{p5{fgz;=+Zppz)uOEsE~;)o*=c}06+BNN;U)L7OkJ-&RpwyJ6N>H1_*WcKqIpo z8iX)uL0h^9Oo7m{4HJzpXdbD(C<+=LCR_E>;x`FAMG%x zjXu^ozM#p2qcY}3`sH)gv!0wInnin>=W&Fm{$~Yv%TLd?F>{xgO4z=Bw>5;#C#D^pd*eGclLRUA{W;5}#~VP&=cc=o zntTuDA*O;V2r<$&XT%oa(wucE5z*(p^e>rC z&5EITM76<#}*51b4udWXFImqpAiP&dH(JAdHq(skvSVf|g80Y|MXO6MZ9>LwQ_aehgWH zzxF0ovzZr}H3|dBx9q&^3pa%J%LN-8h(5Dweuu@$;OmAP<7%*@)sb8>leHft0j1Qm z*CQHS(y6E>^PmXukl-zf>MKW9w8K}omJdc&wMvB$o-T)FP_;>K)eZzydoUX|E|oWq zi#Kd7E6XAZqU5eB3OH(IPYga@f)4_wKvD5{^DE<8O?KQZq>9pIxNVnRy>pEL3%&NF zN+CTBHth?Z!526ztu1*<9$|bX0{0Ju=Q~>){UtdTxnQj0@v*a(2ZBH_ny9H|*#>{8 zN5K+h!WTJOpE{@5To_*xrR=(}+N_EDwg(-HE!|Xx(R>Y+(=}L?}hmRYKqs}RytTQMO<>f{%aN;smG`LO&6 zN?=cY-fF4+>X%7GM5~I0d{ke$soOIBC2Hv9Bb! z90tU~nYlW*0}@IuD@78tiVs<+7!@I%D8QRNb8bc=D2w1m3?_c(aDM-7JoNDDanh8= zcDIi-sw!shsj4OB+T%3APdNnRZc4=eBcJK5kI8SO@+K{~Hf%}B22S=5EcOPdN9>cB1|X ziD-zrfUy1iLRPvZtM_x^Xb|3|pA(hf^DtxJuX%QTDg!*efo2cS0fg}dTU_6*wK49)k*ZRhUnIgY?pmcIUeaUl^ z!~|I&_rQp8eh@#BaLrEOcbBUu*@K3|;oc`7~YOC;fXiq9>ZVp;53g|$sH@I)#7 zkzjxc-8IAVT0pdM@!l|wy#&$xQSltaHy%=Y;XLs;^X6f|Q(#;Dr$K~Llknj37v>pk zp<81oiYJ?w>*6AbAW^1A6Eh;lQAFb_$_=jhG@x)~cUbeLz2Mn|p=0T$niR^|7H8`=T*0KbG6$&suM%gC7T3DJh3D0r0Paih z7FIsr>y(nZ{A{|({YFot2liF%A z{LO3VBB9I-Bkr=ICZmr+b+XHPoBsZ{-WKnq;Z>d<8(C5t;kd)YHl#eB#FPd?EU_ao zk}p-H6_4iR@v7Hpbsg<^iu#f|-P|g7cJ_{a0%lOxPwEOg9pVyw5`1Yx2Wda}%y`rB zU7aG`HA78s4RCsFf5{^|W;Q_yD6J;ydiT~;j5~GvhKft12`~?T8%k)0$h!n!jq!d@ z2Foaz$0YB^_j)?raR@Fl36VcnddXX}=My{t1=Z4vGWsK9rpiyAs_l7?)7PREC&NiJ z^4e7#@&|%si;rAYXVZc59=L5@s&lJd3}I=t>c-zPUXEEt+HMDNkSb{k&AMU+#OAf zxJ+0r>Dl?0muqTYxlLqW2Fa(&@;12%LBlDUl!&}vK3$}8^Kq4|9kE#;5 zxergyNPYo|ig;5pvAu(F98QzxeZ@l!l+j56f#7++DwM;dbb|KBV7w6d&^UF>SIg0= zS#X$HM+yPSf_OdY<*cj! zom+AI%AAhDSVj2(QdsLyPZU&c0=bw_9TAF{9CO9%j<(rudzIIvI;dAiaGzX-BmHWA z2B}}vhx2sCjsv&%Rqhcnp=xEor8P!Ar&~|MU4C-y>YpI_3KQgi!ertV3wPO4aiaWC zyq29R?XPT&ngP5Cb8s*cMYH6HmX#UzfNi0kXv+gA^ae*{#T`mJ$fsXrwCaPnOs=P; zaYo109@c7UvA%O8B&8%ti22F1*RSZk(H>!hYS8X^Dkm+zZ!$*@1sz?I_c|NKjXnn( zVaEh)5Z2hcOBF+eeEN<*Wj@wb{?4j6cw%W22B~`P5?EA-Z29FF?gNQ*vc$0v+Vy#$ z>_o^=|KLmQklt&NMc^RQP2jqK6q^{;^qfE|65TX7 z%b^8u>4AmVMGgOuMD*oTAq_;Ag4u!y-M2j?9tHzfk$q;0K;MXO>eoV{EeY_yifIfQ z`mR5#nA6|-;)jBo6N1pkgP!Xo)yGWBoRuJtDk+lD+EH;-v^y>PgEhzSlHC69Q4tqq z>^m!G3qPD3VN7t3H%l1rN$B#dz+9(vVd^o%)5hZTXQ4SjQ&2d(ja@ABSH(jk%G54r z_BnWy)(Qo{rcmSC;>j`_xSy&NGMSogP|X;e{Nykp{Qi^3?n7R8hM81jO*E=?u^zNW zBwzX;kJ5I|VmoZf1e-KG)pPUB!Cka!%s&$<;N!}=l%c3Bf9+xXfIk>X)6{Uv^zWxg zy-iWJ@dnsJcey&RErO;ikl2wzgbHYcQ>dB3Zxkpms9hmxoI@`~(l+nf1MN&HR{Acb z${%o?8L7_7U*lyEbaX5eSKTB~V}yi4;?9J8e}wt8<92h2S0U!h$Luf@;2~^7A2{xQ zdlb+OHP)i-^a|U+Z66SLr|`JmI->huE`VuJVv$FUKu5)H0sdJ3y2##U?!HUOW=y82 z7=QZR7W;=A&5K^<2F{SGXK&&D{J33g01`1=jBjh+fw_ zO~;<`4mt3@k8m{KjM`VvPIhMZhjVy2yx6rJ6bJax&rwo#2JO9_sy*I9|Lhh74sWT3 zhUTOO(yl+;uN)K4B% zVYdp^@n`Pk=xj6~0)}eZx{3L`h|2N6p@!{%tP2Oi0&U{e05!@0%7%CpyW1B;&mT!1 z&#T`lkQwoJ>-y-TN@bTx9=|3ug<$jA&E{HtlB|?sZ6x_0YuP?RqCNiE0Necvm7o-w z#8dPmor{~a5(CSkt2)X$yt*=Z!M+Ok{7+*P-;RupEr}|`f=;ww&^+YDf$ZnjOVlgh z7WMb93|Ef-f`G6c4P7Wdx|mQ2#ONfZ)Lf?jGC%Ri``VO?4Nl(V-c(62n%akYf&uv8lKUyg@l*vbtlgfsR^tpEhZo8lc z*{OQYE>?A8?$c0%6=3%-NM+06>vI|C#x!j@i<^bR;B!^VT$k&+c}9kB&o>$)&S|G_ zwXNx_?f0ReApX|lLrS7W8MXN;TYGU!5n0UAQ)YVcDwKMc0>7BkEV+iS^Q2Vyc-Cf z^VQh4y~#YJ0dyX`TQmU5M)hx?7Ro^MrH3k@4`y$yjfw7gOULK(n;!}d*sk59n>Keg zzW2*DgKz8Ir+1w%V@}kUCe#$Y44~V3Sew0B{oA5eDfBOKt?t6^$GfZo>p3?;VVGx>1%wu5fFa2%izAi`$vO?=*!j15n=mf z8}t!7>SL$Rm$n{h{A1ymk5dFIMy$Bz3Y)W$u1qPP_PIdkT|RmZhP49N1?5iWV@2B( z|EvHv<Wal-~=x|z@+kyORfs%ST)idS~<;Ct<4eEw*ZT@pgU zPzfQOczAC+$L@d;@J1z$zEQ*{*Adc61ks=NdfkSkBrGt%YW4qh=GHB74}P|4_AN)- z^(QPcBhn^#xx@UG>T=WMFQv1gRB#NG?C26tVWhbAp5mJ=MlWBaKcpXiSPk(0mSRI} zA?1lMtgz5wAxX;J4&axezVS~=kweU{KE!G5Vz_)r`YMiY(8!e6{7~|D@O@61_s#L6 zj9sf_f#30|Z=`)<&&tec$WW{4!#P_}l;re8=jhRPb(UqAxdFA40crq+H6*w^l2jhz zpn7u@m%U0IFHf|s+$#lv(lAFd@Jn1a!G2#TiwYuO3nabWX9Ogkc`DvP10j$JL}Itr_Njc}%o*0QgihG1;rce|wM!vu zo`z=Rsze5X;~2;bg3ggR$(Jyr-F{^eo4-4iSGqATe$+|L+S`CzI3Q>P^C`y#{83T%&VaQ7So`Zks!EPl-I+k zFD2J)lW!BqS>#fEjr2)yChTQo0P0*om$|^z!r>VxlL^kpPZ8StIR#=G#@FI}Z4Z#r zCwqiivGia?mTQpYdWUj^EquUiM#6=7g?TVXv_>}jvKA%qu3D@2hQ_$yY&)ZYfL=sQX!Aue(&-Ap={ z!m;Tks5Y)|xTH8JMH}iIKo+3Up;17+$p5 zhdxT-dkGtyCl&uV3phfUOubAF`gnT;K2g~o?N$4`51QQ z%>jMw6eKQb>vlV>qR#M)&aalao4;n;IM5q+hgr#8RLQ|;58AOV z`Ak-5iEXU$HLg&^BJpwnE<67yK0yT2IMRTspe%T3s<~m_2?|c=t?HH_foPq5kd$J3 z{>&lo4%Z?9=Q%b~8{eD_gFcereh_Xa^Zu$5*!OcNXcSR#ylBo38_`FIJyrYST;Hx; zIOrAJT5w|bwX5=-@R5eSP3TrRXe0j@Oaf4?-}3klj9`J~J!C+o_#k@*saH4qzk*PB z9}xEoiljvvJ7CBc5AmD$n8P2|72>cz)prXMwmvF6pGMeNVbl%~GqIO>hKp zU6)2e8E|oce96*b#|rZ1WCG|&2-c+Y&uAg#3bWAM_O8G~DM@QJ%QPfqoR{>{_(wEl zmI=z97TL9gB!l3*<`Ibp*br8QDOQS-tK68dn|8vBCc?`_GOp*O?(`E~%Kph8a8d+) zB@bJ$ZjS+KfI9sc>e*H1%(2;p+c5GlYgSBL^AvyobpcoPo(vpEzd>6~B}T(EXKSiN zVMZ_!QDxED*y~(lFO@JIO-N?;X|I{3$?62*<=mr?ab!Pia9wN0@c+khAVH;pWs((T zLNGmE>A834vv2y6t+hrX6lo-38CIxE_<16sZg@;{RD0WF4sY*>O!Xv@z<~$>S|Qp> zz+J+G%&Yb@vd=`xUc2)mmCHk zHL*RB7jiz(hAD^=d0cSLB@Er2#N5p%EIwFh3@~d%b)!gXdYFDI95jER77vQ-8Pu}Z zhZ@U{azN^=ObZGQx5|Na3TwjO6YvIedor9Si&n2ACg5*-n>VQd$1gNsz#PwOoI1~X zj+xT08%w_PG(tg|EXO9$L2dvmmbvvK`v#v;vUbj#G#=mpR!2E!&5GT>Qvh@nnivpl zlNd;jH%{UdbZ43fKR`Lz5ej!KhWd!6@jjN-iYU+pLk?Fr*^&!cH250~ z41%~M9Jd-S66^A^0Mb-X&vtmF#SXpn7csJ9z!eI10ZXKMU5dEWk2rsG2fbjXnK(X# z7U-N_b~CInA(A}f+e8;=Q~og;Y*9catvKE?6g%5q6>LgDb^tnNpKpK|?*UXC%`M$T zh>Yi&y)>c?Uts*t4l#vm=02*Nb)@@~uAw-^Ar8~I^J|idzoh8k`sL85q#U8db9(+0 z3<2MZ31l5mQA1;x9;sQP(`$e51!y@uRn!AA4udLt=~CoHkMrv{@2j<MFQ>(UjJDY&5)nniM}>_gjAyt;Nyh1{4L*wna9~o)GChZ)^sS7sdSPp zW#ZHj6?#n2&8F6A7afd?Uqr3|&g%>H&=_O4wd&kP(sBFr+*2y37K>=~l3_F75aKjg zhjS{^rO~M*xWR>!i5AVy81z_)^@1r_#tSMQ44UrW4rQuom5&?%^afeQiSt*7(qi9! z*PcKR0Sfbw8R*aY(gG@!vTh5ni`8C4%9_n5UwZ3EZ(~9`#IlMMk=?BHeEA!vWk0(R z1K>x*AlXR7sEP2m7ld$@wfH`^7V_K+c5&~2?uYY}iTVr|6)G!0(4n&_R5ib7wu#6j z$RrM{Gr{+#=nsA`Ty)^5y0xwlp0R8bk+XuxNSwcqScvy!E^KMhs>7a~v%lR@zC?cc zyr-VT7BKv|e~lXzufvHCt)&W++*jx=?JwOs3F|9q`SZIN?1AB(2;@(e(^g42L8`!L zKTL;VR^{Y1@dr1UQfq2*=Y+nW)4x^UvSrb>q~Icrf3n6q@Y}$+TkTlj0E!g4CnsB} zV#9(A`E2iwPjMlCq8J>wq^5_T$k^n=n{6w`Js$tNczh)!$B*o$L5U?tUr1>Z-hxHz zXLO!li+*Keyfkc>yqL-OHc3UZcNgNl-$&NVPQP`O^4%@IJYh;aSN-L&HdefSIejC` z!9|}@m#@prsgj+P#T>Vm;r{iy$AI~3oOSAB{7GSFa2ppbeeiLwnc+o!Y37(N6bZj!skQvLGL>rsTQ*Xw$K1p+#j8MjgTF@pugVrJW-n`I1qk$J5Did`utB|kdgR3?i@e4p-P=g zC%)VXHdWr}Qi0lfxOJc2eoC|#S*t|weOZqs@pXM-f2T^sdcwaiSpIp*@{jp^K)ul( z+WqofS;uIA`oGqJ0ZJF@N%y?=|M>;|cmL#mZJ0k7!vFnGIC>`kkwUhw9 z@m>1wkB|BP7o`KO4*tS*V7nd<{vH4SeoFG^9@?dVdV0?OFY@pAvcMN!{ax47;XgzA zXITFkS`pEVLyM$;XQ$bJ zcR73GKNpODGJF4+9Tijq-lc+`EyLP>;ny1f;Eg+$vNr!&qyBv$UuQTpa18`ks+wlZ zf6sLPi@VW%;ZGXZjTFuQHO*QuU(?)Rt52+2<3#){Au!5?EQrMF4= z)L70U#{8e-F&cP-;{~2G)ySs;!&)R8wqqnWq~!c>J~Q6y#-mNH*UOafSw!pR9ruu{ zd0b7{p&sg!vG%tI!RqT}yDf;cqX(;7%WHPQQ?z_t77<_g_l>4J;hhvezH3P{qxGS2 zQNHJN#id(|%JN8>sPc-k7f_L1cRRLi#k?zFe+2KB4PG5+klx0oA$CJrDBN?4W2yeN z%wwwU`Rrp)orBU5Rgl{jEGUbB!N<@Po_jPjv%;<@C_w63%F%vUDn8_ zl=#rbSUy%6sllb~=^sJD0KHF%HSj^HKvLwTTiAC|@NY;|3V(%`t@ZMov$9qHZkfVi zt9sJ7yZh&g7jOL&tarX3>hQaaS0kY}I|4U8AM4&u3KhlQysRpGKnpWN{i+3%rtI;e zy$J%0t5t_FTFUzZZt2I&_W5%&K7eEKMrr4B#`4{3+Oew2IxWdkPUEXD_Sxth@6=-6 zZL+=AL;EbB^+Ts7{?%rc40qDsZmXiZ>E?p<^k%3x1EgnnyOyDN(Le6#pVccIrd;M* zY=<3tFBiMjG2eO$B!!Pu%JOTAg1I9+x^51kbIM7tf7(e`(<3A3ha79K$|{U;O&B;4-X8zjd?W z8`v3au}q2j@ht6E)ZMoaH$V9*>u9JTsG8A686n)oITalfQ7t<`m~tumllv089#x3E zt?19^QLBC7qrtlbyhm`;=LEv{MndMRu9l>^<4V7~!RLnV2B;2DP+PXp1aZuFNOImP zhZUKy_wW@!0Y^Mxdf3jMkY^hsi^+Q3tpR9Z7r{ zTS5hAWo=XEgLQ4520m=)hU}~*Nw#hHTr@ApoDNH^)sB-_zf$t%D`ZT`J(Q!$qRMF^ zP$r+`PYm#b&6R))7C(x+x+f$#13Y{4Wmt=u58K9)51ll&x#WE*-(SIW$r^E5%NyGj z+LIQQ>(gH1sHw4jYFkK89%ghUqhiENnN z53rsRP2|JhC5#8`Jp1?6Yz7swLnm;5JBCOR&48-x^q7X}6^q5^CX@(yhE*FB<(o`C z_r^9-d?0}r{!TwWIw+nmyut$jx-43h>Ul?Wr|IB+9e6OfBdGo;A-0q%PJ1KU%hWP1W}B&Wa{+>p}L6X(V=%2uZB78gtgFU}jK4eG#{ zZs~qya7_P7{_}<4CsEK-Q+2r9nb4K0qZfy?tO=EQ;q1d!=b}EBncop?lkroXD}22* z*rw9MY}!|P7Gs8aVOcV1fd!P8Pw}%v1o!>mh{dK;-D)bY|7?CcugGo@9GMjee^O~` z7j#rb%m2=n@s6Cin~#SZs|KTj{?Wpi}eF@P~y)6 z&!VM}y{%W{v(JIAz9^$&4li;)5Dvm-Q?kS6tdeMhX)E4p`bG@hlJ5Tjl|X90Sr>ka znO!Id+BJ=kAxChP-`g({4IVnb9Cm^&uRrkital?3AzX)wz1=;wSvec;Q{NB$IC9;@ zLO=yuWQm^c_!=`f`6vtR>pvRwL{w9kWig$t_xJO|<8qdp%&0;m?Ze+w9^8>U9g?k@ zH|>W(JdD!aw5$C=;Ksh*#ZR{3aIasd?&HH0l#42+orb&ygCAJ?_?!n{Gw5LWW{V% zK0)g-D?eY<_p^Fh*t%}wQ`C^TQ?(5{!%YkYWN$9=5v6!>IPOL745Qmt-weDW@M@9I zDbzQZ1GP5uuRdB{=ZuEIq1bMCHbT-c@H&w{j=dFMio#O&LpU5g^i$30B6`6z%s6yK z>wGcI?mftV_pttF6?}a~U)9my1kry%!{)24pb=mFRQ38HC^zUf*m|;J`X2D6vF`f~ zH342E(6AGAsuce5nj$-?s;aUq-#=L0d6?awnuA&$Y3~F_fo6f`y38`%r$z3$+Us@0 z?HhJC3ikT-)n15=MB-Fmi=6JKV99U}GT3!&SrKH zI`RvB&~!SawzjD9MRxq) z)8(1xM>@Sx#2?L)_c~+yo14Dzp>oMy#Xh|C{JjWX`?*Enl@WLgIUM3QiQ!b{b+Ib4 zWq}9T!LoWZExw*sf1XwUn%U#hVISw1Vc09$GqL_gS;Xuc_;5ItSeuxBVL!%-PV8?) zV&|$rN1hk+r$#(M<69>8%0uSIIN?ll)$p|q|Jaz={FKWk*KL&buBgz~=UH_!E5A52 zzh_MJ?O|m-qpmo$Zj?hm8L|9f=mHzHZd6yN@=jsP8xh9^Wz~;I-8R*1R(g%V94kh< zF6!CJJepU3sk|?>opV2;Pn9LzX{%kOY^GJ=W0|+p4?kL?z;=mM6Ri->!54nwM}rSl z4JqeWioaxTzfgU|WA!>8iCj{E0_=d%3?-1l}?Z zhjFpUMQVJ7@^D%n%!()T>i)dCx2zwm%wf*n5JDzP!)d2%CLI2dB^k*Ip<{7@ zBpR8W8T*-X9p}8t)@xJr!+H?dUTC{sPvH+KwsAmkR?r*oz#WJ+TwwMiZWdK(3I@+98}co(8k9lJ^P59NR{x)H6k&srKtdTYr>$%xqn}gC!#S)I`0_|I) zl|#CQ#gEu!G?NNZ|B3w%lS4;_sHbFtu*cT^sW9cdUQFxNI4bMd_}V047yEr5ov##qkG!Dm3zGZq%3c6feA;vRY83ajhBpcA=yG)8&J#xv z9*w~ARF>JWO6?gxsQm!XhjqT5%^nXQ6D@hY>=(V^-e~9Dp}H~H+Z%L;-5`#|T<(o> z(aXZGy>5%ZPl&+V#Njq>|3wLbThG_ClVbm{`s|?o-I4utWtZd*@^q!WZloJ;(*WD4 zg!uDelNHCqnGiR4?5pdFSKL>vDlw?5+`O_x^31&YsB0FvHIdd`-*p2+(;LIM7rLoJ z1)0q*axQq`>EmpA-yCr9pyrpepU)0IdvCsPN~G|!FTYvj={K7S@W=U~n{t~1_}IK} zj-_U@a!Um_$MB=n{@KUb7iaHh$2LdFMdbgpk5g~PvoAMiK65A&dGqV$+_R$3-p@{a z=2GWhzVe6D20pFl(@*CuKP{9R@MBYtZ&XHpB2*YFlF91zs$OR`aqAc}VlPHID&nXq zn_BBhbOlXz2?DA;=a;%E{z3!fVp5x@H9G%vp%QY`(~~ZIeEQ`xpU#i}=76E3v-71j zI2T|x(1^8Yhn#oz>oW!ZXn$-<-^k#I6A|>f@p$Bl5FZhhDtE)GS>cLYBmb^)msvG6 zUhQivASZ$DHD+gGgw}}TUR2Z58?G5*r=O+%F94QbDCi8a7nOMdfM*YBt|A2xLBnYL zrCjg~h8HTZsl(~Wf3}nh#b5Z?+>I!fo3Az}oEImTt>iW$4fJe=a;^pIhW;G)N!Rf zzPjG|Xnj+=t#RjSKe*DrzTN2$k|@Th?fe%W4Vk82V5egxhk4u>QS3%8O)Ov`dU z&5usXC&%@j6Z?mmyMyu+omT916Yo~yUrYS0Bg6rF+BtgX~8Bs-)W_q!cSq4im`WZ z-2V7@3PMwl^ERBd<<;Kh(oz9Q21)~^nN$OCPMaX5HP_)?P~z2>4-(pq5h4t1(<4og zD+y*+t&3`z7qeEW3#r%!oK2UVwYfQ*I`Phn=pD)9#OoCO73uc~TiE z1ystl&2qOQzm~Y#HhN3Kwrnce)b`E1NRxa&NRx0lu$A(nZZ&f5x>i#x8P;YtS0_px z*X9UYIIR|*!Vq4lThrtQ4`II2U(Y=@<5{}9>1)BYxe-A44A#>gkW!i+KSe}fA>}T& z;rR%BuIy+3+!QChqZ`}4(%5xrS8HPF37-dW?pDA*y{uGK#<`R(5d^WVm1{A;zD1z8 zsch30nnF1YL8B1Vqmq?gx$c@< zPbXUMk?JiSkGosl!A`ul6>kj~FjBGSn)6bXY7c4=Xc4%8z+1xMmJgO!s7eocsk0x=#> zOM>=hg%<90wtmPZWuk1@iUcY}Du|m~7<5Aq|H_{DPF#SQk5CW7*VYwWsV*!LQf8H1 zRH}fpNyy89i4v-jXeP=kWHu%glX2@BQIUkOqFPLrGMk1TiKzS}42Yb9rbFAVMLkSZ zF0+;pdUioND>TjxHETLO^J--|axP{Up*GTf?)p=XBP87L=F$ zOO~n%IRGw@hGd}YCK;%NA_J#p)oKdbbc#fdl{GvfbVMNa5sz0zSeXd2z*s049_2bK z)~j;0ESB>ev&*BoxxaGzm6&QtUe~O!Cthak;VQhFif`&73AR#yL@w9V??lq!%|k{D zgM)3usxS*mE(w-KpkQjn53>9>&8A`-7AedkAh-rYHw}}R3JOX*ag=^;J(G2xXtW~z zFhyy|f4VANUNaiyn(SEwd@+I}h7K#JCNf*SG*#g;j0k*X0vri8bo{lD|K1*gV6eUF zb`s+}7Wyq?3S~PevPRW@8F|Uv6ZAWt?3F- zg;zCu47k*V9zPv^n^+;ZI{OS-;;Ntr?}dIpA|i}T4&V7W`RJ=Nx+U!U+nr$M;LcKw|7M3$smmUG{FB?@oXld3s)0bC6f;Zz9l{%m01=h;|^k* z1m%+bkjy8IAd;fxA$JtF9-)s>Ix~6`KjsM!QsPrA4g?pAhZI8D^OEsgU@lUDk}F`G z1pbgwLB_f;j0nRg<6j~@H~X;*1-#rxAcTp*xTr6)5cUR}QGM2DO|{8NCY^!%n5x#A zc{7`<`H`7U%W1t_jF)=g55~LOohv)>?pCln@Or6C>d&<2<*L&j)gsU$a1nvGfx~U4 zbY``hHF}yi?hA+Yy(4pHVUEgXiC`TaMrzVYhw+s`_};+(6|uWgugi+aE>-4A<_{}# zC=85@L--gTXZyYyuy=&etW93H|73nHs)s1fX4T^qt&WH7&b5wbH``m89h;gsBzYAv zOONKobXhG5W?so4zLt#bEUV2tE>o>W`iB_Mk9@c!Pn@>~9krJRlnEPyYdxKCrC=u4 zO2L5EMVY}h+o~SBBDE#c^CEXgxx)!}g)XlNj*9*`;zGkD z8R@j=4X`Ux^_D4nWlqzkS^bdMD+_y&*^-IeNswdMp=Ar*js0#s>IB!j!Oad;-cZa9 z@@V0zBCigQ^ZWbN!)ZM$(Rm3WWqsBn+l1aUCsJSv6l*heQfon`utuXEcZ`yY!u` ze-InuYWOgt4$GKzvOHd$oUEa4$II$qS?_0-=aXDLs?=P$ z5azTjxZ-S)-$}w{9NkZ%oix5R4BwxGw=~2{PS zjbwe0WeQ0MOdWUWAc*}=CmIdHt&tx4;Q+I+W|UCg#apo%FO1}>%nO1eF0%4)WuB}I zJ-fn0pmuaFZ50BSL_Fczp{tXsj++bvIaUY0szsdpbPR54{G+}~(?A`0cDJKZr}ldy zD^!>@C_H_uAYGKR754)1y}1pc zQ$mVh6QZ}4dr)M{vIXKc1cy$Pq~R#_ucZ1an;nt)J&V9|7d!tEQ8wayIGn4Or8>y# zq%Ia!2|-_0Y`5H^MsUv2fR0eg_t*>#xMP!UPw&JU3qLkRT<#*?TuSOn1~fdh*qUbb zlV!QTlyS>+)Cj$WkFxiqGX+TIFSc6@;7E)MyThnIj0T=J^c7DY&qYLnN)hMs$wO)FDkrErQ6AbmB3l#o4>=QQCJA?QY-UTB5%ZO*Ij0pK`AA44YkUSnT zdrDj2+6%JVlJgVlUxLYvYxMV535x&^XrWeykSOo|G2F+219$h@6Rq{xx$6 z)nEZa$0l-``C|R#D1UsyoLleb<{-Bxm17lQ$qu*_v#qDC4;Au>H6Qe1zi<0}b*)k# zTa8V8N2uLK)^&yx%MJpbvQ%pVWft>;;;a4opQi2$7ght}U%Fw>-VXH#y1e1?fh};p zJ<80Tx&0z@4@-?Sj?_eH;&(?$uTUf9Z)tC@#z3(4Ye_=t*2VF>ymwUp`*HoJnLWv! zUxSa2#I78~=(p05dT*%y&IxG!$R{XQD9a%0Dq?|}MP@q9zI>Sf{$BZ2=C2X=_2Pav zcxR~J9jU7W*A3a|*N4cfj;ec8bCgM1D(r>0f(iUGj9pTX`_a3T@K;EH*z$uyLv(mK z)iGU|us5A`2F-WBv+6@=S<%bo65>@H%<*xlzkmcU=@*tUOd(m(5fIQR9%7h4Xf9S) zu3PKt#`_flSKmhzAeFP59vf^ZxKRU{v%t%ge zxD5{rhxEFDwTt&Kq=yZd@MBmz(y*{!M*}LJY3%a{p27p> z;(Xc++$KG1MkmVc`SnTp=~Omvj1a~xa~@NISOw4)G>eVM+#z94!|^cQtK6Qa_dC|*4C z6+UkulSxw49sGZ;AmMNF;8ku3#HWi{aquvIcqe;k7RS3Yy|v!jOUC2zsJkup}yt8oKB&VCkC~oMbi^>OgysVTAzerHV9F%^g{2JrX1B3dq)-^xbeY7ZstK@KQHaS6{5Bi$3rxBk`;qctu`s`KOmLq1aU{?A`-|Qk(D3-i)rTh z&1C;AeRlR02gp)!{*Zxzda_s-%T+#QX_yrcCEBz3!WXY}OXk?sUr9 z1jdC*?1mF3=+5cdcYbbN-oi2 zAJOAnfbkT&Td1}X2+H-EC;G4w{~O{LmdDd#|G0Q~RD6C?eV*BSwVPUXC`wO`Wr6mH zV8RNmFznClwc7qYG#6=J7>Fn{MuH_`QL$J$ZMTdQGItiTmne>B`IBjR|G510$o$`P z`v_erm{TUJI^k$VJ5Tm1^S=hp23VY%KK!pz%UZ3m>toO~wsGfxzKIWw6h`b1&*-{z zVBjJA6%QmaTIG(x%2Q;Aa(&=CeUW=fdhhh><^~bXBRqRm%w`3*;bWZBkE=T?`+07^ zD&6Camn#LxX`I)*2DL&rox5(OcDPTC?RyT{RV;R7e#&m@j3RU6HrJ9G$U_QfzJN(p z7(v#M_=Pk3Syis;a9QJu`D9&xUf6$Tq+u+{&y149h83FIVC)5dtGtd=d)nVdHixgq zW^SNlX$vhO`f<6OL*VGd>i%){=V|>fOZx?-3BPV&V1$4O)qs!kt*%vUu9WUQ?LG)3 ziwRbAfk^7TPJoi~jHzx$*|EZ!3el6=MR4DX=^|oIg33Psb~7SYO-gT;1*Oj0b~mA3J7Wr)LA@ z+X`vFYY}J>cm)J5ox?GUu_lQXxz15FA_T+W$(>s?WHK0C7AnZ+rMvQHcLda*_R__6(j&J zR%Y2EKRKaCiU%hpfo48ks=ws=t`!!4GeIKjkqZk=H`z}2bN3io%DN;g(AU-tYIDPx ztq@&symDTO{KoSg^002yuiQY@Xxk^o-jN}&F-wPQ06Ns3G#~y>6fV=qc7k{M{%+sz zq-dA{LoCfu?yeX44Cfo-7f#CuMn+>fN}Tg`cB~< z&Z!L6i9TPO!Z8Hnj^tE`0?q(;7DBO`=sK2NGUQ}Z~xy7*BSxfNVv25{T;(!w}-P@egR~(OjEvo7+&YYKJFF8B7UCjQ+IdQUO81 z=P{7iA9!}5)rlj~pHhSpT=-Q~WZD}U^`3QyUO0)uDAL$-Ku{uFN-$|fy1hn}SRNb~ z`%}hkeQ$2>uGJUJY_Rr@w}TGa(}}Y@86n(yN2U83G2@CcY!9(1bQ+@tySC9F><)ss zC)-oXZ|oDKyTpt{_MT$Ty2iR-VIM3p)MLtE;|0ZYxbwsLWvg|D`;XnroAMZ9Jh6nF zRprTYadLPxJv^uni@|cT8*N`1T-_V(4ZGbA!qLsP_RUnIEwx497l*)$iO^32g{++t zqf6*aR%_Tyflat%Rus7fdrlFfMLS9OUN^Ws@I{5%!9Ml{61ej^YcGVV^;H-Lozk*e zldCVYl549;CF!T3pT=GvXTy+1j(HFZm3lVIj}EiLlYDlxKf=&Ngq#?s}eRQFK@%5qh%PSk!` zBjlj&D`ST>Yk)4ap9Ok6@cv8c-@qt4HV>BWSd5&FniovyNady-@8KW_2iW;2;^eV@ zF#<#>CAL*!*S8z{pCz_Lh!)7wzo`8$*FG)UuKjW94HLcB#k|a+FqvVoj_>fSp04e* zHiu-^$nEh`eKPg)z@NDQTAms`K{TBiT82W|7hr6>mdSx_SCawbvUmMGcz~UV(w0I?5OJd)~1mz zqhMQOKIn-RS~n5HErc9Fb=8&cxq7ATlM-7V#DF4e!0DL z!K(f|b6>Hwuzpc^<(lNIuIIuYyAF&yp>+@ngL1sYn13-Z9v)Yp>{p+kn7b?Ybt#O< zCHVKqB`~PJ9;jpoRH?vT~DS&|)#w*VhwuD^c4- zd$Des8*aldSK`nk^5)w3h?A@8b7j|_cc{G^&b!@3(<=FsC5aXc{A_@UXjl-XO32uZ zTQJ(}Iu@<>3AY2zCuxv4)3g5V+P_`ZS&om3y_=~=$O4(KP73x^xLh1NyC17Qbd0Tw zQWM4Az=SxT!1A-=aGHO$pA+X2@jDHtP~=VJe4arAJ1@o+yr>~Ce;yS-bxTYEc`?X=qsi70|p{ztDtTVRX8 zFBXBz=5PSd4iQ-tyH9b#gu_9CNRZtK8!%8IH9Cs0IKVRZddI_5c`KB_yzE3@{9yhP zGv^pvIy5ZCB}RAI(L!NFKB5y6ZK0B2MZg3_78NwE>|)EKS^nTj_HaLcG_N15-Q8M$ z;k>(uN0N|Ld^aVgiGIJMcOw@mi{|k-s;+5aa!c%=aw9%;rTI*$hk-xv!(VB?R9?^X z5g1?(#H8AAII((x5h4(T!wunRtOtl*>MAZRO4wx(JoLh4=#8|S#Abp~QAeXJg(p2& zS)HrT(9x_BdYNff-V3~i_8%b1@f0MWpLqRLU5(tXvrA0i{i>(T1LY3$I?LcD_&Dh4 zAh@5$XgmyV3&;owmZwD3Mf3u^L&z~ZiZO(9$A~d}G{wa8RQGECsPsMv16p(?@vrp# zUL>mOWm4?;nC!Tdz#Xj(21EC2dz|V2oJRM&`Z2`CdYJNDk3C}YAq|qLd-Hty{ydYD zHHrUnT%#z$Fn&d_zgR6Z^%pH!I%S4|9 zc$C`d${uFrBV1C}^9D^rsbV}uiTH63J-ess zk-=iYD+n_ln-q?>a`)hTa}u%9QTD`Fj}elM#G4!$d!Uf3)o?gAI?O7E^WwAp;y(_} z*M)Zw1ha%3!H9LkYh!Fe0)*UMkEDRc$L%_=W{Do^s*AjKX-G@s^750l9@hG1;?G9` zLbXogpt?~FQ)@a`oWU_~O@m&~?$q{!Nata2bQXW_ikJtm{S>wb#Bb(-f`EWFb5$Ri ze1DzoAJ3=v56s=;;c7gIC*#T9cz16w9{AnlSx>JwQ=7Kb7J*+B0+-Ftjfy4#t zA~6CZPOog=>4YYT7TafhKRaPCjJ!#VPDBqC@mCw^(RX!JL*v+(I*HSXAr6*DsVo_% z{#lLTk*qXwd1O+R*x}9b8k<&k534)u{*Aut1@{tvkYLvzUBko}k4uDbu^xtSIOH%@ z=IfbqGatu``T#9PVZS!&3%Friswi3n!Cv6K8|xer6G*;RnMXH501Ga_#)lUoFJw^{hYVLPy~A~^3tSjnP`Poa)&b^Y4Toz&Qf=}z;SO@_ zRL8=3@rKoFxK0wdZq(@o<3X^CF=C|fTqSnewyk%aIq~hjXFF`dD|MXLW>KB^_3w23 zUKq@}-n!OExbV`&d+oU%SLlstg#6z4;S?p7(`~^!ng`LY52|{yfKV+$MLA=4}CYV zUbj>zxpP7xdJawB0SfWT?c?`gVD_S{+@SnTsGsYOANSAAtp%wOIwwV6xc=!#IIS7O z3~NnfyJ!UQT)_nJ(Q*0JL3uZGcdUQc7t!WkClpi%;a-RBoZd+^1_lAdH7=LFn2ybS zq_E~O)MsrzN39=*Dk)VM;bVO}(u+hFBwiP*iKa&9O=YxgJ|iSVgUyStw^DCA3MRd1 z)bS@MZC%}WK6zb;STqRA(9}_Fx}|-HgT-3StUBuZk5*NfTee9=;eqzyzb@)sv9GNO z+4J?x{BFwRKrBjzu75KOcRJD4ezZGccPbn%wI;ypJC`9e@w{EF-;c4>^bV~?nY)KK zg9vC56K^IgnKJoI&Cx4FQP8ETFs5JNgDbvuxEiH153BKkf&1 zl_>A)rOOYi`F=J#MxpU|+Ivv$gz0E9=RK;;rcnQ8*#7z);bG{daf!KDCz#jQf&+`Ua^F#=DZlV00z1FM{&4hsr?EDlO z+z`0UsSr43AKCAsj9IU1+$m6b{3+LeQvNy)22u3ee*EE1a$}P0^nwZU5(x|JuI2XXE3wr-FN^1AOJ~3K~%tBV&FyQ3ACcdh|vwg z2}E$waX;>$n(2Etdg@AKBOEKEykFTUtCO|uhqmKmVI#)c4{)d`QRBHSl(*;~2|v-u zAp|VeGb+S`SFG$R-0raFqu6kJJH98YlaA?Nu$23ju+ytakQ{UIaW_B zgrT0qAMB*J#_{z&9?Jgp6k!8?xE@}{c|?!6qqwmM%IilNHrCQ3_%@D%0Kpq(=Rp!A z7`-XmTUW>(ff`FbpumD?idu-x1F_|h;C7=?H`?nFkj1;2sGUGfJVNnd{6~ljXja9d zP9U*6h59q5h3M*%c)OS2cJuK#xjBipy1|IJbedBl2G|1GYw5hTjdR<9Usm;B5Ynu? zWb(j z1j(blvav)gh(+$C!?=pC4I=Cl`o3dh5W@NJPUw`^l5*N5UA&u>dTs+e#M5qkbr6u> zqK6Klabezw4uKG>jHD9{$6ZW-I|DN=+>II&Kw^tTcEAQS?DzC;tS|?}Y33aAGw6Su zcagXfF><-6=0~fe{pJ2_b`%_ER~DTs*ZSj~Udk?LucI)i#kqv3`1!%` zOoFD5MW0Yw%Ti}mkV{+@sK5~dGi#Tn?Um%QK-r1eju^>N8W71U)>M$<;c$!E>)Bw7 zsIB98Z#TWU72oKFdx^ITgTg2a4^ZR{y3PZ0M0hxBfvzXvWznnP9Z>t2np5*CSXr|q z`m5L>iNH@9EY&}KTwL5Yb!g^zXgyKd%(67ct9l;>wy?)597P!kVhZ;DcJj_%dKFn$ zFW61IJ#sSoHo|HzQB^FgnpCHB#L)^dByPT#I$^z3l?W=RHK9|x5(bvx)k#>Sy@{{! z49+#d;jo9(MmQXCT!N$<4!a>#2@}k#*u{GW*(AIH{h63St*UBG=v_4v7tuha(8grD zAKx6r9}ME#L*ZChJK?JU->z}21_3-%R&+=0vsXcPX7v5an^)0cZHKs$G$S8x-yZw} zi>M{>@Q~y*t`@{+&( z0d|>+sKpyyW<%sI!hO~SZ}$B=t|Kwd>urWx*>2xG*4>2dK3G!z;xmw6ps!~3E?$_8Bs-ChJG#lOC* z2%ZP$vo+hEPOK3 zmSJ#tmPCXIoHgF8IN-lQ4u@04x~xu?>n8_`$B$)GLOrgqfi%4j&+?ZqND<>ufP zY35;YZyzlJEdoCl0+-FW z6@mZ}HO*&=cUY{CmM(?Fz+lMPq$&c$*`Z&QA>sa4%0~(XjU%?jrZTJ?@|5bZlt819 zB8AtI61ngyz;T8Z0H1ZV|UHd@ee z#LjIbpu(Xylu_Q#=kpCJf`zbK5tLvPF2x?N4Fqm~X^+XgiCtqy_fmha>yP_kuNU?3 zv5iDkgkfKdIkygQRlw?oToZ>qGDaA}8>8Ve^hk`~M}->%hEZ@|q9rr7;>bs+N3er| z#zohqVlmhZJV}P(i?|MW6O{O3LEj;U7t$1L=X0XXiVa;w6hD?smS|C+qr3~cL^&p- zL*nlw{x*ff4Zzg*#DFml`$?E3UJ>h6WUmC~IwF(0npf3DD20BezZfjlD>Fykf%hvS|FCha|u1-^k?$|qd>&NwGY*IE3p+g z#a6oZZFw*PWDrQr5QTHk;}s+vt`O-a9L~6hC=a;}Ls}eerO}msbbSVU&N&D$^klq?4_mDbc!i85A z^+Z0}i^w76L}mFp3yL28HtoXqj-uw3SHFaQt((cR`JAwPDM4PSj51QuN=6hhr%9H8 ziJlY-sIhm}Os=965R=u@Z$1Yz>sZ4ti4vg@m#35E|LtNHVSj1*kRu)LsUHlU69RsCcs#!*T;s6ur1Fz10Rjjycy+E^x9)}NGzwOZ`O4x!dK@QIf^JYg=YxCim?bJ z!2`BI*zVwCiaZHV3>F7BCyQeDi)etYcY^Bzz39xtQCQ^>P6jzv=WRQO9zi-?9FvQA z{8EAEFgR!&EUbRY1Bw0LX{EeeGK~up@QFKPSGf&5t4P; zuB4I2T|`QGl)T;=VqKl2bvZ%4CyyBt(<(k}h@Tb#RMj%5j8HgIQ1JL`>+OMxKw3c7DUNm ze1a*mv6##jarYQPj`w*C$!x(ua>980FvZdkuLg0}RHJp=#~>`K5sJIW8tf&?W)P8w zYE_f`w1vuWA;)IC^J^oAb7d)YT?DLTdVq^F>=DAT_0r*L+@$pYA`Zf-w;)=0+~Nll zGvXOwCHYjE7kM*}!!p9-+s$A+KMXN3HdMY05AC3s=7_~p7RozP=6*qZ?2u^86ri!k z-jIQy-4l{|A3s;dr;_;fVxs3L^C%lg9=wYK80AmyW)k|E-xSg@R6^va-j2GtcWT6s zeNYsS(+WNFay~nK@#gs3lhc#Ec{r&L4u`i62Gil7pDIQm`|%&UY`dQ!Un))S%x^lBH2MO7m_&v;Z{&p>yR~jg%D2rX^%jlMDHbXy=$AK zO>RnM4RM~TF53Z#yO^~p3$lpc24VxZRLNzOl>jvmm7)%l))&!ii98Zaqu@s3qj+Uq z!88u0>uLx&!C#HV19b}GAq6+VT}}kW0S@n>jt~jfl=DNlZw?`^C6*$|2|0+m46q`g zENP4w*E>$AfCP~;kbo)MlL8v63#E%C%q)_?dM*B{2vrarw5;-<;J!3kw|6Vgd70-$ zt6}giEobrQPZAId*MrRyV`E?UCq-dKP!Fp^x^CMvZ->9A2uC*n+%L%*dGw&UC4qBtmnN7FQt~Tj1(-&p28O;O zb!H3UL{{oAcNd0Z_ZT8|Bvecpk2%bZ7@3vz0uCodxfm@2cvFWClI{7b}!I<@oqZK7Wc(*mm@S5OqAm(vmJAvDflZ3Sro?&8H^Td({ z4C{*_8wdC}ySo*G(|eUQFXK9`%Ay~hks(Zxo+S`>ah6;bE&$>I=E09c?x1LZ*ZSUo-@yfC%^gQldz?(*@~5i;bnlK~e(s^` zxF1B85Xnj;1!xrKTIgh|&>bX_Ibm15mVh0IRhaM3yLD6K)xzB^bq1ulB!@$#hLtD5YpNxwKZ#yDZ}3ea*6SaT$0*lQ{X4u*3Q@Ny zsUWawl5Ly};^p7Nz3G2O9&{`br53pe_+D*-c)UHF>{@EOgb*Y?u@A-J#GAX?>(}Pp z+iQ05jyk()>oD+^o}v2Wl5lUv5R<@5U}GSs9PB} zkTAw9`B&%`h#}rON{$ERW*4EykWCQ{?v#4GQ3Yhu<~{{s5@wM~xE~-hhY?GRb@c!} z1n({xFZp=&N+ZHFyYT^sV>Hkb_7g6(S{ow`Z=+t7r0Y@C-u0TJrg~WsShB|d>7Wh6 z)#~gBp?icVoqN(is1DZ>B^b>^jJIh&z=mw-v5_R|oBEoDzT5VzHT;Bs0fvhS=Yi`< zQJu{5*&Ff`o~5t%$N9~@!QDNA`u3-I2GJp4$@$*_^b=OsiZTTLRuR}WhXYqux$nGK zJKdTiqQOEciIH1P`S`~=R03)cq#ckgVmV>66&pdS z&|(MX2-8Lg=PIRQD7P#6I~~dg;&AR;G@)dTVy7Q0{B}iN142-c?sicW>vDAz90lGt zB&l>uJ8gI5SR;kS6s|^A+^1NCAsmhH8K#~T$qnFO4YZOMBvBFm9U1DX1llKTy7%eb4*q1)1Q|MQ%--Ld7UFQEb_aCFnWO@| zYVUPJD$NxQ9Ev%M@gE#c0P(qM)q$u}v`W%D zkafhas9Dkfq9Z0w^MyT}yV34{XneVstGlm2Le3CYQgB3ZbTHyG388{Qw3XeFlRL`# zl-GAjj6(qjpCsflY#jH(^NEII5$7xSm}Fxehm#`GJ(}H<0%EEINb*aMyizkKX&}N# zGBn4{E|{G9f%BzfCygBOe1tP62AzKd7_Ej4atDI$2%JRKtBG&6XAB{-C|>`F9Xur9=P@HEG~hfX~+BqyD7bA5U(K_6`SwaU9Fg?`KnMtB4`+H;2HEIUEzZmY?%@ zX|hVk_{}GYlAA#_Gs<5NlP0W;N|rPx+mGu5W@!FtYVFoM?#_(u)(>gy*se#h=|8-~FjdIf0AJ+EhK& zQz!Go$Iu>Phdut0)0p!erN|}^R|&-hlR7LV=U3o8XuJm`XMnF0g^tv0q{0k>eNUEN z3?Z7t%_Y-O5kTQ6wG(rq%Tuph=yo@?)Gw~1pZOC12jY<6DNCVbl(^KkOJ{$ww~l#q z@LqsQ;Tn-0y9Fg6;sJLwSJ8zw)Yv_PJXDwkxB%2JI>5Pn6W0hj6F3ABe4X*i6|lVw z*&ePA5|1Ge=RK-ULi}tV!}D2By|+Qajdkn~a03;HuAs|hQD4bisSfuCVeQ>`@2t*T zmq|p-Ow|oYbg6oBnizvxyfF@XbrXzN^}W)2S$R*Z=1JN9kG%aKUDs;j-~f0)hra>F zhdHbedS}T$7VU2sU^sGN38#IXm?Z)4kjBx_kDLI1qMmjS5DMzzv{|2&%hTn2b##`# zDknvMFd6mtC+Yqq91id*HV*d_b%EX15D)_Av)RtjaPuTHufz=4C^KKnw1*i&i!-K1 zim@aYMM-q<+h-nsFP(a)4c)?_B}``ME}<7wAX6c>%i4%jGIKC862}$Xf~*-`voc?~ zrBfz&t~giAckbWa!kz4v&u&>lyrPAH%IjZ86gEHk^WwO2LrUDNL@X06qJnq%qFe2_ z|E4yMgE%do78O41ZqR*C5o!fZ-CUqQIr#SCw(bP?>GeOlM!7{xAZgj)+OEzQrM$g# zbGl9turwcCn4>!^$BrtA&G5R6z}w5Od8Z1BnK>YlU*ds5Xg9_Nxo!32K=)o@ zQ9u5mWjZuwL{jyqzE_}^aGhYA7~GL++rxvHVa5RogT~Pp=v5sTK{4oK@{XiUBGNQU zri7Zn3`@=vqZY3esEF>*h5`m6d7B6=chc@g8ywKSt`FSDYP(*v?)uYIrB8IYCkXVY zEtLTbUfpyfob_KXzSZN>J%nzEX!vwhq?tT`u1OlsuHEmcJYTA#x|1Yqb7de*< z{GOrs-a*Lu?kReU;N_lorFN+6ED&}(9D&?b$W@vl%aD*i4lfcYMCe1)8wd1eFd7$k z=hd5K{hH)p%jW5-{c2tRiTp0iZ<0IX>|Bd?6PP4#(XB^Soi$B@a5ry?K~#*wvL9A` zIY2Plf$@W0=ut@qvA6T$mZU`6xR}knX)hi!;(m!~xyGao@Ky(T&Tc~93JoC)B{3^%;N-V8bhGjlE z`9vnVS?n*(3EaZX*m8PPoB@1JSr9TWBvht~MLp52b!xC^hty~8?PkVt1&ESeUlO0X zFFGEe%OPNVqE&Y@VO_c(Z@ZU@xH#MijxTCUPPI0x@5Kq-o9;AshvuA5bt}8pJC?xn z#mSx6Eo8KYQ&i~1aq<4V>KEUD$cY;!PaagiC zB9J2xEH;Dvezd=r>`jRLg7>aJMe2qHA>3CbioFGYK!h~N}` zG*n_=8>JqXeP31I`Hp(=$!4gyD^&uLnYisI<1BnI2p=pYQ@1g z(O^S4Ot`x^OHw#oUp^?J?>v#dpSv7vlHti@>(ap=0_A;UvKvGWws)$1=^mfzt*fXj zjCZ^=w$IBJgzmlGxOeWS-T{<8JU_Gh-QD5pXIKBy)2Am%hr0v6-?@qV?$XWQDaKuX zUJ$>(d%0!^-BVwxp4}V$p+h5s?-9E~+H_i&^!ok{EL@X#e@ZI&^59Ib>7p9X>;B3Q zSM|TITcZ3pt4!x5t{6)s{;i%CRko^%vuZZ1PR7Y>KUp2d>%^agAzmHGYcGT8O8>M! z7>5{yE|%@tN%iIkH;wfyT;%)f(cW~F5yS7s&^zqQ(F#ips(LXae%e}EX@U|@79>_6yjC@esGRc=DE|^ zR6El-w?pSgCw}O77jxO>=DO?o(S3B@5*Uc9(Rm}!&*HZ(ao=?oPMf7p)l)rPO)s6T zw|vJBRsZ7k&Kd9h(_IJ}&@)Ld+?~22x?8ymT_0`CfLtlg)6-R5gq9le>K2=>QE!X8 zbeFduUU?@(xV)1BA~uW3#w43V)R83GO2-F1Ld4xp{9B&4hmD}L&$@z2I!?@`-R?l$ z>qs3j_=90DoMu>|DP{x=Nebx>Bwgze?tW3dYYtV3o5B7yS~k7Gsg$5dW^FGW4CBLT z_}Mfdg)2cQ;E@sn!9KgY2@nZ-99SA@5pFYRSFqK#ULr5fiB;q9Q50b23q`f@p zg>kPSi*vXP2_iFGH7{57Nv>egh~!ETLa9t*R-zq~o+PNg>6iESs$XElNxGL&R1(wC zwWX88(XQPbv>eCt#i~3y$zQ+DUoY3Ey=Ajk4X6E4pEyVRnNLPOWo zbp&?J;g|$rI@mB{9;^@*Q5e@U2xcafrV95w>{%k>x1Sa@aUxRmYS_iNN%{M#uH_DU zBAJMng-D|hFvnqoT$ zOj&?2BO|fWfMpDmb?%P9N}(pwY?xs5b`TO0z9ncJ1#y#ZDOblx9O6*o5_N^H>np+< z&c7OQR3C+}emhR(#Hnv9B|flO2e7v+C3XfDkqwTwBtT<)QeEyZKpZGK8+dDit>`{_@yWY~ zh7p=Mr~p>{m?3nfynEKi{;jU%;4=MG`R*^^RnLnh316i(c5KNtQe_BRu9D7KW~_zZ zF1VyC9kkpF0Nvvv)ucqHPTkBWx{foofgmJ@OQ4Lk9QScwshh7H8iC4)0M3rfF;?+n zrY4=M4Hj-PDRa*t0}?W~xr4)21#!?jog**^s)cS$bYHojoaUNCQCK}Hos10lTtDi2 zZpE+#2-jh{r~dP)4C7?QNCX@_)2SY^SsWL#Dp}Q|b$xHGWF!PO!qb@~cz7if;#fI{ z^Ol4Yzb~6V$IaI$Er)@Bn4pzskHVnB$B5WNJ3JT#gkq`}>-=Pvzd2bQ`OEoq8IQfe zIP3L^MLkwZ8`ZPpnmeqj6>SLoZ6ok~MCenryqo)&{t0PWMMQQA(vqc*(EnP36!MY) z4CP54y<9b!qU}Svlz<0AD9x(xt0L1qxs2AuYFUtV=gqQtwQQdi{@M?Z^Lkb^`-6BK zux)I{m9sR*y`R7h3ctZT30slsPTD)t9V$qO{3}Vb$O3TX;7nEy&E=ZTSwRRkyU@BM zF9)sYfl`DK-yf$@A{vN%cI(q-2|vGC*OK$bbico^b)3PAGjw!+bnoB)Q}U?~qi~MZ zxaG4Z9xh_gRTmzmC6mEhIm)I6B1){CfFRPuk2h2S1h7kU@fi?yc5zZ1jg#)WJE(hC z{jRZ>KJ1#qPdF7fl7Stm%11=ll8~<@I73d_1H$QH@14fIG{o|3KWHbuBDk{f(hF2a zT?Um=&#>|;7A!q0KVz7>qZ$%=IYjg|?_7Og0IHR%0*Sji-MFN+UH$k4ZANcjlvXFZ0BpGTlAVHaBC%db z!j+b+@FY&tGZ^qBvY{d)SBHligu<69xdE?G%)CHYBu-zvEk_Toc%8^G4+S7Fz+A*; z*B%%*FO4mRhz6UuoJJE@$ZnGDjSS?l5A%t`T*qZSEt=1WOsQBZKrD3uxmXs9lISzX zr{$Y@d$VkRUzXog{_{of%{rc}o8G!jd)^@NCOnCk>h`vlpXi;M!~~l!^Cf}dj^~TB z;j+kzbQBGS{WKdyF?my9nH`6trU0?a`hFrl{Rmb5=A*p%D-%5G#OeOTmbMBW~XehvN<$Zff7W(XVcjeIS8CdpMp-UtaAIX0(K2S;Ij#aLq*V(sgCQyS13 z!bEyBmQPQ!dC%qAx>SblbG60#3)-aHyXBpw5MLMg%>WJ;dTi^vP;(~|+6H?p{%EP9 zkSOres3nDK4C&J1(q$WO=33gtoums!iH_F;>o6U_I_tWUw_SZ)!JFE+Pb3|JPF0=7 zq|@iBx_k%cfNc;MOfMqOFkYM)lQ;yT3GpyuQ>1a&>xIPn&yX##^wUsBEfxF>A82SX zv1Z_K!0h;gZY+s~x~M{S>!SK0ykQVAnq*p~k)||mD>l!PLN*Pz-KELpLe~W-Nkc*1#(5jd%t2e9W^_sZE{=6Y2F#Dc_L?$=K zgmk6Nl6AQePD$b}_$Kp0l}tF5Ber9b!nqEnGQ5yliNcDvCE7&!b%HotUZ}blNa;Gl z1CkX3NmbpfU8HDchE6@VsW(Nj<0?=>la`*xkR6wI0&{i}TlwgQAw8lHJZuBb!_ZtH z(aj;$MJzBU?`fTr>x|IS#KYF7R}QLLivlKhE?K1i*oZ>Caa3{c4&7#r z@h5VIbPYdQyo8BXJn#nnc8JRZW#FyKC>6_v4%X{cIY+AXdi5gmUU;O-BVaf+DId4t zw`H?eG#Mcg3vUt-02yJe!|+N4KiwZZXY8$)c|M;lXD92~Y`SRr)o`2*M*XyhZKoKs z6MTLL#ZOmBE6EV}dq7~<9PVPxhQp;z(5saDFY#|g-akj)x8hSoaQDAjMNekstzJ2U z7`4Q7B_Dh+!P)-$E`m)|t5tb=n!k9pd~uXNoz>qg+dr57*I|5wyZbV{IST&UAQ<)A zyFIuTR>wiw3!NxV5g%BO6@Z_`LqXmn1$l22_kn5it$Byr`m=a6gn`Fi4=Dwdk4qtN z?8ST7_Jsrq9t0s5^2*3DiW$4pFHyq9DspWLzHj1_I6xPj-+|xfZ?D|z@`v2-`)+yZ zI-Le}(elaqQj@x~KCatA*%04q$|CCg`?$445{|$ThKn7!0-GV3a2HD&0Rh2Up9uA2 z>vB7Cq;V|pSn<0RO9kpC`hEFu3!jDr)CIalpf0jR_#__Ki#_s#EeMqAH?IVvcaxFf zq&bM+ph`XHj;bxG8tIRe0*kRj5{}9%=(dnJ#0an6Dk1r-dr?>?GZC!aXrxA>WV|2} zuAvuLQZIO$oNwFiI| zK^1owI^L(#w(bNdx=0$s2c&9k=V2R+G z*#S-Cz61DW@UMHXUPdoV9@bTUwmv;69v@e~J86Hv@?SLZJnHp^^@Bq1ss`F^*kBr#c%7sTG%kgo(_~0#4rFx& zhjR%}iL=N9$c;LcsA{au1{@A#IIXE%gk&gA%iWQ_lJp&O!$_f9I#&S3A(tal=Tf1r zPFHo+gFNy|p+>HVP_g<5kK?$tO3^{TF~}7rdQl4G1&NGlokkb^tM<@qlm+uf46Pp3 z1{oGsqP~&u*|L*?!5r0C<(WI8^bumB7XD8*^JYKNN z8xFH!pQye){HT33# z`q7PaC95LED^$KRN2UHDN!&AEp>dI7aZTAgFY3>*A0nM-L~=>5Ml2zM;>3q^e_0iI zn=9q065!^j)u;@WLPSC^WJ$l&Oua-ZF8*+DwDdv?5acSUWl_|tBp|%-nlM-LBo#Vn z{-Ea!m?*B43X>0Y3D#=F=|wvR;ecx?tQ~S0=b#9dut|&plniuTR^p>gFjr9isOc%( zKM~y|a$|g>A}09~5#xG93RCKV5S5~0Ob~to_ndOQ@|SCGk$bCDfqdOilD>15my2~h z%bVl8CAaKJg{aIcxSZ;B2QB^BLzl>|5NFAT^q%YD_@wy#^X7j)_D<@!CZov2`^S<@ z@!pM*zc)Zri91pgqse56UssD&MU_^Vd7nMNO6Q;rYvzo zvK&P5fbe?}(PYSHHNYy)^*#*CHB9q`Ui@QV5rSw{(KqXLuw47|+@B{Qv6`78HLEn$ zGOrdz10kE`9ZM#I2nkOii)zH}1B^y6 z64hZE*^+TOU#-n{1<9=&;N7*`{-i`++T&JDkr(-VzCJst=c{Brz!STl5KJXSqKbjB zo0+#Q#;!9241o_sVCNj}Vq#%(Lvll^>GZVt+!*9^(pz4D%5fwCwR`aJNzYTKtx z^t_5EWl;A+91K||HLJ)d36mi9)mZM-OVTkNt)3m{Pfn^QC-qme=1)ui_e^)3Omv4V z{HIZPI1aOsq+~L4E4)yPM@d$kn|||KCc+BUOmMsoUeAN)vmoh_k-V9bLf%>A#5Kt= z3lu8lJfEMHFHWkLv-)HSrEwC5LFi55b`r;f2vZ@169Z=tr`6O=*xf|P&utEOhs}js z-r_G53*CXGqA^64a)wXIyzq{5A_-6PXwUIxP$qpmnIFqVu{clOdTF+c? zwnky>kwu15V<}O+$BN$(8j9}_=-#;6s3lGm#li%Ilu;25%A}1VoLf$CtfHA!O}xTK znREzEdsZD~`ONcYb$!|d#~6`O8taz=2||=aE034;(Xu(MeDv@&`HX50W6N9Og&vw& zhRcN7e3Zcfc{j_mv;6gG@p4|h$lK?o z|Elp$oF@v+bgQ>BM!Y9SQA1(ctXTkodEKJu`L@HQ}BPGN_$_kNR7Wr9za$3ANDc;Q3abdFN z1|gP`dpw3Xg~k~^Fxr5TK19;u4JN_`ovEWjV#y_n*CO_2{pMAl>kzz!BuwjJ+KNQF z6vA%+d0D+$wXbsjb?(0@f)i)`dM^nd_WcKaf3Me)crRq5m>0wdSrQX&RzF=dvpQ&S z#2L0})(m@5&cnr?5CdRY(8=%;nJ+5W1+S_hSs&(uh*9DC_^XoMm+`7>$kGhENWbiP zC3YqPO&Ow5(YC z5O%#{9%cJkvSL8e8nP=t%cDOp<8~Ij>V@PKlFd*zRgfQ%)f9OLB7kLiIxAiu=g-gT zr|b4ldH5$dADhD@{(KbwYCrn&X8h$O+Ut9G13ucxlQ+cc@=n&#ej1?EBMNJb-3^=2vwU@$pPlBfPpYqus^@3T85%CUE!^Va?IqDj zM)z*#crb{Xafm$ZP430Yr&)V&%A@?q^T`(fvPhm* zVOaVI2xIKg^0H(_UzYP_c}&9lMRQUG^Io(Xqx|uwqp-lVF7fusaRhXk|gTZ;#jYH1#Jzoq0`&f^J=&offC3C01o8Id=}v5Lfi;8;5C0MyY;OlWz@nf;O!m z#zh1N7%WIM<@`U%8zp-xZymO8!uoMili=;Nj7Zfb!lV&j%ooM0lj7U6`kTD{bL~HG z;fPvq-C7U6T2w#w&?Ry@4Ix2k8Ta~4uh(QTR(w(hE6Qu>`w*Jb=BpIryX18i4DmK; zDo8i-#@W1hdQ?7NR;QSS$=+X4^y(g=HX`Dj9s2$a;=*7q2}^`)AgQK_S)HZfVdP`0 zTKLr;aBV95XAAGElORm}MbjP*cntjs`NSY9sPI>Lezsbi=BFq5%j5E!lls*PlJDX> zbJ8Bhl5skWxMv?XNjI>dOPd)(ozAvT69}4LZ!KGZJg8nL_16%%0JA@yU~fPo)<)^o zmh%SHKO)x0R$mu361@L$$J(L!p; zyX9g>CvaVN2|X&Sv}(PO)h|%Ptuca}qlpc|vX}TcNDq4h%1XUkNRdVNKYEt9I$+?E zyqJoKXop`1BQ#&Fi^Up2b+5{L@gPe8T4FfnQ^w8~BkpQ}_Lm`G2>c}kekL4_YVy#B z{Qy+8laeWUhu@AS*P zW6|0yW9O${`OP`87a`#Zni)}`X2lUHESD7$qihn^!{8{!w5!gMK^_;glj3w%zMj`l zSIvyYkrjyy38Oe4#u?Fck$;5BrhqZrrVg&8PV&Vku&B)!u6bZ0Gk+&DLTlDfqiPV> zdA!V$74$FOSaZSBv&_8=VZ& z(<)qr>E}_?PumCBFiF%00%Gq@eW^4dA>uGZy?lWR4~5(;NB;FjX^{w;j}9|j;af4l zH$paFa5%z4q**@c1rO3{S+>VMf;R8<%KQDyU&Q{i+Pk@|Zl&H0C}>c@;Z}>{&2jPa zP4V)qdAamYn`jv&gK7IE(XFRRLfQt#AjWp6(v3$Oq7b$Xp%(Ep3ibmZ0*8#2u-j5OGemk#kQMzbI?RGEdKJVbYpXvu-nZ*EJPl@n z|Ey?l_jwFaueXTI(WMrP{On|X;vT~{XU(&-<`q1hvM{q4N8U6#n8X7fL(-dK13-5> zX9-FJ3yHTtN5Xpv^3fN|IxVQ5D>JKi>^0MM1P`40)qwa$L@8R9a5z}vHe~V!)iQAZVlr*ljznc zgvOys!1ync8-86Npq=Hf&dR5&npD3}8~ZmO7=lF}sLm{bYQx@_ z%l!Cw@#@X;^=bKHiJ3w0I*OK<`X2bnL3pc){(=Q$vWQlL1a=?ydkLm6G7Hd1{ovc$iH3k7uss4l9}Y(wun~z8Ht7RNI#7t*s{DsV z`QNMR&nP4CI74N$@?RDHi$%b)f;$@~G_rKTo`(61*%|+bqQbNpBN>I3kR3^I8imz} z(ERbkLGr5`>2LOuFUHZGA#5zb=#IcqtZq!AY#)dHRC!=XF0W2#z=p|2bEtC|YmieTn5hlCRFUvAg%B zYxRLU^s6fieUV$Z6ncXTySiLl^7c<4!9N$JGdw^^C+idng95XyUgAzy`ET$VDC_xI zadcF^ImY6rK{R&S_{Z4iBbXY+`;+L4NpyEe7=h|`Scd2IaL|(b{C>ZAT(qxo zd}eGf+Q)u#7PjADX|S%xVS7+jle|pxsw}J7y1^2THFg+bc2+O7_rWd0jT;7=`+a46 zb)E4&KSloFUg%7Lsm=P$cv#<=)c^CM-H+>8+b%-|9C@~=-jq?gz)#gr+cs@#BxE>U z)))n%7Hqs3qS6eRAuJ~O%`~|)iuTfI9Qs7{>&qjZei41fDF9T*qxcSz(xy$)I?KJR zuK!T^|1mEgd;ZO$xrecA=HJiSy#$qIvse@_kBes~_3K$XU;8sJTx8*mVe;8D9!=sK zllV78T~<_Q5x8!$6JyhZ;YOgM&nRpgBvbo|uwBEF*$b@7=NLp3^?uMCdeyio6C_@% z>g8GW^`iMx;eCafB1|!iZ;j*IV%-|2-!oZg2jVT`9a#hIzeK2n>>|8s_kTtmXR zY*9ZNRiEe4qddGl4B2~-VuVmi0ovNUK0^q%sGsER^}Jr{eU{3t*^T<| z>Ff{ripPV!v}RV!T%fh9^=dK8R=ucHBx9U?@GHh3KuY!xxt*P32p9rC5P_c;hm%?( z!C5U$*`T-KjW+xx`lh6OG^<{+D(CgrW%GuJmyj2u1o|^#5 z?EBq(rj>)v5*cIlz(yM5cf8vaQ;rW&wPHf!R+Mg)byMK^HLN&Nw&;@6qIt3+T4=Kl zo8zKAbL%wAwj*;F)3iv2JRsdwPSu7XcJSO;>%;$8RuM7*)rK#*8J7~fdDn##!{H;q zsrEMVl9-U|?Lf@6S+DEo*id8XRWv6AZ8<5Q%+TOa7<#|}^6L+hf1D-{_meNC@fVYD ze-Oao=q1&R4(fi&hf*$nNY~z=mO(0_<`bP>5#8V%4gtq#(l{NY{l*(L!Hp{TMMA!b34yf=CMZ5kXW-dR%L!egdx&X(0Tr_KL0_vQrr zKvpqM?o1dh$*nzdOa{|Xp<85hkM~8I(3Yt521D=GxcR45gPi7@Mf+`4KUu@1o5%PJ zS526O`=NiYZ4bbw3KN&$sTZ~n5&n77#QnKxp7WIOu+SVVtK=W5aiUSh{|`FU3kv;% ze3g-$)qA7*(IJDky_tDWR?Rm>`^SngiChM&d`{aiBRzu}PkIb7^=pabH%vYsCl4m+ z7t`ePX?!>c2Y7GDjv?BA z^xBxIHK)rvt&vLr03ZNKL_t)V5%|@-eZg}Cbr=%XBBqpwJcj$}opE$;7~Jcj0e8_E zA@d9{+}3fK)qbxg&lPc4UJ!SV=^_iOVRVH3WfBk%647g$;s>qr`sJee zdg*%?Y@KIH)+X51joL?x_J3=d|5!KwUY4tQ z^Y3;2iY;Fe9xp?j>yc8YJU=*1l+Ec{HVH`dAA8~REJX2tFH7%E(|gnOVHSQC`8R$4 z(D}IwYk@5qR}>hgA;T+Q95sKO2jAi=kst`fExG-=l4on~M2#BrKj>i4z~LA{&K?en z(PFkZdop```?%Mh)xpq@@X;aS5Bnc_pR<2(i9e{icCjJw_kh6p6#8>$D^ENN2JZJn ziU^6V72i-4hkVj6@65`_r^TPns`**uTc`;`|NgxW$Agt2dhn2Fg zyiC8^OMktW-XEuTC4q}>^&|3ZIwg*5M6}fSAsL(|Q%s+6u0r;chrJS`9_ZN_vJHX` zV36ed&m%8STH>A%RSs?2vsJsqP>vN2|G-g7cEIT*eYBVU!(RFTUCAIg==pmXmT5wB zhsX~WdyZYn_`ZILoCHD!wPI&o5_5 znQ1e*UJx>^(0VyZfYz#Zxo+O9IAgP%!{HX#aJvQf(sGVTnEs-vEU0*qj4@hD{rf{W zil514pgF6WS7-IJb@OV~oXqP}?5@$P5UWbI)(BeC{;2osz2sL1@uR)yKH8#;&|m5h z*ERff%Z{Ar;G-f*`3m<1f~tv8qJTnrD*rmw!Y#?zw?k|EXyk=AngG!w%hM>T`}4M5 zL*qoFPl|GKMl=d6``gOJ_hRu{qPC+``bj!Y|9L+Y_k%;YNT=uVC4|FZ zs4h~yaiLq(J*@Slb%WXa8GgPq@A8ch2| zd{mqrp~_?zK`+9Q(cb$18hLNR<~EwhmW;ja*K0!Ey3sOCzPy?J=Aicv)AaLc^4TbX zDInR0Qtcn<`D7201izd+Z)=L{^Mc#E}h7d&C|F@_;icymyHEU;3`89-$;viLtH%{Ae zdSjCQV!!w0UV3jF-0Guv^`OXl(y1ns2L!HjA11>Udfp`lmd*fRfc;!r^`mv@e_qzN ziaNthpsG&UI3eQ%Pa=I_FTu%7Q`k$eKikjt_j`x?2@&#crNONL7Y(RjKT~sQ71XTkE)iOTA7kkD3GWx2*F%7BQA{^V4> z_)=h>7e?(a4s^(h&elfQm`&o~=D@o>_50X$ z9(_%2G(s(4{211Ypq^p+jo02B7Z!KA2$;JN>VV<*xR*?NznP@}u%G;9KmFx2zC9qO zAfaAFnALPDfc+K{yxMWNbvr(5CMhaYdRo$4L@(I=Dc(dRHv+=j;^uizn5_W5D|(Ioi{ z1Ftk>V=zV@BXN)m2ZCxiYx~3IkRT;Qf@KNz+gaQEza$(3cH~~%ovlR{*Zt^C9G#}o z16(k}@{ZSxka5OD{_xM%-b^dlgc}l(6@c=d(YJgjx3OyEDBJ<29w@kzjuF{ z-5n?Qr^%yHbbk;Wrm_Qb+I*ov9bLjI>Zu(SXkfafL5Pp-Bu@8x(?NW{Z5}R*8%>q2 zODt<&AR8;%-@&k?x7V;_#}JK@Y`^#DW_Eu+y)j0t7#+mKokm8>%?JXgpoIic$y)!A zVwi0bKg+7%L|LEzZPn&WZ26r|kX{ELRRve?QP-iKjga&834i z2kJ&T95D-IT+7m5kYAOhI;{`a^)HBRA=L^}BDd;LNpTQGK}-5kIKfD}hgJnq-#z4f zgWyI_DoAC2ab|Ea|MIeISo7$UX0=hCqwL*#H|gx|UWl_Tm3 zasO{llK*@wIL!RKWd9xy_d{lO9M-1Ta7!6x|0gD+WcC)-wgFzcarDXwz1Kr3U_R#gYs2lAB4wupU zznsJfevrVSKp%9J8cb;}#jyIJ=4+Zk?TvHq=F0nrk{BNq0V~L_fI(m#hi7TH=wYAc z--3{UFjqMOEdJ3$XQoe}8aFHdJ|eup-_IKC(#jGg7Z#i>cTguzg0Sd?gXf4P`Ji`q zKfAq`+#1EyYKmo;2wc|}G#(m9kP+0i^?2lxxMA!U1$kW88S+$yy-d3>Jt9JGGl=RD z9)a=0#Jh+0qTh_6DQgBM!RMNI5Il*KHN z67l6`bOb9eIBi2hw>&`ISitQh{*jQ+Q9ia(^R+04;y5{^c*m)N+QyJY*p=;ak-b2*BKd&%Bjdc%onZ}xD&@+rb?KQ@4Y zw5F{%O7TFQq=$9zC_#9PN;jVECB=w(29ILz79wdTp0^;JZ2MS^Pr3x}o`Q7c)iNQ0BaxcK25iiZKZo2e#2Sscr&>)k1) zo8cq^i4rf%JXp(}hJ57l;ktTA+9I;$AX|YlO3t9EQ<5$zm}G%Y42)=)ro&{?k8Wq- zL4pKM1a3o*I)oi#khf>AjYh$^i5|ffC+(M2Fk^yGd;9(P;Uv0ANK)7MbiO+Oo>zsPmPmtJUoaeSvU!NES;TspDxhu9J)Zn&1t`eXb!97pxrOM z--Pjc&}+u5Ug3?te?L_YEA&|spE#kBL((lNT{BsWBoYX-K{y-`zqQ4|U@t1a=wp8k z6tqwvG-0H;Eh$NMqWxif8^VNv7hy8ajjz?14Ugfpgs{K;D!SdK{}m_4+&}W5wCSjy z+#N*^Mhfq8wRf~mVcOlxuo@x|G}e+8E$Xir34{hyhA#!7brwJ3a3a=F7HrF9K|dPA zcLvd|9)cf&^(%3LDx>;nw4fqH;f_5X4w2K^zeiZSUin~E6(!;s=?}5*S%<{FisK%8 zfC!n{nDnMjswTz-GmEL2H>|Of3m>5h>{nl`Wwr>-VuQi{qO%L5523{HQ<5!Zknd?S z9!C?AX&<#dgyw>Oca?K&CL}aVA7emJhOglUJ8o{SP(W+<0#{&IFv#01^>7JHBO($W z5_C;DjtE~^4M){*(%dP6Gt3c4Wr~ge*uT{y%!CABTP(5Q<*>zlNc^`7!aRtGBeFNf zaT)&wF%n6)B-^6Ps2|T@s4#JxaMR#*k&!#V2n>P)oe*lV}LVkuh z=@eRsthBm`mnS(7NW;L+4PFc?dfI($+YsMjMcBg^itM>*e~|3;Blb3UofFmvs(txz z3@Pr7voV>kaDAOL!@~Qr4fB2y?q%sXNz(`~iauf9@CY7;B{@v1_8|dnq%qdLLZZ|$ z9vEHIq&LB@1&O;hZ1AK@lc~;REaH@t^_g!7LdmeKqO6&s9A6U(PSzkeIcVf+@ScxG z#vGJ1VWNwYA(H9{kIL(dWJO;C_2!lpbP^;@{|_1lEIf z{-k|uDR#agUKdCkf04w zI?9G+b6B z4#%~>MjWTLM+bqGm04VTA_9XnL~cirGg)f8b(;F<00E+R#Api<=hUO5zBg=!dHW^) z17I!@PlyIhe6$DuyaJwNKCrl+;|6gZ^_zZK-$3G^eBn~2i?pgXS_t94EVDRRkbVQR zHF}>IG$ea~z`eVSX@J4msT}Isa1urjvU;+wbXBN`Fw*0D7T20o3bIjC^~TQ87!I@% zw}{5cDH?&t0kYp-dSeeWM{JX1@(9mxk*8HLQ4M$vez0^kGF9-;I31s5+!8m0#=&&) zLu@ocM#zXn&EKzklV)1B4{_MvCD|8mSFe)-95}jx$l2N3$(2giad{GA-R<{*{JypgG=@{Tu1-V6?*5yT^hVR zO=`6$crPGm2tfl87&9=q3m9JQ=C0`OGg_bYGJ=m_TZY71DMxt*X}err98*-M(?c9= zB#0@?;{bOj#~sf*j3@RXJ?{`W&IG8mk1K^AlbgK+Qtva3K)+T3EcB7egcY?ol!W0J z;lDH7#LlcU`BQ~8!g%BvT7*A@I3{G&h?UHDW2?|LLO*pT<2va%@|!g5KS!|YvYWvG zsvEc6$ko13mjJN#Md$>iu?=z5C1oE50BlJZYgh520OIQPnHmUNKFszU!IkNwucTfr zo26b9r$FIN@toL9DR1ld@$Yz)*#yyn%t*cdaCr0n&HkuZ7x-O|{cMjI0V!6JY-z}X zxOwb9z7V^@5HJKj27#Y3hoc$HYw}db1d3&t&<5>+#6@Cm9B=8HThH7_%&}bM{^nXH zInB6Qdf*S8Hi5b2{nLx)NGJ6P*#5G?9XlsUCR?o<{Oo30SGp>o+pCiL#k9`kt6j8& zhY+Ky^DU*iNmMdIcG=B%K+uHtCd)AkAFGkBmS&H@wh%I@VRUd0-l_hqe@GRl*gC2C z?h=Y%O(Itqd>rX6W5SDon+{5 z^gjC8b6V_9>V9-pQtz?Uw!^eZM=dcyps$dj5vKwp$YL)$@gez)_Jb*lJ=$`g|#T1Pu*riz#@bpnVOe`tH zJ%j9{DCqLR2@3CrU{DHlH;Ot)xutr7=YneL3MI`TkpcogB8_(=jZZ08Ww@Geh@&px zj@Lf&-Qz%xK5fu?EfUo!{8Pr3Zd`D;?pE%NgUU5TKit#g+Tw z70M$*cM&%s!uwV=|B&_IT`*3?Kr@eE|Hh-#9v|i%X zisbTp$JpC)-@a2*+~v?CkBBoL+(B1cap{I{8__kH4GbolZ$G{M9XIHXzj)^6yo(FY zFZ|%&>O?4@h>LWDA9`Hd#P?mSR?vKkVjtpyt-bHe?|$wIdVgtfE}|%Z_p12_ROV>gZr$EpGYf}<`KyTa+kqODz zB&uMw=yu{Ca`$&?d~yHxT>JL)?w!u(|GUTA)875&2mSpQZ~OLyx9_f$_@0lioOAKV zm6NahadC=!1NK)xxb`p3>0Z}{@9GvCwB?gpD#-u>bSda{dJ@Xmewp6kE&;~!kR@4dTyX$bsm z5csKw`cHlT?*np|Wq)72>^no??-PN)4msT4Vc%$~*<;cxbv-Nq4-I*TU zRh)D7Z_|#GF1I3ISWit;uWLR~YVz)R_pGaDfaNL=BdCx6+u=aT%6j>~HNM)3sj`N* zYup<;Rw;EbYkF`VzE@FL9kbQtr1}Eplaux4*FN2p{Vqpdk_k8mp8%YkYdhfI*HE}) zzRaWf<&2NgK6!>-X-;#sx{^9~GIPh8dv(UqTh&>@RQ0BNum7wGY`^4RSW^bq`cBXw z_SL)J3C~RnyrM5!?2uuyV9CEULyLVvx5ZV>@K{}I&O7R1pWeyHuZG$NJQK;-<-n5y z`@?=er-U~3KeTkvxnSVAsL=l3RmQ@5dqZVS|5n)W?tjt#`-zaU%$27Shvsi;3Y!LT zE69+TcA#@#2mIXq!{^Oe*1Mr|DknMkxSegjy&_E_a9S!8Q%U5a_>`#(z<{p~bm!e+pn@*%vX!Mo%~cu4%}SN%N=r#C;f z3F_XYdums~$v0E|4?Rr}(7G_gWU1cesou9Z+KrV@$?p_?g9y9h$_X;>9vl9a-sAL- zN2a&?>1!tL-(6~#mfkMPJYYD>IqUevza{x^K{*fV<~KZyj|;4u?{S$yq8%Lj3E3<@ zy4y@k$`&FNN-$YFAF#M{Zo&2@UQqggCX*XdK>g9q>EcrG$Oo>&FlOmF{FbLm`vy!I zP<7*4K?A<+qlS)#DJ<=cCaKXJHCmt$U!LB1cmDtb1JhMc7srs%vURj-0k_kJMa}x) ZKYLglSIq00MtcSz@O1TaS?83{1OT1V^7sG% literal 0 HcmV?d00001 diff --git a/docs/img/AllJobsPageDetail2.png b/docs/img/AllJobsPageDetail2.png new file mode 100644 index 0000000000000000000000000000000000000000..b7203b2e66586cbe1e1a0f6a08ff0d6df1a60e35 GIT binary patch literal 70557 zcmdSBbyS;8*FH*HTC5a;wZ#eU4#g>iq5+C~io3g)B1H<6;!xa*1d0TQw73LP+@ZL; z2L17T=WU<&`|GTA&L3x;S;@UJlY8zxvuF06J=a`GgsQSEE;cDP8X6j|yc|dk4GrTx z8rnlF%!l_S{jzloXlM^Yt)-<^<)x*mom?C(tnJLv(3s-<>YZvV6g$*Q zpL2wDKeGUp^+JIfm@ui*XAH>!uj9k$Iy>JwvA?>g)Nyxm9v}OuiH|NJX<(2+WSHM^ zR}f0VO-X*fJlE1aCn9O3kz#TB=~0?P`lgu_Voj^FRT~{2^ny3MAHlOm$NU(1=#EDP z**6jGznaa;gAC|DJ&5S2MsV3L;Vu<}XMQ)UkQ>wM(u*@v@@{_Uh&9mGyY6b(sQST6 zYr!CxWmx9M&6-MjNaw6E=GOPTid{>{(RrMOf4r43(3q*Bg3UE!jq8g}2LWBU zs;cx$;>qx#l6Vsu@D&!`-RJG(o`TP%Pv68m*2sD7aq~Oyph>@!3*bl}N+7u(we0r^ z&A&;qdhgXjcg%?BgKOlE#Q8&QvL7dmo2M}(Vh`J9essFas-L~@r+9bJ`x!iyc+D3o z^}tIa&#~{pqdG|^=$O7QRS}+KSfj!m8pg*TR&iAmV)GOmd%Fv~ySD*o-5D;u?CRu; z_v3=)B&X|&hQ>kr&-;N_frQ6>A%>egSO#Mm6C3{_%bCeL-20-JZZbM<(vJ4_<_>OX z(k|x4Zsumxp4M(w)UxtmRqdc>&(YAR(d0p|HN2*G=ByLQC)^n?d?9uX6`F}KPD$g1 z2M?(w31*oc^uWDI?9puQO}oy>nq47EBt};1-iUwU)yby#GUfT??ioj&c|cyS-!FL{7o~4%OoYh-dc!D}0%=>o! zC;eV7{Pp-x1`X{yKEC9mzu6xjB+>rq(|;1q!v}nSKBJ+rVWR)~H}{?T|F;#nh92)T zg8ZPFGbcYfhTaBpq}6-6^W}=qkBnB2bP_&Vjk9u*)n?jDip}iH@*vO`S!%&DUo8E{ zeE+X>fiA*o~S_%7q(lPGdYC6CW|7`*BRF3)xy>d-`6wFz+IA{yB8} z^iYU%|4E?WddCt;_I`i1p_j8v-!#QLWx7lxf6rJHOz zXzT<;M`NC02A19>jCN+6fy%ay7F$!=7PoV8dX`dgpXG^&2?|By2n=hbD%ZO1{1ms& z3eKi~#IS*s%6oWHh`hzSmk;%8)5Ho#zN$VkE&SVrA1XUtEI)I^0&A5Q2F>rh4E+q> z0N!RtM96_sH2J2}`7^R|#yeu6b3bae0?b5cw<@_|0jDeiBY`;Rs zwceKMSlo?GF@rtC>@1OfR-*<6sMx8#RBRL|#g<>%8RvJN;9w)~uveK*yMi6B>Tk2> zWt1!xhf85xq?MUgLdEQ2E6t^UP-yXp`;)40gJ{2VF~x2LNlBW$NMiJ(Hn%thdk_CAAV(S}L`P0Lp&A=-= zp*o>?Gn{Dc8yhawcdpj$>(HH>!^;&f)R=lAbNj0yZEBHW@>r-!i?0X5F5M2m9YOW& zTPH)jeQ~}0WF#AqKuC}ab<=)Y=K>8Jq4Z!W7$W4oJ2!^eezdNC?K}!4z7}J?V-@S_ zle_6W=7+CjB1-+nZ+6X&DVTXnZ(MRNx@O{=WNnu_gw}eoomFl!`BOjYx8pwH1>nSe zHkU8fnQgJWkf77M6QHzydKGWkO|`4;igi`^4e|{Y>UX2m`E7*<-jKIA{xu`_2m8Gv zNV#3?RfDTWHteb86no4x?nqO_(LdXJ_t&#J(KZe~w1SL~(I(D)?2^62?lbcVd;*Pt zeainDSlMuRn>h8&-r$f8a#4W^zcU895NOGkj`+>{*Q!F;i7Z{h3)gLAyRM~xhjO#Z z18aCY6j$^4V~shxIHd%;r+RnA5#7%aeiNF}p8O6t%!-@6Q(FzZId|ULBR4a-;LyGy z>GlO-{^K;lX?MlzLfGwo^64$%mMdP3#ORB?7Cq-x9K2q?BxNZ&mqlk90swd=0u%5O zv2ycm=kD9&SzdJUfWFw>>;XK`e2FuR*3+>s;BYX>++sJQ-4qVD+=?A65-j63T|F8S zl)dd1GDWS8FX#gx7TY;_w6XhHH?Ln?QaQ9RwmmT%$^-IIui$W3n}6rcytD1AEw{T> z(Epa9*c}sn*Bdl-CrV`7JH0(S-9T#^OI57GX@9-Gt-fulL7L*@gWNgxb=%o9sAnJk zQ2gZ2{B>GxxYw=vA$TB8%5e+1zJ;VnqfhTuJK1YN=f`?0aB@nqsoxlHye)>~7@D{^ zx-HiI#$S3@W`@u zyl5Y2NarP|vC*0Gs=Mfhl==mZ8jiiNuEi4MLI0!J`XK|$2YiI)ZEL$No1c4ep9CJ; z1&Q{C5&_0UGwMb!$9wi2?3)9@(&~tSyS-z17|?@w>?sM5y{)75XRS$RhuQR^oMTtL zd9E=3CQ3fa4aT*+SNC;h`fCXBXyo2vvo9>78Gysnf{4_?FbqL@ zK|vPaQD}-B&dYp)x2*D@!j@)cw{t6hyK*heS8q#57=l5fwfv@q^#VGl+pCwA@w_UZ zRu`yK@#0`b5(@xB6K5SoYP*-@?mC9e#aTe^R6~ncxxiUNnhqLidSWDAJ{MuJRy_Dv ztpOmA((}Chi5iZL`{D?%3huFZp=3DmWB!&(_U7qaeB{Jw{iShukKm^i9+T%=7KAut z9r-DuU zU7M^iSKp4dd~&2uv0JAf}GA0J&Yp8hDNV!T=M|jJ2ptYygq#1s1XuM@rSpsvg-ZkD}b|;aE&mFe4 z7F%18F?4SHV(fGi8WWPj7iVHpFv-(SLL2bcgVaAro!Io|eLJdkeo`n!ms!v%u^HfM zIWosuP4!iraHw+z#aufU`na!4v$5lQ)sTj?afzC@_i*Nmk`{aYIrg{^^Upa<9hC9U zpB0ScocMZ@V9WJi-x&gU;g%1+gUt@QlX_bvxLdb`Z7Yd|%)Wp1l!{JOtvEk^x!pnb_m>IX)x0j~!QW z!{iA@OCFi-dA)OY1Gz9jm{bm1myM`FdRb#7_H-!xV>{^ z`4E@Fql@}z+cZ8g2iTnkv}b0G`ERGqU2*<|?+w&&E`mB-b|-aG`auvlXqXw~Y@zAxy)Z{}YQ>lm7c+s_WL{8BiH7~30d39_#O>&7J7|7(4gnO>0 znF6r*yj?15p7efgEt|!N8|R@!U<3|qc02+uowI3?g3P5n!@hK#4TVD$9|LtNcb*doZBODeZgT8e{=YU#g8AFx?*&rY3NNj8KTGC+t>~M|2^8b06lil}{x+5yVcFih zFhB3KAE`6E64s@vqBnHQ5(@06%}G?E0Rolz=^f`^CY%pk-;mv4;FTJ&k5FbgD)?f}Y;$Rq5E4gSbRdmVWlf{>-B=}rv>t0oL$SyJs z&u47)bvnR{Jus5cC8VPDrv8LBPA8fS9*)4qQHNmS1pU*Or*{N<(YJjt#yS#r|trIEi~ zuIMY>fhJ*mEgIHo{B5gGZbJF_;zF9@0;I3(dkd^fh+^kw9uJYt*GQ?O-8^-XNs%EGZ-3vP#wLNatF$ZbCHL`m@Ik*AdpXnl}OVdEWs zVQc@C6PG#obk?rHu|}9U!qLjz#OU<;o%LiS)GDGz_qB7ClSvYA%E~dAf;&|XJM^d_ zu1|xhwv2fgKqK=L;Z_h>CTe(Q2)0X879O4@+KnN7I~ww@SEIS#5b{~G2a%*Y9$!tf zBf?BZQYv!T^r9-{!@-?hiGY%4ZrQ7CdhaZ>>|WU6V?|gU@>iNjABJYspzA#}X*Z(> z(Y?-wedb~Q$2nl6o1ibWl=C`_9aSo z0#RsUv4m4Wtq1Wqid^@E|B^T+i$@Q#vXHLih9?_XOZ0>%|-R znHNmHGePkF8SwXH!_mb5jXWT6Bkz%xKk5GqAmSbt!0GY4Svqk`{a_`U5%V$pqHo>3 zMJqL}2V>ERCsRJ8d?9Sh2P(|VuTH|6=pZKx?tJ0*r^RUX9FMF3LwZp?A04|83e0Pz zVL7?5@c^zn7VTytLI5FIm~jeK&AF^bdMWws{ITzrtH@r!MvOZJs4H>cV6AX4a2=VI zHq*SClfFMa>vLoD=C!rQQM6@dK$PA-e0$HHX5{B{vcKY-{Ga@)uC9)ck6(2`N%L5> za}h*M*{-qvMidQareNr zjN1u?;QqQf|FLzkI=7v}mLk`=rmcxn^L#tj(fMi05T4Zf9My?`@uKkZCNw?J+S*!J zSlIp5ZpLEm-0T;S2Y%3?RZ1Ks1F*KX#`@U4uitCHIgl}Tz7cp+?dUOoB(kmL{z@Tk zTnd!C-NDf9mYw(usip9Nc&1mbIs;4O%r{8_GWmmRq(5yMSbNI-u~zEjxa@;5Xe64< z0d5P;c{;=WHt=7Z>VKMHJGZa^VM!$YpOm1cZHNv{DA8cK*NAj*aE@w^GkLq0%5}ez z31c!FYPcQG-{$yC^KEc1-bv?r3KYafQdMu%n`{#VY=2j`J z=F@RHbfnRG$wnQX{ROOo;j+1l;hjpSK2SXtA0i(5gO*4#D+^xoXptCphe`wJ_#?G2 z3hZveq0iPfL;MTJTW@rx!j^m*G|mj())J2zR;<~WnBPa`KvHM6;BdJsm_mbUiBBtM zAEvC`4BxC?wv-1Q37)aZKhX34B(^6L3)mZ}si_()i31kk<920n(rALAl@&9XUW<*~ z0H`lg7}`)cUTeT%W?=9D{M^Qwt9n#{(DJy%sv>yk59R zAN}lcrilCH$)Z1J3qRkwk9d06+;D@>NUg}YN0F23QPNxSTnk^xc28E9loR5no8)O)8;l zF(x6i#c*GhJm=M_%ED}QVX%A`lFD!SS8KdwR3;}I!63B3djKprH%f_}4qAz)+@HDY zn!!-X=x?HYncF_F?WkL4KGJoj+L4k*DR#Z2zJ9$54bx_C=Kah$##(Y-Xec(9A?k6_ zk-Ge7MAJDfsl4$f6I$wmyP6sy*5pYw1ooWwo@w+lJNi+ntxQ$k;oJCODx$&F($a7t zd?m@T*|j??5gJ|Q&L!x4xqE|zd-x4$`VRBlUYlvG3?uzB8t=I4 zZy#>>v(Rt5z`|1m+ zz1?J#^W!ZY@6^)F4xrlK+Knb)OKYn>?+$v%VsBTin1{dALrdG$r06<2l;~ccA`V-S zHwTH8yzCK^(Tho>YJ550u=;3D`sw0i!4*^!an>IE6TiU zS-52NS<#0dsdnIddMy1q<`lc!akdoWwOz+RM<)(U376wW(CABWHC|@;T_JBf z@fZ0xPd8g3k^0{5sWX9zd_Hul&Dq^g4*MSV7A z_u^5U6&KNoa#??|Ns@@|eTIvBqdidv0xUc{D_y7f1u@M&t*=qMC^z?UM_4izu9)E|3fpwN#-Cn{N zJ?-}Gt!xBzK-`r);U$T#oUMn$J*vn4a8#nNhXrooKzee`o57pIJQc@1d6}7se2%F5 z(!#w&@Eq%U;E6@-Q$)r^qR1`k5oBh#nQptG`q32}b^6>wqGfwR;&f`JF`c5=U1(}U z@NR`{!FzKY7F{}Ec6gn$e?79-=AU)ZRzx*DG_|v#mz;QgCfIu0LCEQ#KUdqu163_) zm}0flYw=r$lErgx@i8#oEMI=k8VCc2*mF>mtr_gim=r+%{A)^^PH=3zV&PmdM9UX`V6^mfmK zh`|N@ZX<3BtSKTvxCn(mCiY%t=@KU!HXgKp)q1y>9udBOC5$P~_CD!Lzzv2`bGiF{cP`9BU{ZI>|Liy&pHhAQEW5^ z08GqOTfo(C|131#(aS&Uv06E763SZYpGiv9}tTbtJ}BQCEU$iJ)tc zp;h}@5syrgJbfX_%QJ#*(CM#=3@PDHZ4jLVyQt!99>epw#o{*WsRr-Oqs)BhWK8XO zS;Jk7x#gLP=EI-=1t)a=qwmcsNtCpfWMByH!`$x;X3H@dw&SNEMC#e0IvtD!z^QS! zq;OD_RcW%9Hld(sfG<#iV^L9aEfSdQ!)@9gc1OC{ty zNeouV+^~7zhL65nloe=l?ioC3I3rOg2Jq)@-~;MHwW8 z>x#q_OuEbXc7To#IU-b{3gufn)_X?tW8WUJ>-Fqt%{ahOwmC`jYaNRYVBV2xssy=BE;o#vX5S@GkC^9?&6ViH#YKw>~2zlD@9m-`*0De+!Rg6KTI%?A2 z?6n{(!?L7fk@DD;;E(CP8{%@DEsVmfj0yWHH(6!hX_mG>^FeMc`SpnTPbR`j0`qTM z2CF+b80X{xrQ3?|bKCRTd5UNrO^g`=SqSCPhu>{UK(+IG*nk8xieQ4)ex^+6%VjA2 z96~9EoyZ7g-JO9qKanS{Mp%*<7yl7{o~&XievHC!Xtb8;YUC=3luGJ^DMSZ*Dkh>M z%GYhV!Prx|;NUIuWEqIU@gtLo5xp4(X7`+^^0IImpf;#O3gj|Y@hIhK?s;dg5-4(t zM~9g+TM}Yd$Y5A}6)-0C;Mt$9z3-IH!70}&TwXF+_3T$L@Nv3*Y0~+6pcGdnp-^pm zqUQ7rM5_c!H1RHhSA6m8O<70}Jwa?JnU^a(0073e$0~}2{rPtKcf2Q$?RcmQviC7d?K z_QfwZMl7%1*lK6KXGq(YIJ#6+$)M7^P34>)u4c0}6i9m6ty>wOs{_BBo~q7sr3A=B z?{-iV&%Pc{JL(a}Fp{CY0 zR|@c7b785G?PpGcnu7IB(0M5#z(PyG-w9?l^jekuY38egVb)#V(AMv21n5!-wxG{P zZrhOqa}>Cl&l4;!Srab{Oo;TZU94?Pnl;Yz4iAo?yK|3_5_frwdADY{S|SoWV&;xl zV{4WWv7_Pb4DgpeZ`a4k$KCUp24vn-+g6`%gMu9c0?*bkT5GFH>9;hBnx_a&A9;=$ z#4?c^x~z{kDdtJ|ZC6&z4Bk-aiV2^b1?>k`*PqpSzFbSh9imKpwmWxxLUmo2>JCZ$ zDtvf(8<4)0`8{X%>|TPvFl;j!&UX^Mjuc;(SajG0C&)s4*M1FYAd@A$6+P+N3yHU< z4^okHr=G~KJ~3G=jyhkyeQG{J+?oy;W?gr%&mGp3Jo8SR`Wc5McyQ{+z!s#Bz#Fe8 zHDH=83(gi0bnU3{*5li>;lA^0xD-i~V{)34T96xnLN3cCZQ-JV)e=nF9Z%_9Z70fF zPGNV4JDyaE2?CZiN@u%SREf#YSA%iqWjt^9Ep93^{o+9OhL<62$)gA3IBP8)aA>ni#| z&}k=$ZZ?NLnZCpO(^`A|5kt|2{hl~e)A#^nt*_^t>v>g7h|HKm19hV-YM4J`ZUD>N zoTnG09;{6vGUK^6y6H;D`+9x)YX4?5xKt!vO50C6TP#hCU({5v!hTPF<>O$6kpIz1 zas2*Q@exnElHw-zk!f<3npJ{BCM0gAxa$#OD#%!H%%!`fX>YcbgYX%vS}ldGqe54n zkHv39YJAb)vzW%sGvB?I)7tcLGjdtY(gHPZIYCZ+cq4zDvs|TiW~O@{?m&N1`If>P z3i?w8s0Nfz(6bCOpD9rAxq`#iZsxm-yy9fZx@Ad(>P#eY%dSv&RYZ+Glt|&Vr(*XO zy<$_mG6Th64ZQ;4$c^RIlR{g2cChUXG;YYp&L3GaNF1G#^2e8=MLmE(Se5-30+*u& zu|uU8;;pT$>bD+R0=S>%=|Pp1mDi3@r`r>rZf-BaoL9A8w`ei1DweNb9c7B+(~wl= zMd*k+Jh^vFg$OGkfEyefE?S|-MkKZn(@8MIb!Akn%92!Sg#Xz(grK*cg1ZDI1vB6$ z?~auHpb*U;Wq)sgb2vJw%1i``g?Z1SMz_bV?YO^7SL*Os5CTwRHU%l1<6Euw-i&qy zWK!(b5uhSXPwnYYt5Gq^SABMa((IMW?@%&RI|PlIoo#nY6>Bh0_=6E)=CXv9Aht6H zl?#VTRhC57Q%Rg^W06A;8)d``gobBnfz;1 zR&4FM!i5Zjs~xPD?g12pN1SVNi6fH=VmQJfCgOZD5c2 z`FC~Gp(}@@>2X{#k+Bw}ylnT|#cFmDm*7vY7@w2u;mWY$Nwt=H^AxB3FOK$I#~|>( zsZ|EgRrmf*UNkfk^nasgG&CK2%$`xxI ziK>9Oyqu!nSy+$&43Z??78$0EZbZw7cD#T*<^F9*%2ogV-0}y}6D8vxL|yoIJ5Srg zJ536$jbg=IGcxYuW=7?&Ov(K*%fG9_Y*z+kY3gfS0v^tHUe#$x{vkb!l?+2Y|&JZ$KcrEp%Pt1J=v&CI93u z$Dt@+>}5kA=Rj=E4yrc9P9q0z1q3dhs9+eI;d)cB2GfH!Gzca4FD{dmKMph2C>@Ju zR9SOr2n%~+kVJI|9XW3H6Q_#prs#eh%dT*Lw_0)jh&4KS)Tv6|)YO#IEAnR_hiIZ) z!VM?biLg_Zy11SyEp`gl7pO6lL@m;KRk9@amv39}yabU&oz`ubRKCCt1;%0Cp6Db< z8x-GTWC{Z5okT_n%iC80#TqoE0o&cnP7``y zr4Abrb`C2E{la;w_idD>j@i~}qq**(*KDo*NmuU5D`lA8fEmw5qzp6W!x28(I85;m z&r6+N+3AuL9sqBNr}@A$wRuBKKBjN)$C??250{DRnVg(Fo#CBW1!MXzzyoLlbrJ## zqA)r7vre=v7tZ|`WVPAG7fMBu$D)q6kO%F#ehbI`RCQ)Lv+A$@0W&Iw6eEAX5|4-?KT+g*#19os}-eQpdHI(1$X55c|1waj$5JV-d&qCzFIV=Wp`MIOw zd-Ioy!;1sn_P5GlIKU`X-^tLDFoPEV*8KjBy%4QYbm;t&_ihy~=Bm{cU0e?(0z}}m z?hbxQJgB|x5A?n(OixbDfvBRfw`!C7SXwERWav+<_i#I*{#xS?liF_zPT^-Cs9%>- z-6morm)YUH1m?=5$Un3w$jul4$H>3_9razUD>U&}?jxv7_`Pj#$Az~Z-~#~MtnI4F zV|(}%R#2FM-`Q}$v2P{MQwtLy0f55FSgkCfq-nL%mGN2T_p+1mvFIS^Qwtykda}`{ z6x6VFv#*H~5ZKnog-fqzd$TtMs8$OLS{?n{L8k#-r3ZYhK-N@6cc=l(r?XL67BJ9V z1C;7eNDbo{FZ`&Sft(;sa;xDN2%XH%7M+rC2@rPVb3dtFQL7!rscsK$Hyx>lQRBC} z62T3@~O0*+^Xby%osEMV^4TbVkhjdB1 z_Aco?00Wf>uw>edk4vIHxWfdzcYZ7cn4b47T;C(Z0lt>63N79|lQ_P}qN18cg+d(r zSJhVACJ*bfyl=;(^F|`_i4%fz^99A7`ZGiv_0klk2@<+xW16pbs;}^#j<(4gLYX&P zw_AN3n=I84EEGZ)i1Su%=ktm?2NhwEK)i;9#lpq@-HDR+8Y^A^Lpr(m(NQX)tcni> z^rMkCw6?UG{tI`%rP1Bsa>XaO`ggtb1V~zcf08}kywCZZt#3m4*D22p{lL?!uJzmV zg9bJ3mEC6gs!DLyjIYUg)SZ*6pny=MLZ{TM&E&}sN73&v_DufDr~kVCa+yo8HvN<%3+BX_u3{?bJIG0|L^<1MMgySynf@5`RGeI zEZi=~1_qu>BGVvOeU1+Ma@&WS?MR99-EN3qL@utUoz>IAtMl6f{i`R_5{IaTg<9AM zd#2BstV+wp=pKl#!CkgZ-ogcX`ni3dQazN60DOqqsg^Ui#_2y_C*3LCKjC*eL3w}J zc{BJU$ny3|u&C_gUC~0T>uIf4@iX}0Y+c|*J7WWh&299|uofF-q;`@@CRGC}4#RQF zaKd!*XN0AZ&u-@;OeA1?5i<8B`q$|1MeVGsaYbZHbMf(>YGJ1Z=wE8G(}0e_gPN+T ztfOb5XFBu~X*@EZkZ`(hQO1@@^S<5Fdj=dyR@3*m{m-HW(T4NR6$AZ+puF40y^NXh zq9yJ$5eeU=1E-!zpYBFY3a|jcJZ;?3h~MvQgUqz(vb2RqQ_Qnw!D_$7^t5VqDxV@G zd9E^V`ZIGruhx%z{z@OmCkO;WNvRLhZbSgre2T`UtWkoC|8TtWaCDH(^J_TL(j~@glXLrvf)#uV$; zKxR6XXXWlgZkjKh&9!>ZSA$iQ`B1O3A5ox-qgs13tdByHw>EUgUknfc;&OFdU2Prr zewES&t5+3AOs;e}<=R?vB)tWh;ln)Gtsx%y`#W(r7~8!dQGmR8HH8@NF$TWWlYiOf z6EU5{lD4s*KDFG&XI-Bh!3=4nLEES%e4c7RWEh)_5{SrpN4LslXCOIq_PFe-XFa)= zy+7p{#>#m8-j%DRvM4uKi^SSz24fl#vvE=_L0FfcODQbPddefDml_BZP+}aLOabPq zH7ExqoiqHnp3k~iGUaFYrl@MKlyh>XB@roG!FPWm13jemRq8o9^yIiJSDAKQ7g85( zvO`z;f^FU^v|MeNe{68-IWLdO#LEfQ0fnqoiG|43;BzDQ%|54G&aF?$l{K%|4i1p* zQND6MLiBpZWcVyPKHJ^&zZrqtv~`W|)dXSwmzSW3+F=NxLzulN=??&jy@&uw}|n$wUB}f{wiUT@Xw}lYP81 zJ#esMVy%nesH4&$Zr=B%`eG0f90hFDXS)XG4U+@%*_Ore&p_nWUQ%5+a5Sz~bbtD?Iq+78zrV%HvjJ zk!MXPn+EdF_;C0yNoOtnT|iQTIlsuK8aSIpdC_jb-nsBR+@aiBIV-Ns0Fn(82opUX8*^2 zS+Nb$$kNHlC@Eh4lQISQ`1nLcxp=glT5ODuPIx(~gcJ3vT$U*mMFbbBtdC2Idrd1O zL|*YSL=>K+;m)zRY@`atW{IX(B=@T?cYEyLMoC0^dePb#2 z$+9-7gwC#Qn;^a(Ob2;@7`_gI!;O`V3skSm)Y#b1&A3O7&oq^_Cs$A76E=H zW?qfIolCgu9KgPtzZ89u85^AAbJqxz_Pl6-kE7dI$*Gub@KU+-!k8|y|knVNgJ(Sb=-|Mys%TF4qm4UNp^6P%M=*Z`2QorJh zs?pG$m+NPo3+Xyvc6<+ipf6ng?u!=c7U(saXmVLu^H*uPjE(M&_uSnus=bbbvcv*rg8vEG-RA6Zu^r0{!31k?Lg?kN~Y|c2QqI#MaKw6$+vJBp<>@+syGXT>s4GO4u5@VAeRk&RW}#d-O_rq zRf@cu(*L*=R#2iT2(W3q!hz__&85B@UHCf_?n4+8>}=`O@o3MRJ@~vvX;Tooc0vWd z$*QZg<)Bh%x~D(NqT1eCzpo}DeCK2sDgqyF?uafROp?!0by#h@GMR!A@@gUp&0igw z*;?u=3asJ-imlWgPT4)Xlgl&NG#;9W035CMt@;^=0YVXjV13F#i(0Lc2k|eFbnR2$ zCVZucEx(WWCT zMV6-qshSMSxBEEc`0E_(99mAQiTmB42sh~U`xbVQ`SFIvU6zk;!vIMBPFHY$xmW$} z@ZC#``i7>ahNi13%jmGx7TnRRTW!4v4nyB4eyIRp2`^(msopMbkfPJlHvSk6B5{Attu7!1 ze%mD&@Shq^d9TrMzf(WV8HyD3F}ih=`t&~c6Nh&)KC?x7e5)Mw&s@iy@40uQKk?jM zOv9aM{!{G-?&Al1KR%=|_2#Xg$S1>-mh-_%bW(CuoDGwNY*DSLQp@~4=33b9H#4P) zp_+dJAt&1Zk!FykoXkX8Xb@r-3M3cgo5}x4nt_Pwa{r?j!58;rMVw9og)}o(>=5+c zWn4BpwLa?I9LE3L@jEOy5ck=4)f9O`>z`muM>jpW@E4DD{(RdNj7Iv7hQ{YlIR2h& za*6Zj+c7@P_^P`R*&wO>lqCU6dza6X|I&B-_@YGLm;1Ps$jL_XtbOTGL8<(PeJ=ji zo^vX{6;WybyDBIPz{^`&>cceYM!SGNtVa_g#W*cB&i>#mVXw?a7!& zYbiJWLe?ss- zkN)vEO5lCDjj(#cvbnteD6x4|V4)El(q_SRH$$qi3I^*$Vh0`Ik3CSP%#OV3xGcNp}1c=`Wkuo;#JZYH41-T>%{bg z9Wg4esqBi0)%r{gn&YxN`x^^!Rm;CtP4in!Z#Qi@i=<#34j%sNOO3xBMIJ)p^Gpe+;=F}C~{N4SmjG#_U3>AEP%wY8}K^{3S;TDFkePC>FJxMO;HJ-WL@Wt`&oSyBO ziqq3H*K>)s2QT8y=xE=x9Zs~uIKSwRH8NZjs3xox2q$I_hScU`&xzk~KN-4F+S04#acy5V9u5OdU1{YsVT z2_WJL>pcSsLyxcer$SuXowAHvjTJi(bgZA|mQ(xjv(<1AiS*0J6s7|ZC_ZQMT_*_i zdhR^z>{GN|lxNXh;MpFZT={k*;kD3tD|=lZjD^>B0VlMrl@lH7Luq}l9=mSsj`NJY z$m|H=keEYvNRt>ZMa*7GC-?FuWp~Z4)(m^Uy?iYw0@(Q&h=os$iH}zKT-7({ti=WC z*DrQVgM%*lmOzFe7d;a)ia`xo^FMF#sSv+j=yi?9ZR2=Eo$cZ3tfnTJsfNykt|0l4 zND}=w^M37boosX(v!ta@TvOB$LW$E&>n}sC&15t8;@OYEXGt2q$cg; z2-;-|K_`-o$=!PIl_&T0y{K9wCV*Z6U6LA&?x1<*%8(j6h{?fdY$fifME<)ZmedD) zG`fUpz2@uFIFwEqR2~zJCLbLniTgvdGfq9eo78W}cSb~24jtWjIUoOl_m~iEIE*lc zAmQUYLQ%cKEy^Fd?Fj{^b9cQk{q)ae$8@h8H`@@e>KWR54_cpF>IZHx?hhQXnOkf~Z#?DkNBByQ`>|DmS{|2x*f_ZKXXy z!!(*=7Re+SuAvjoG<2p=0X(2lFeFV%cqlegy&H**;;p)Eg;jg#*-Hjzp(gPh6+!L<%(%d!J{Bl(htwC%>|%;?z(`*0^+xuA!Si_EB2 zVa%#%K~Ly{@G-H_!c_@1j5xjqu5IBl>_)5DdF2a|6QXb!-`pNR=jp|TXRD6`pSD|W1a7SRf@dIdEu1vK_<94j z*`t1ywe$21$lJ#SH3>0H6Q7*l91o3Yx!%za4h}wCk+v@an|AH?5vCl#J%GiCVdF0$;(;Ib8I4~Z?5SD<+>Woo@TT)F7zzdfxa zlPY!^&6Xe;<`yHTZqgG6LRBGzNk1LaydPn zKZ<6FL}Uv41$>8IX-mW(4Xh4I_#dtkn#R*VU+RU%o{3)Q*!oT;r}r3K?{ykmx*IE} z3xfL9XELW#9B^3fXM(~aF@rXT-}%-d?IUd*N^~j1b2qUQ#oZS6p={ato#8~&;Dw8| zUth3-x-Pq_eta+8RKr}f*e#|_eg=0#t!Q#q(ULge63J$ODnRH?yHmW&jPTpF#mY~Q ze7-kXBN=_3Q7Xvxj$#)S*wO}`!-Aq{aVB4=(5Y>9Y95RoXLSC@fIo42w9ipW2A(A3 zhIS%)LJQ6=WvMosy@tp#H>h%BNJ3yNRzOd0Yr74oYjCyDL=yk$fDa$q{qK-W>_y6+ z;o{+CP<=%vK6p4_zi1ABD*Rs1lmqrk-=UQ`E)fqVr9Ubjd zkgFA~eB5MUu@|@<&$SqhHYGfanJ+fk@5~G{>NCBMQ7Bqtu;`0fD{u7*} zbj2qsG55Kdu`8ARD6-IN{qD6`xLaq|ytgB1u};7-JjSqe|6(+7Vz#Aw-GxYlUI7M8eJF#BvxCj%l1pv#%=#yb!Wg`-X?^!c)0y3Ak610WIlngE zny?)xVvt=+HY_GCBUR3?7uUMa`NoRcxr2!wU`jroAMLw1ehzshyl~L~hQr16>KnaI z5>u0vQDs(O!A5mUOBM1|9P-(dPzwHWg`B%pR!DlgJ$f1oJ5q(p+uEkN^fW@H&b@cI zgSB0@+amb{Jz=(OtfBwo74k7K_Ii64_hAKO@?Pbi{DmnY_=dXgIj&#)G5~3&U&W40+!e1(yJ~xmh|g4@f8HJac(_BwKd($ zOu)q6kV?FLogl4YAG|0?XYa5%w=k4^u}(IPDrcP~zHYoSQd0>!-)cMk-LQ>;KKP~0uJ zyBBvT7TnzldUNi5PkYAr#`x~9n;-i}lC}4mYp*@`Gwqq+*0BaBc`qxfWG&qmR}a^K zUrhBAr6YPN9CWF3zV@F?27~y_+BNmThZo(*pDPk784UXQ?vvS)FKb;B51YiU=A5`S zZMsX+Ui=@-fdC%1%YYl~%9PXN-yIwNV`V-jdepFw^29JhTJE(*JFuX~1@;t2a#wlv0}3%%gg%mxv}AoaS)vt(hq9;{tp4H-#0 zR*AFfH*!Y$@>Z-KJOPG^E2V$38a#!IyI0lNQc^PJo8}R-Ot;GxvSkMxOZxb#tpy#A z5(1D=P_=(8RMQYWMaR=aLZY3I2X^l7V5WGOQ#Dp;xw1J?R|+Ra*%j##6VK+H=9^8? zt>Ix==Wi47P2E~@uo0cH)DY69JzZ2I#p#t~#BHjs(*y!TovY~Z-p|wWGREjgH20K^ zqvBxHFgOmlcF#SR=Yc=T#(*Q+URhoX$({oU-W-#Bd9SVfN{xx!7fyxK#pSuEYi@IErKudjq57~RjLBI>X| zKMqOA7d&Hs*ZshXr4p@S^J$|IIy*tdDB`A+myv< zJNpHV3-LA@I{rdU4N&_+B43-JM~MUA9RQ)?(cv~Ps;9Jl&?-B~tmtSUL_x|C#~9i= z+d%?CLU4JS=@|nw7O8-cwQoj}-g>kb>tmd54wQ>PXxTE3D@^|aZJFP~kp zLt{nmw)}i~Jc)UBfc`MrhtEVcNJ6oT1}v;)@P`C^XD03;*7r5LTdjs?0;+ch6*;1g z7EiO+jM^Q@xH^+|(2_cXmenEno8=o{WF-WPanKF<8=}%Fg>Fp@nzr}U0QAg4oKE9*%ZLf)GYWtLBnF~aXzR*!|y%h@WdsHtHJT0t1qu|$M;NW z&v}Mljj?BsRI+7Mdt%$Un0^YMf*h4PGF_}tUdn?D7jqKfrPk6yudeB#i>5AQm(?1q zlh3LjEX3Tn>*(*VIGusei=fQ4{t%PEKFdM5($qa@{MuR}> zkTpR*W=%?J3_q*-O6hZ3wh($M0PYhBCS?rqCjeAR0#HueH+)$HECHQJy{i!){}tPR z0x4`nQw9;{%{h7=*hs=E{27qy0BDjuJ_3`n9+_^o6*_m=Kk9Zw{H9&R>=%0^D(ZTfuf0;yQSl5hmP8+F4g_UoO(Y)HaVXeBOd5-(r!Lc0WDy! zAxEx0U79i7tWP2bvQ73{013DLEGtoMgpHtOw01A`uqBQ&L5n>#(f^yIR6oS*v$lDm z`Y(6wg=nbWN4-R5fzJfE<5b4+9P{#KrAU$u>WPd5TiiJ=R8r5|^F=Ltl5dh+?M^Wh zbREOsa_?q3%|%UF_bR&sUng6}Wl)(_VzSiJg`JlC$2vhqA9YiHoj@VREg;0}>2Nyl zys>cD@0J@lKOdJ zw_XMURT;d^8jfKQM%?U;5uAptwhMxVIl?w%$GRIhO;GyFjSRr|37 zGBFW>Q9ZZE*pyJe!Nk&^LVEmDC;2KmM@Pju?U>IhNjrDkb=S|-N+VLrGkiYvUM5*+ zR^&b|oOgZSkQ4Sd!Mq>&d7nz`RZ*^UmKY=X*@6A&b*?5mp$z@VI_8+AMR?Nat7J7g z($TNyt~bHM>|fp3KI|Ijrlce$3ubsHdu5Hs^{d%ueeNv*;eM4>10U}P!gpXF;^QvW z&8vOa9qaMyXLr`#)<*5dAOqEB^s07~{S}ep6I$IircS!ja;$b0Gh$_HZgocD(4mnd zHW-tWD9r_fzY8*+ak(C~J1eAn->UQr(Icf6=`v*AO8N|qH_K6RyMhfJcj1NupW+w> z4XK~wf70A^{j`YN;cR{}evBsrerw5W*?(j2?(;Bb(b6~_5x1WhJ1{}Kxi2;3BV~11 z{?>pZ?PXI-Re?yR$;0a1)C5g}prBZ_$n)@?#tLF+iar}%R_YER$JZz&MT4b}EY~pO z|Lscp=b+Pxct=YD5_KKcha|KPZkv}D$I96XtOGHWjcdxBPvY}>p}P9EiZeY@5}kQr zMaqL;@__&pB_rDcWev0p_OVkqPh+&(htD>qRb;v;I<5qGemsGA{D^7?%mGX^soD_$ zP_{EVZc`5W6c)kuzK?~C-Af-3&B7~|WYwt>P4?J-Wbm^|U=klwHUSUjy$v`6%|RqWO`($v>Q>Bx_x+D!lDL(k()$ASozh(U{Pl{UrV0}H7)Z_BvyFu6C) ze9q(nR^&S%#>)JKJ@Zw{?Bh{XnnAsosYIDK8mZh&Yr=(alyGm#twhknrx9dv04;}@ zeiDpX4h4e}X4JzzL-2S|xhvoL0;fBRGCR8sdc%gAcRdvDJ1>Tx&zmCA8~Kr}_1opl zbC^Q8KceJf)Pq5(zV$oQELA(368Kal@#E&}0+Fhg ztkRXxg3!lX-4DR^{j zuROn|DbYC?$9dSD5)<*_49Mi!?Y}r-3uZNfPh9Y(U00u{-N6gQgdAGvKCTVR3$BdQ z#Oa9%WeVBo@aAd9Qbgddwwt=q+oZWIK7`W-`lWj}c)DuuKKmbg$; z&r347vpafmqSzf2LeVrne%{UfMZQ~B0qJ9|wRP>eNoT?)LxZQ#1d@fBk?r+)w`3nO zFhEqk}HGsQNi14=nery=7_sMhKH_z)zgZXnK;NcE>zF zuO^!SqDzdKtFuS)B?&-NodGwvH)DPI6y^BxX%awu@_gE+w&t-NrVttjnhZGNRcyXU z_?D4gpysqIZy=UF^CnZ$1EoLg5Kf$YgU?m)`CD9A@Gg$+?furAugEv+B1{V;5aBh_ z#eC8g`p{;M9?^NLBkeWa`DG@cM%7^>p0cY0LbKgI2As?IGwZMp`fI-nm3lo zYZ+(q%@-zLr#-M9w*}o=i}d&c{yZGrtD*)lM2xXmDweLOTJ3)`6@=vL03)z;)VmEVd?6ZKwEzh1e{oi-IF*_*E^ z*f>t!f(BYuEq?y6+seZ^n%3Tsd9c`^@bWfMSsba^H1jo{oyOGHZ?-$?ew08|WlTyW zSrlcn(X1htMpDan_;M&Bqg7cYeT^zcEC4!6<|u%-pR*F>l!^fv9$h*LQo1Zrf9M_r`Skz*^C>1mg+aH^{l5Z((JNpa2FSRCfJa{H{XY9MBn{3Og7|0k%g3 z?D7YWgRRj001!n=5Q=z%>n9twZGbpCmDWlN*a%k~38LP2F4J&#JD02ic&9qGK$KI; zw?K|r%hK@r85RVzspu{MAq)W;VGc2Fr~48bt{Yi{JTcs6a1l?*O$1vCGKa5=;hm*zFVsfp zjoD6z=EKCJSAE~7K<%2TZt^;Qmz83Ts(1#|^rKW{FTLCN*4uf0=u5@tExBYV(=EaC z!*?#-XS%42)jfJj!a%>7RPYYRa}-WGErmlv%f zRQhsVNEf#?BzxG-Y!t3GZ%}?p_@YQx*IqTI+*qzM0|B6r!iZQpP$)p_05y^%GpH7v ze#1l^&6pjK9Kkg03C;@2|G)yXqtCK(C=ALkp_>W=&|-Kv*#q*;3#KLsA(M?{xKx>F zlASuPep4;fl(-SQ`-N(6CTHkdh+5^PJ*-%;0Mxi4NVr`>QEvcY)nO-Y9Goq$AFE*# zjCd<=-&6#x29HR&WbdNm*)r+%?7p`@z_oqhYIBH!6j)yKb-iTk3$n$ZCx;VI7mjR! zbi6Hf*r^?nJL&aY!@|#W2|4X<;+|POAx!INfmN~SA_Hsyn^;4kreY&FJP?k%( zUWu}^J?YFEJ4n1$^4hh1ed)H83#eP;{^ObY9c4d*Hj3X~Oyq(j3v|zlJOfAN;u2(t zbFC}sH!8`pJTJxR0+ne)J%r*q629_Jb?0kuyg2XuVpC3fCM6BPct(w+Av8~K{NR3W z{*j(qnfBz;6@d4Y9?9>TO(sOEq#p$!Jw#!3{-)reh75Nxzr*YgdNTQ1F0_IO$FX6N zf(?;2UNYA+o*>6FVg!U?z6iEdRJa|D6MJ3z#=Eg^Uuhu{avb^d=5plW*zoA(EyMmH z15VptJNdZ@MMWbY1p9Lt=U2PI70egf4*73=KgCX8VdKU5zafRr9*aq*GX#kCX=4A8 z9WneK#mxo~0l-5Roe9vRs?!djLnmmvWx4qPkZhx;@a6<0XFXpIB%Sq=ro=<>M_-`5 zHc_MuvDEbZvvJC!5*vjTYzo>TfkSV!OPU@5(EwV>4u5*Fx6Uo*zrC8R!%%WD$^E1O z>!6#p{F}i6m1wm#gP(@~F2nEMQ9W{T8p(wK60-Ok@e>NfeP~$53~JVF2=Eje^eod2 zipKs0Ja1goI6o2S+2D#JaE8~N2>ua6)b;Se6(6aM0PvHnGOc8e@pO!aN1-E_&>3FI+x>r#zraZB-OByC*X z_7vxz19BsYqc~2Tsf=r0D>=Z3cj@6k|8#g zeRaM5>d&keo9k$pOx!l7C6AvdFLBYLKuHWl}90C z|DE7JUoYal{Euf10o)sD1R&=B-hqGS;Ezv)-*|0tE7=UyC#yMQFn z`=7yUnudbNX6V~_IB7FIKmbx2kd4#b0^NFlbuZ{o>;W+)pUg&nh^z1qy#V5o4xNe7 z6v!MUZu3Y6E)c{;=e~)7;;l>^ER8}tBe9h37}%Y-H$higQy@wf4mxBV_TefNoi+L^ zm|HdCWb6bPzAuR|%b?iWd*WeN6ynby1j?U~fnPZ~MhQ^BxX1U|Q6BGee&`Kr=#J2- zTG||iAO+?o`xy#6e{~f0T&KjGK9c;QNA@Jfz^jm5199KHaGWKAC?iS<|E_uC+#;#C z=F1;CsKITYAoKv#XwS#2Cn$=NZ2{IVzH=?tiR~bxDEG4&2t8blTycEI{60bekha1k zv20g9XkfdoId@@LJ7pJ`kuEeQ#xPT2YRM6$4Qp%k3J<_*cPU>D4MWd|JyvF>QC4z6VwA_FFWhuC<8L?|@Xb zkNQ8h#V+PBV_9sw7r{q80dP!TyqyMfxIa+y^yVuCcCevEJUeUj9xP>J9jX2Dl_mx% z5wDu2cx%}^A^)6CXjMKAKxXvs^~Ls~`T@%7&oXf6<{CH&kSJSf^Pi|#w5Yqu@4@uE zYGE^0_3U`=sXBv0Zm!QCk5^fGquyjKf z&wY$Eb)o`Fwlsh;UDejX$kq0>j-%drZ&tl-JJS~gN9j>;#%jBAP-ys;Pw}Vuo{U?g@}`L=gpD@buiVeyXeZrC@80~XpJ}~H zNenW8G3RO?bnjThQdurV5&5qc--F&q#_#W+Tl8$tHQ%4e99dQIl&I=A_*`srm*SXh zz2wej5=wBIJDjidojHsQA@PP^S8}U?gGFxx_1MNofo)9nCNfbz7yG|*8IF9J$yLIg zi1o$5tE<(SK|~FS{x1w>>aSqU$-5IktoQ>qtczCH{VBDzq89r$3WeI|whj*E;_v!+ z(<%HjvFrv%_dW{Xc7_1)ecOf9??P@XZe9k1wux)NsMG=&=JLk}9)v`hbiZcA{^;!A zLZ7`c4r!@rCB0>QLXvEEbDKNn!#R#cW~?Hb`2jufAKhdMDHE4 zq;~zW@cleGCEojD$r^<=MS(vrhaMjg!i|L3d|?5IzvPj2(5lk&(AI3+b?Q=vRXS^ z5{AuQSc zJE>AvP`d>|0$*^3Z&L-gVIb>=lkY*n2^uqEEm_U%e%!=Lt*(BDv-ZcMu!ERafGQN~ z_Ws`nKLy-u40OOv?WyEVjyr`dk8RL`fsr{tk?JbfQez8gY6 zFUYJpZ197vHn6R~u6wM#z`s1JfG;Nl3^wJg3{mb=h|*8u#0&VS)kkz1n4y+{d36q@ zA1c_$EV^*Xo-mRE0JKl$YS>D4_QGp=+0w;$!lu7Bub;!WUbf$ss($*TQ5*OE@selY zGV2Im-4(&*vsB)N#l^A^aJKJGyYxERYNXk|8#2aJ}-?4=Se}MoFBfe z`NtY-i~JZfKsY4e6|V?@5hzY+q3Tb>IXYSRq%vBP8`W>i7`kn$X@fwv5=?gIsN(Wh zsMX%D(>T0%T~3_s5IkO*Xd0qp)61qm3o`mP8io&L!2}zs(EZWV<#PU%l^qDabbgGU z{&2779DY)%Du7>;3DZ|WOQOSa1WUeCcJ$ax!1uq{VlfPPTPOqq@;3Lh8u@-$azG>) zDRSSl5zAok(R+EO)%$TeLQ#$1$9r6p_B3;sKS_1x4b4f}nP`yUNe}IT9 z3&NE^t15?5JUv8OYgg~qJmMtGAN!tRwYyYF6{eEP(gu(|m+#CAzHq(|!b*m1Xs$9T zF-cNp>&Sn7GVRH;aEXs`Oxm%9+!)95L>ZLXFr@Lm+}e^Wp9wWa+tc0L1mMu zNdjMlbng!M%n@}Wu7mSgO37k!Ev-b06W7MnSBdeG^*|jWWn>5d8XTBu(#Xy*>PCVQ zM~S*DqydnoL~@kHyD#n^#iR45zE1ZdV3YufqZD6OD6fOlwLPW%B#{`EVM0f^xRSEs zfQ@k_P6q->|F&u+hm8LyV=n|sPLT#krjNX0eI{YqFo{n#1m5tvVfR@(G63C(;=$%q z$5C>>Jvfj0cx3wu%*>I<$pI?l!(%Z&O2IR+E<~S50`8RkI3JX~3?)%A?SE6)Ky^d~ zg=>D4mPA;R0O~L?on}gfV#MX=qLI?4L2hpRP2R4@9XmS(hS1!N_FVA@G{Ywbg;bd- zdlWynZS*60*3cl#1e(O3h;=KWEquEPex|=U9J4Ux!wLC~`~Y??E=JytOvs|4KZh2X zLc}qC7N-ow%R#s5q^0c)02l!~{HsqUMqQ4E8L~my!W^3PowyC`uq|vn0IJ4MV`$uv zP>nog4u>^bM=ljgOREH`ayN?^=9`4`^cNViZKO0*kj!z*0(D0!3pDQrrgvjr4H1h9 zbegQ1&NTR(&Y@u(PJf}_)ju0t&ZEZ0D|>Rj^m&G<+L6hXi@{_2NtWW6*1 z_dcF9IA2B1&3%eYc`o;{y$7wc;468(r=e zj;l68cU?a%$;q>tXuEok^)tLWr}mQW-Vxz6OAWQmHH`DG+^F?aO#j zcWY{X9i8%mOwhx})0?v#yB%3S;$T=@TJ;C1R`d0K=-TX0mU|#_JQ9HV4eE>S*}2oc z;3TA=0`Wyw`L_g9`mazRA+;r+W%*}@8^IGouvU%paDXxyR`y=O{%Ig^dmF?viTblv zC}N3gh4c#mcDlbl?Z2~Iv=JI#ly@6tpv%Twt9rtJRI=n~C*wxF{45_9Bhl*g?fB9# zL~BvV-(G{6yY%VHo56ZbcXMg*nb{KjC zLC3T1zu+(s@PW3vEE1EygZqmn6BtDQ(XS>)L;vK^#VJyOWM9K2GgcR;pNNOQO?D%j zyPEEI3Qhy_&ds@}GsIbM!ybMUYd)L=CSZGgx!-}*a|t<`%Plol*U}jS5s+4sX_H3= zh^O9YSGO^h=`--F|7?F%ZtOzW?5z%|tLVez-~_{a(W{ae^`Z~M5kKqA_ZM4yZ^3Ne zmCymBYz8I<^4Eg~E$0g&q!bo=*q^CT0MT#j93H=T$puHXUYuga?;$f`Zj}xXdIf&J zbqo6916T}mJNjMz=`u$%%6o68{_+Nb`Ecj07R>w6|686gAjPgoNIIiZJMm`v_v}GL z>!tfAuQ>zP8J~Oher49|L^Plr0g43QLpa_{`pF47aFNjTYLa$+^Ikqfd?enYqF{ZKTAhb z(gNhlxN|ZzbEl?_A{Nk}`j81PC7&`0kr_b%H*b`@~WHT=X4Z0AE?-2u~tc zAMD|XYDelO|M)n($>N$`f2HQtSYvA;AIz#8P`#7-CGEa~C=@~m5SMMvy;xbgkFzLJ zN}B47J*f%u5V*fpY_W zKlsU7N18i0AqhGbIhEx+VyKc z472>Z#pEmR4M!hN?!*rdLE_Z-ujDuC3l`*ZwNAyKClO-OJ@d;qXuw$5Slzwb^&WP> z%N_}%mvVVBpw&0bMpT(JHj2-sJR$FDM#ml9uC8q%pHr?Wv*bQyWp&P~7S0zGku$z4 zY0i1%E>3%)Y(uF*T)-K?pJvc+t`Wx+w(+gaka}GGlU!&3fUjV0!j3F0_2~26er)r6 zEU+*Q1AMa=R`t-+wjq{&EC5LA?&qRCODY0Y5FpymehapEP!$US73P#l zE^yhcd`ZI6IXsw`hDchRPQM_@aJ~vM-SVWaQ;}iGjc-A)cXJ?uQ^WAJd;eGMd$1Y%^)Im< z7%30)y;>>QWP7|LQ-5oQxoRAC8Y*Su<(c)iJ5^M~rf8T`<-7F~9-2^DIkjTg=hXLYjm(^?RDr9{G`g{bPe-U-}28 z;S@42IO&(H?dypZ>FyF%=zG*;-cZxV`u17=)aKJWxUPKO$Tx|G)MO>YX7M4HaHnwM zR;Ly8H5OE(ATssx?adKan^E&rV^2t64T4jWquq<7@JmxwQ^fRQFa^f(%g9K_B}I#^E=t14 z_o*=sYBKOp`$=EBRa>ZAs6y@>&SF))uK3e&RYCBzNCmI07>DCKZpm!j5eHqJ`Wh!1 z$XfyuNy>!<{o1vT--$Fc{HQV-dCfMGTnJEgXmQQ!8${TZo|T zLMX}`{A51!T=tZguPGwM&aHHAel=g-hpv_4yetTJQwi+fsBg+DJcuV$uJ?8nXSqg> zIQjhrN7_Ir`EH+B2>YBaVr-8?gkP{l)aQ}I4z}HUdNTHmA=n@`<}mlYyz*z`i~j8E zNeb)v`HxFPwb=x6F$5Nkb%I;#cb=34=YHbU6+AC4&;t?idimiw0|xx`*EK@?Zt(N- z!G_#mob#%zANLROTlzoilEU1ApK_*>-aTAf2zV~X9)Ke@Pd9dmANZX5TwV0y9lNdU z2Aeo{5%tZ(y*Ee#)$Y4ql}cOSG$)bz=)`7$(*^PRol2d{s$7#Z`TLRcGvJqmn?^k9=-#9APkT8!>X|5!(oQ7M z2N5nnplB777yOuB*f5-6)=~S>15O0cp|A8!Jm8otyE88^r>} zZkNtSvl}+BEiXkKZ0tO+xu7PxMxe}bKTvnscL!EgS~S8Db`c}e%fj(gS4?LH_AQb8 zXJ0SHqP?M>AQ}OlhB5ums6=XpfTgdu)d*h?$l`gj4$mHX6#t#&KMFULE6vYFik^a6w~dsn!&RXFvtTJX1}FOLqUryEx%Sd}Ss+|insa^R%rxQ*4OLaN8T)h@^R6I8)gK(+1RRkI`5Kx*q zXrbsXReOSFk2Lhh9dtR-P((~69RSuzuO~;NH}t8V!-U^|%>wZNFC4Z%Z=NTDok5Ua zoK|x<&fYtH!;wMaeJyyaa#YR*R#2>M{;FJ8?zozc3yrFbw;(cIoL|GOHO|#n)>MK- zNHZT~>4=3$=a39JJg|2d>421!Pmn0>K?Hb6tMfRvxVSr;%^$$PcNu!hkDQ0m(2*E*;%PbtEr4>nyEm4qj8v9J5X0_YXx&!%?af#dj| z&^@LcL@!eoKEl&TarXa2d~QE~ruZTihl_P$mvq0{Nh&6dfuL~+nHH3il~`xpr04kB z+LV2CeFtqpMNmCf%kP18_BwgnWtFT-%=%YUL&1TPP@RfO3m}Q;BWGy zs2ZF$7JkW1Z`onUreRBi&WAS%F=V-!76|3;3=xb6MwyepZ6rKuc)UIvy+=E|_=pEU zp+*5(a5T#2<&KQ3aU5C9RF>ZJlqx^%j;E*IMq~Co^-$;XB&@}1)#!8 z$E{afl@(1=AAqQKpblKxZ4o+^@2c{mx$Xb(XYoDuYWDv4sbvz4k zQ|E>EKkd)k+LRfuq0)Pj5JuYZB6YxYJj}bHUuk66ZLzF+Q49DlHOukmooytDnSZE_ zYmaJ{)P&T%T5kNnY&f03kHD2_VfNQ9wyUc};C7DU#Z%{;*F2iSp63(>q}VOi(+M*C z(6hV!_y^24Mo`dt?v~P{scG(1^=0|+1zBj#nVOwm3*1~2?pix_7XAQExHVp4UeY_d z>PB`pKfkMfw=-;3s`OJ>c#41DAr4 z(*Hw+eZk&sY}N_Hd|tY%OTFxCLuAFpyuZn_z=xpqT(UkxO8mm*0|Pl{ikHoT#&5% zFRpiq4SBBoZ}jx1bBYEH4!FC}EoWXaG5sdq`)&UEz3UOjd*3ZZAzsUAkq?gxP9MyS zIM{kdy0V2GvK(a;x3rEzDbgA}e^G>^C20nwJy2XAYpQxFDLzyV#Y`Dkk=%@3fR{wQ zsqxr}b?e#B33q7dnkAHaYu4BI_mjnhIkmT3_f%?&E?c!#&Kr1yh~%I)o*s;~k6T+U zc?~rET;@8M{_h*w6x$> zla}JP!EMS}@ThDw`^D2ZO>{}x-#lwpMKvGXcXKiMD$hMkK4fI~(rI-GF{fRx^=fXt z3=ZJWZ*1LUDd!T6qt%LilK6CP3v~!E7@i*aD_-}iUWEQ>~k@#nc znYc;wq@%ox_5!;&*MTccR=!4x7St)jLdEYok3!XhO@JvmYtXO~ui0XlZmg4R60}Sy zwt>+?f5Lh8FHNN=>OWf;$Q@&-RL`MIUii4Y!P71Ia}O7j`kiUz8e?NVsd=WiizK6J zjbmJ|+We+fdUwt*g?{>8mBH+2TtBs^<{x1_`zto>bKFOxs0JjtkMA!(k9cz2@z#a}j>xy+H^4^=0mF&IIG0XNKH zRjCrIAUHXo2<>jgH1BpbLbUFq&IywQuj?Twof-F@`TA^WY!C!EZHdL(i!LCsn6U{n zKIV!TCtK@yTW}mwhI}l4)9?8>v_IeYR^9y($|gPL9}?%eMCN;s>mNc^lY%2sg@vOhG+zR{0g6nW)v zN(9kR-Ovh*U~`Zd)sC$;$HlJTNpwRiM%AsE8nWO`vWlK5x9FrAiEA_MogZ` zWz>HSuGGb*vlrrm0S5 zU+`+RzS`m~|2wrf%20NIdmVz}(dbLvq+6v$=vTHgBD8<`DbwSqAZXWvAcrk3{S4WK z`38M@YAU(r(y~o>jKIr)HUSfFqmPkCNY33fPUA>)?w~9Rz%lz%_xK=H!EWutiMPfA zK+)JI9$wm~9lK@{HMpk}3RB&;PgZ3pQwSwxnH*TxCSQENxL#%SAqaY|r&4Iwz8(Bc z%x4Vexv^F*k$xs;*L57qz>TyO({T0o+rhDfHLctU1P|%Z;NaIjth6(#?7c*Vw@mTx z^F+u;I;zf}o0{cI6nM*-K3#cL(3o79`#zBMO_3Oo{tl1HRm! zRcFrGtMjH7mVVh=??ji5oBhK&nCD#*WoBc^4l4?0Y>q7c&Z*jh!)s%6?-QN5S6y^s zEdd*^1p}fl^Q9y))?o7>U9gprv|^d;}P5y|y2HIH5N4Q&UqL8yi0+ zj$cKqB4}q*Q&Ua3UW9jtgd{4Y+?Zc$^229IEW<$EtG14&=8m6v;@9|Z>icDctWe7B z^HGW%Kvb*)Ny*7)aBtgM(#m{Vv&varcH&TJ9>q@L+Z;C#fMl|+jN+y$_Sm;t5~KJ&}p`&SQ&%cHS|)!;}fInU7pc=a;*fD7BgY zZnknzr;cMZP;tzx*J0O=*kbTn!>=6JMp`XIFn2V_nzN&1;vNHEGt<{G{5qNyt*P)(~V!j`$;~0)f)M zxd8uX+_GrA{^+i%&YacjEy;#;#6+~^`-BhLslIW6_ zR#2d*yIPF9`YWwFPrQD?lZa~fVlF(lf%=8+{-7qh_0=Tv_#{%&7cXBX%MoH=zZ)VB zp({#ae&#drMp_coDZ@>(CgtcupODi3`t2(M`DvcGUL@Nu)*pV5Q;@X0=80z(409Pe z4~*rFua4)eef28p?*&D(QLMy|*14 z>({iqDf#3EEh-udh*nJ!o|u`S@%j|38r#`v65El#SiQwEvcFr*nozL6oQMcAcpv<( z`glm(->9Ya#VO)N?zy^x^a|-cJl}-1LE;C*$+tqqBJ@#pFJGT{u+#X`~z{vD>{F z%I%%AQy^~@=2}uGWK>Yvs+fgv5ZIHds#Y1_pBWS{#T(hZT5ad~_r-^Z_m-*Rq=yDh z{;t>$jj87hD^{HD(zAUhE$!o524zlcapo?6Kg3d!0V?QCig3a)cEc<;yrE z4N9Yu4+=>cO%jWFvrG(W#v7nxSKCKovC4{Z1dugrGFWzL-nFOD07Mr_Td_1x* zD=vgqn{+l*+mu|Cm&qjjr)w8LH2YlMo-uQNFE%mcXrZU6&cUIPmdFhziD(35(U0G; zR|2cwOHMgx42J48*uzh|pEOc37AJ;Qv97}`o?IzEv&CfZuzex{`z6e)dT)1Ai)RJf2$Ajo@24q^HJ%Lb z(p_`pA?^P9;B2N>4OL@ya&Rc7Z^f9JUe{>RIerGMXv2BwS6hi>J3V#g`NDccdwWb+ zgME4j$J2;-PLWYDL9RG9Eotf5ed|YiOX&{`;D#77vL??B$D{24Gk2--u%w2Y&7K=s ziJ_>W-`z-?c>i7=&=wr`9zYh-72C2>UupMn>^u_4LfqJ>)av5sn4QrKJHn~dl-4es zOG4g)>UwVV|B{||$hE`L7jZT^m+y5jcYi#8(%M!7Y(bs z$b&n0%ty#?`<5Qy1_xsH4#o|JA1@cWMvpIiCe*$c+b4<+()^hT1cxbwWYqj@nn=Eh z=Sr#du08GEik*miMfU8{?haP-fsHePJ;d0mR*bE?v9$Gs-`7FG3I22crsX{ILZ900<6CPB?B*|1O{H?b`|?`q=U~d~&<8;tDEq7tD9k z*`#I>EVVax)ybbK-^f47xJ6o#eq-~oixOW#^S)B)zWrI_)5T9jn|TvwFM^%Z#FFxh zBLhNyeeFte6MkWB3ue_Ti#KM%Qc5q*6n^of5jFR}wHF`*i_?8@y=@*LQ$Zv*T{?HAMX~_ zFcoNNQL*s2`{-W%FXsO7J7Z(?Z&okv$4}yjn&vT+t{{k`#0D7 z^S#>ZFSw6;9_(YznG<{MeJ<{`*UR+p!_FkbpeGKA?hAi%$)Jf8sKGqVV6&i65@~-; zOjm*YxkkEBm8iBmd9qm%F&f2<`CNT_cn7uw$|<`ibtao~xYt*y;tdm7Vi`cbUe zYHqApf3661iG(2iE;q>?}uKby&* z?acPKjS`ETjoYoDyJGX#4w;gP{SqTDV(!m15D{dSb|pZSRZAKh7BUElE~ychZ3Val#cv;tUDr`3jrISB~&Iq6~& zYN>Ib#u?0QCiEjsYb(dAG@r~wQHvpEv9GO~da}x(a1ALCeO0Wr9*;jh(i~nLwj``i zNHa;XQIw=SP{brc$u|cb6;{A2G5(Qj6y60|A2ev*OKv(NAuh=^GB^WU#KJ)yz<6$r6{>`i<+gA~CiL-6kqQxw2#!0BNk!pbxmNKZ~ zI8FEk;a>_aU?^?@9fI6W!IH@eYRZJwn{r1y4>VLL zToMeUbor)vQo@2%oUDGv3NonI23=B%jT>uO<3>ecUR#|NF z@OJDWj_Th}O~Y-mFL-|H66UmwKa(M`NL%8<8RBUX=-WXb@UYc6Zs%lYwmdxRJ5sY@ zu%sMKFtF!nj!$p$sckHBwR0h>T)2QPO-ol{`I7w1&55!oYvc8^sPXw|Ls<5xgI}Ac zsOD4bO!Z&LkPVdVJoqF8YU5o65izX7<dh^NQ9vIdF93G46~^Y!PlV zaerW(Wt37;2Lz+{-|)`|37M|7Gn|*w(qEetc|hAbMUAizjIq`a`k^-GbU50c^Kj0P zQ>&^#&ra;V4X!nZo%|pSnhC6msP&$gJ>_z+ck0QIeh6Y*o}Qg|b$qg~Z*Q+(IUXC8 z)#h+Ni83rI^qwuB&3Rdix#WG9CbT)B-1zrx+$9*fcIBF!Y# zt9SXBEc7PC5eo{lO~2e+|^hUvQh?;Nh}6+pxK`C*yZKy9Ts)A<@K=n z?rp_3`g-hatPR{5ndwA2w?D3aU`y+GEWn(rD6Tv0cJ;Yye5 z_wgIO{Kx1@Jr0MXo9=j1&{oCGpi{16@QOj^ePq1X!{!$|EwfP2cWS-pYi3iq@lrR~|-N>ghyXLi^{5)%2w6>IgD4J{Yo6uE+D*?ieR;*b1W|6%tejWQ{m8rf+0jVLKyIVY znZbI$y!EqcBhRu9{scs9XS<aZi?!!V=`D2wf@~!6{VlRK?h)^=4smtE zUr!B5%Cz+ooe~jJS-z)ttY`&xaQ#~sqwdskKcmS$q!AE}VpiDb;#Le}`J69|jBbWG zxhrZbci94Cffa7n`(r;ZVl(WapcVQTkLSlk)&^x5;HOwT6{0eEAKO)_JKn}s#x({t zh15+uD(`QKT}?y_Ce)^XRKIkE#lK(JtTw(yEK&MCihtQHIf;bI8|WE2snN@Bq1y|S zMvIVMlU)6jo_@&Zduhh#I|}swJ)!VgB!}1R&{lTc*ic(8m!Hq41=_qLjv6~Ls>VL4 zaL6T)kM$q=cR=(noz0H8)l`?Yxrp%cbU4^q8F_gbShzV=9iN`x-|)T97G;`Q-p9^_ z_TkP;ql5fERDW@flJA4+=og=+ylh+d?jEHpV*;@ho9jdhf0e#_5NL0z3BypK!(TrN zPD&V)nFe$3Pz=J)%W1vuWI2;r3=u`w>^VS+8IzHdg=h)WpjQ*b;o!a3-p(q=GGJac z3l5wz7F4v~?>HUawIC+brNDhOu1!+@Jq7}7OK~TJ~qlXF#>D#Xx`8Xd` zT$PEII)d^G+ihW4(2D#0RDiIALA_;bu4sgzVJQ+YgjRc8)EYEO@hZ~(Cyy6Lpvt#C z_h}w7(<$oWs0x_*?%)+$3B*n8Xh#oMj^j4g<;+g6(KuLdU+v4z)9S**!ya9<%2KRM zLe#$lhsZWrRnFb_c3CY;14&HC2n!N|CvLK_uu@c_%lk5(cYzqT=YOFJ64)UoqyjK@ zi%g(xU806+qUTzE&XYU!&LGBzzVNDwY2L174sR@ z)Rghi;hG~v`Qzu|ml{X9&QNlLw28G&Xq$kSmc8`xo!L|LRQMmDhyb(5$DVD}?y5rU z+8yWTZR0a!=Woegc$@jtwLL><&0*qSySHYX=E5jR{)CkQF@P{SHB^5IBWC90JKuZL zN<8*uR;F}1x-tcqla(7c-SLf$2_^Rhj3~1mdup*rS)=_aLDbkf(C+U2%5ftrLJ2o+ z4I$0G&zL4b&u;&oJFUmp5?D-XYC%BD-*&RD2xRm6?m_+=^>tD zqmm`&^irJh(@lbeOwZ>hxN>1gDtb108My#?={@KP>y>oMKyi)sdwwi8zi4$LwjozvKaqQdH7*FCZ>#ng$ ze!2B^aE`9_se5hNu+}Uv_ocp8`#BnS%yvmpS95j)xm;^?>T+o9tRetI$HES$k?vl* zjU^*3YYw-R;HE^)VKlA2{W-=tgE#Iw4z-Njk;i;&KnOJI?)Z$-oYFu*RH=rn4xi6M z&mzBIDcncsXPX<`R`dNW$YuSIYPdh9d3GO2P7%c!v1`@84caAt1g9YU4 ziE(^v9UgAvd)-Nzo0SlPvE4lyo1LD<)Or79JI?yhe)hHCSG}{?)rxYSnI;X3>S;?+ zmeSLt>iW~Ka~)(wi`qu;J(=Y5|Vf337A{-|EiZoMnTQJo2trVK;1Ue2=HnPgk{IVJaXo!H-6 zF%pOpX_iADGEr~jjEjd1&ru^1$ z>2UF0?!RWaG05AX*Sd`6Jr+bB$$y|rpJl0~nQSs{t#se5P?`fF@=7aMv<%~3hz+Bm z#E20SB^P}MW5JTH?EUn=Bm>a9s#=Yf&5{nJTYQ6kepExR4=-BYRkPGYn zxGX$|(v?Ud_=x^r9RMh(F#2Fd`I|H>HB>yFBy7eza59a6G*n&OK#17Bv7R_^qBpk0 zF8|2Tb~`*UKGyE)l`nej@uK^y2!qmP?EPF2xy~&5`+*n`Pp+wi@P!fi{8F?8hj_NL z7<1Il*tn?%bV2)es-$XF8pDhdLbWlRswW#cO3+ahxX;-qwW(<7Q^W`5VlDrFa!FnxC2-Giz?+z0F} zK2y%YTpTvoZNIFzj&+~TD?vf_H<&9`;`HG_DXHbjj7$ut=|oySRm2L^ zt(?L*fpY~x)_(liu3EpA#~1B7-1OAhQ)p8Y!ay5Qf1j-RYx@`rQ&U?Vk6qK`xtC3; zO*^g8n75<#x6#>Eu8!(w9dI*Cx05Rz-*+~}(f1=?w~E0VqQ4-PexxcbU`~^{Vd^#L zsdRC1fh6vgQXrrPxG55MgtYPLna}EX+AHMAX&oLrVxFFbSJ;ie&>KIP70^)LUft32 zt}SZlUZ z=<@NsA|;S$&(n5PK4P9hN%~Ivi!r7n+Wh*Vh0pZ|CD>i%j>U!}```>HT&(Z4DhQL!Qinq!e8eDK`Zi$ zi-X0fi_&8Tr0tVkJHnaRivKJu{oe-<9asCJc-1?6xKH@x7PQDH#D?%1peccI#q5oF zcgEvhb6<#kby$)@+4QNgeCZBUOHktn04l-?7SFN!lry!5; zh&b0Mv-U%XuU*UYu$)Rrelhmh)zMF}Bi97$q_moonTasAaU|D1SH5~vPoY$TT%#4s z6_0y9Jf7#55(n%#D7j8Yul%RZ_o+Q=oMkUgjJ(uItsVYx8fr4(zvk-&fWS{jQc!XX zvj{2#TCg|GhT6b{FjO7p5LQ@-(3ic*L~LJ)EPZ$cX+*9@LDpgn=BdqqtUt*>k%`*b zbQD=YA-v66lV{bj)}u;guZHb;FEwyx8oTjO;ua5U#MS6W-^HjB_ghfnu6>pf(PsFZ zQiH8I90Ce+4(cCo2EYvf&BgccX-+20_)-B#34Pg^mzV@>s*QveHo8R zI6|+YHrlP7$;(p5JQ99py{gSZOEamBnt!m34)^U5PIPw6U|8yc%jb}Eyyd2LZ_jc) z1CzKkn-rTlqWW$|&k=)2mdB zyTKGH=9XQI}=!DE7qKcO_Te6oXA_iBQi6z-PDIhALx`BrF~ zo1&8-85KfS9))c3rb5OpPY71q<=CA*^*tDU$x6fRbf`EUn2acPNR^c-aA?jg7st3x($L}@mF=cl zYADC?sM5ba8n{?=)ZSk|&5|)@T0A`VCilJFwtYV+!Za1T5{PNcX(!do;+=s;@5sN$YMy$7{{a$yA&gvu<^}y71 zN6(bXS;@H0#=`B}t99#@O09Q5r`~b4xcbV+=lPgnrQb#<0AdB!##|n^ksR>ao{@j^ zp8$TOs#9P$1}D#pK@;%F*V{3zyc@U(jh!C1!}Q9csDm3~EuCn{|2Ix}xyMPP3d-x=vZapP zWAN#?^}<-6e^?+;5O^uoIW0Lt0vW5eSY^-ngrt1-_Jny3GsA;%O1o=Kou&7YMD05P zs)2p!#NhHmO?!^<@@Z07=+F4GrvI!dp;D~^mW;m=NvCCd#eNnc342`ARFsa+wTcDD zqk@n1aeZ}Bl>h;j5$1iwe~4{*t&B1MN@FsIpK1?J$UDJQ!3Xo_JwTQQnKVAg^)m`> z5p*$ez#V%~vcz8Dgc022XsxatTV=q8Z_gyQm~1` zef5JXklEb7VP>-Mqhyl)+MoLaymI3@U~+&rGbR=l1;Za$z6eDVxyupO8B95sL{FA@ zP{kNb{XKgHvAyqJ4yGhhaY8PL2xZ+6*2n}kj+^X}C*`jd3~-g8f7NDj3Ws3$&!xyI z(RBaZD1_25HZMgDhyr_e&@*$s4GxEVk-BeXK%^Ka^}Lb^#cYi0kfIeQZH;}SIXJT1 zCWJ&E&D(+t-~|=$sriT&I}a=jesc>QDEKK3bieg1tARx(B%|U~`kD`WWrnplUSkMv zWK2OG%GbG1(6-|e51;o$@joQxi=VZrP}4WPyw1#{g0Mt2B);`3Ry4^d+xp12CU7^~ z0*Uf!k*fp&h9jEYP?ql*Vhs(;9ynDyJ$xh9&%a-APZ<8q^KYUNg zm1T1ko;U?uq{>U14m{Q~;wg||wDS(dnR!C$nBu`TD!lCXw)N*@QilThli|cB?ZV&p z3E$!=c$CKiQ?6?DV!izaO*^K#bw!y-rIn zj}_0E*!+!+_+=SZ{+PT`6iKl}+?S_E`IEK#0p@kh)+0?&gpp>}HQL}#*i?RaUnIwE zmuQw!CWTCcOrQ<4-*~hEC+}G0*!8(8Ovs_9$?MMv&PsUMKo-9*& z0y?t<>aG!ec0Ix#nLX(VK zG7f3yI47^++ldJhi72g4I*(Fk+S;gxMtVkk!)T;e@mez9upVh@9XDofism ziJP^1kpq%p7h|Dp!k~H};h}ug05@*`DvV$DunAT-%}ejF0{4kTegFZ-jP z0Zo<#`22ETpP*!96*OW|@0Hv|QI7MKzgC4A=wJPZc~@}D_Pu5j9RQ=k-YhQuPchRA zGD8>bW3D1r-?aq3l_#u7-%5lV{FMz}2xJBhDORAI%Zh{#2kt{}d`=Ja2TPyeNC)#` zckZ?I4(VVeh-BuF&7O3pA3O6;JB5~cb~IE%T5D<#%V-ORZ<-OmRc8Fd^#6SDoc(>; z)#BH)l3zJ_TJrMC)n2-vH6FU=xNy*gM-oh_+HB#uJtV*QW#+9gwzvmPt>?GYE9rk% zx|jc;L7ZB!CGd$?D8AJEXMBlw)(-(@W@<6U>kvFO7{YwAAUB|SmE!NH_PcXmH!3_& zgZcSPtxXs6ls_3(F${$0)x9JB5b?nLYG5BcpXx{(kPe;@(tJdC=er(}1P$i=7kCkp zP*N1wO&XArXG{SKqLF+C75m}{DJ|jI0(ERmAt$DAZ_u2qF&RN(i}!;WQ3b)c%n;&~ zG)|(g0%%+xXKwiKbKrZ6t&$*0b=eA=y}s47V3dA3rV{Faz6j4XKihLI{h+U_tketk z?;`PqIGUFDex)n2ZrYDDimJouVGAm}O6mv*nZ)|%%4i)tY@8uVZP(vuTGMG}rQwh8 zhlZx2`>%K5b9JQ&>*?jS-Dch?Xy(b_R^vMkw1H zJ<1AH=WNLlV`uV8K|vUiAB0lmgc9ue4{zXRL=K^>sE`UAHD?vYf-W8B3%YNI)a{m~ zI$HG=8jTo?T+xYX>6zFWrK=A;%yK7c?Ik^0N05U(c0o%Z)7V8~w7?QI{$H0T@igS-O7GS^u2I`Y7I-}5{lnpPuiwHZP5!+rv&98 zvuY=&&4h}pCyh!APA}LeXe^i(AGTCT5Qj!p!z3RbA+deqYcToO(cC!voTvT z5CQHkM4z&^T`;tu3!gBUw-icHey#o2oYG5;(>PP9KEJ6xhbnoy94^%`FmCSF-#;H~ z<6;~RI!co=eYcT(mFsMOT#SaIrEhK127#_N>sn0SYDS8$sP5Lw2*B5KI0@ORqupDk zeHWb9n(XrOnGMc3URu^FJi2?g?QB_LW^F0f-%g%awk%^2gnFfzo`jSN4Ga3}Q#pL@ zm|7T-lDcq=oYL`7`=LT`{@dhc$Cd@YcDwz}0ll2tyw`E5Ce=--sL?KysZY~=AoFC^ ztrUGlcyXpwh%7#Lrew!VEXOL(qPL+x%biN)+V#|)@+@`R7sNJ~hmD({I?rRMGhXHe zUqdE2A6@rnG+pnKJTlaV8jvYmq}oa}iOY3oRVujT_m znLEj`zvb5AY?!wm3WRl~d8B{8faNigM%CbTN0u~X}$?H5Mu$wr8kHQLZY817EeLYwD z$-mH$#Ju{I2$NmZ!jxQE?jK_99#t1rgg;d)jGfeP!p|X|9}%J0(}9*uHr_ot+D1C7 zi@m*%Kc%%gKmOw5s_*1IfMqGai&$BZes5jp(CB>{n7hj;$6=~%^tkgmKlQ$)7JB=- zFy22Ip&C6iQ*ZM%|3_+0b)KmXo6B!nYihO%%cAt{*L~k=+x5*kDp!)PMaKJdnII>x zn%YHZdE5GS@8rd5J=KsWSy!g~U}cr;!WrZLgfO7%s$X`XX5i}c^)sS*Tuoq32r3~H zJLw*jq6)zY8zLKJdV-M8=)Uu=3fm_G6oB`niLtf!toc{tLrIXBX^>5~9 zH9~2Re^xCm`~!hoAD}(Hu<~+!*^ZcbBIY!=GOGaXu^FfJ^(K)bK8RW(Y@`bd)oNKj zwWJ#!S8}abZQ--|k_98)u0d5X{daiaBC zB^%58S%hb1StSd3NOULj^H_IjX6SI<@h2$s|2GbL*oFe(q4^5Z;h=1EtNwhra5Zun z23RsOA_3`)CD(0X3Luq&xft9lPiY^|( zq2l&rO*MHw)3wez0imJF?%G62G-JgEZ|#N|?m03I5f+Y?uS43@tA(P%g}bGaeYqWV z3WwN56MXE+H7+0!w^F2IVIq~)g^uUJq$LNL#WF@6_Wws1Ac;OFJq>$m#)~ZP2N7&{ z^zX5;g4)zvLlw6uX);62$Nuje+Y71w=1u;BuJEa zR4e)>)k+0QFnC=oBGFqAueO34OM?2P2jbtCm1%%PT?t|*t?Hc5#nUX8I9ID)(EgnNZQZ+F@A=GRXJW#IWCRQL_xyX`LickCiXIROp&WZdB$^e z1|ZZM8t>xn)@FYvM$A83xN3RYk3?|2Qt5Gx48 z(T5ngT5e`e=DhvBCvOf2?k8VJKysj-n*38sFZfLoSv_EkgmMR7m}nL*rLe=x^kxj2 zPrUtUmCx&NAvPLUK*1*|=SQd|tP}ty$QfH20HYr$VYJ##Ea(SEfE=er6j*f|Cfm1< zAL4eQ5ymhy_a+J^2q5%V`*=XcmTH6?ecvc}?m4{@W z@7gcd9*X9nO*RdH1x`mv@9#VS++vfk>K|Oyy=nSg7qxs0ia34 z0-JzUPRJln}a2tn5FTYy9eHEky zi{#JW8)dTqP#7~+!&9hsRzbbq2C??jlKOpreov^4UlOX93Gq43C-=M#phkOz|Lh0u z2uvsWh+5sxZ3kc|JJ~p$>|Zm_71vesgMo=)##FZK#Y?uYH#Kn}wFU5d`P{Ab52HB_ zCk@#hyZw2+a;*Kp6Glj6OBcZ(8J_4=wMJJQ;8&YeMuzZ z7GJXz1WCduQAv@9)Djrg4N!^s{WjDCVO05Talv}13KT%niFak(AJtRDct4CeB>3%Y zsp$Oi<+jo}c(%l@=~jH-ldVFxgEV)Pm$s4nM?zVdK|(kw7(h=$lrh~euHL0l9Hvxl zXy>YB5RBhUlrF>NXm2S7%nh`E>#&d#5IEuRm@mf64uGjuliY;|m2iUvCdEqSrxvDu zkqB$RXP;nYW%n7pqqHoIS5-~k-&;v65&I2ptNcGs`d?%H0er);wRY`S|8_DpJXq4h zN>RdlGT&JuI*cDd)7_(`+|pU|4XJ^&OYo9LI`t`%@oPDrvUA;MPKUPdotJMt?U~a1 z%!wUGB^B}eWFPrF_CtJWQ;MnyHLgc;@(M$+Wd{HNt1oiO*RlnI-=5hr{7^mFXAIc9 zT_9n8gSl!;Ma&!e+KCh9NEe&gn%_5{MgoUkoM&;rWowf0r%-EE3)MoOAlvB4_gJm9 zXqO|kW}pzdm)zrWs4zI<>w;>`@KO%q?(0OHaMtujj`{=LhT;AaM|cyl+yH3?wj0mJ zR%;~bX9xWJK!2lQ2azkb{v-y?5lfoN<1vdHy=t3r>h`MsodnWIu(-2&vzhN{T+W{E z%(jA9dTGauK_QdZi2iwU zSN}HTdyCxMXA~xxf7Q?Tkg*RNYowfe6xQgw`*TA{CGpAm3Yx)6e! zi%ZwamggB)-$6h*gt3A9;_bV>J(qEl0p5Q}fd`Qyf=zh155G2;#DwcJBki% zm*2VSM;EuAY3Ib+Vgggi((bR9Kp;_RNm&W9?r@4hBX!oJoZG|wa5xFy6D+~ki;7J#&sRRH8(3KWs+fc86dXzHlRiJ`lYD5i zK$h1_VfjB9{J(DH=#Dhd^0#)cLB>Kg%_GBN{&xB9llsaI^Bp~bPT)!tXdWU7{d4)Y z1x5dAq&){8Ic9FW^2hnf&0*;(n=R5J^%$NFqDQ-%6{#SMweB%8bT@4>NWOco!kz$8 z1ed{DJ5jEa*@wFd(XH3P>ptwPe)Yy8@`dbW&YlgB0jJtH_m0K`%jOT))BZ-}=MGV_ zOfb@5i=9-Q`DCigUd|rr%KNs-_rB~4FD-uSInMk367hW0W3%$@17Tppy|5+-4^|XJ zO~(LC=oO{L0?*(_MHAKpeG}@a-zPm<{tDFLXoa?}LCX(DxGX?%`X<6w4MIbNdr2z`08NXoI z$4<*nAgGOi*#Y2ancA=jTm9t(z?O2lmS^+(a+Y3=U8b1=wlQwxv~x$Luqxu7dvfV} z2PGdAk&Catww=J_$Oywa1;YS=P`}Z~L?l}A$pdsTobj$=EW2q@S|c$HpYL!A@H{iW8iEBLgD{(1(ur_TUe4!vP=q1TMUA?py?V=k;XBJ$f21E&VoLnL#?^ z0G8OtPlNpN6xZlAIy@5G8?;-g6V7%P6>zot)<6~Ds4NU42(#f)Mjt6hI`}Cg^0wbmrYn@wIMz`tFV2j`4XDyvA`~eYNlsG6QaB0VsEEH463oGH=NF?9fOIBDUa5Q1a>)+=J)3cx_BJHMNM^`Xix4{IxJe2)f zYZWkrk&FnJcQ*I`RG|{Qc*81W@OZky3}`(Sij2B$kkd!F{1U8vF;5tTZRkeXU=No( zjJwe3I{@m=-Va5|%QgC^DOf^^f*+0mfp0|-c8IWT`-Nf{M6J`H}2?ltDD)>0uUMn4-cc%P3t2S za75y~5pI{y8vO*&`w9Dpj0zd%X&(awF#sY^x~yBljRju?#*v9`V{IdyOXE0J9pmS?wLg=k2+ub#8LtjSSUtvHPzycBEiH;l5Z-?HlEjHOwp(tsr zK2Ta6B{oS-O`V?JlQ1QiqlYyx{p{AKxM(gBmtR7)F-mlU(SDxu<-Q?Z^O9Yjvm8@vpl2?(k9Rj;Ikfrz3Au)N77M@d^* z9&fBJpHTG5i}&Q^YemqCAkDEKM?x9yzj}Emr7sw&k0l7Btc-1#(z8s{ zwCf&`OR;-|v3GuV>_3diDx2J3iD{Kq2oy}(b-0BSYJ1f%*Pb8FJGin?Z=+eJ2^t;` zGoWsd64JsGp$@NU=3-+QKt$wR19nSL=<; z?yhQ>aL@5=NlAFdS%VT)qC>M51$s};b;A!_`?2lICQ!+Q904m5jA zK{C@LCbcy*ty*-_o2Q%;k_8GF7#Qgt;GC(_(NR%6?NUOS9pH7q+_oJ!_=y8*!6akB z&_0q@DoEUwa#%vWSDVrMdnc7{^Sqj#6o_MdcjpC>pn%reE_`oH%q~EhlHEcIjcp_O z2}MC1NqPAy|8k76`YzB=vz<0EnJ*dNpsTc+HZerANvikDn68L?75rMokS2+~S0m$R z5nJ0r^9mlL;D)R#*uVuz+w(96Lo4d=B)R7?DY@M_LboH7rxI^Mut0r>bdnbEzF&Gn z5D_0fWJ8enHNhr3JyalD@O1IcL%3U2&5OOD7fKTV(EY#}{6x^gKaV|R4D1M4 zD^A=I2UgLPwoSr;%1SStCC12gpSiUbGBivr^ZGJ2ftpJNt8?^$H&+}@E@&)z$h)2c zN<{l+`n7O5pOQ)hJT1S8JqGqmN;?vmLGDL&L9HnQQveJA4wvaA7EK3fT9hr8uQ9Hy zM^sjsUm_rjtRL}%B$*#T&yGJRxot-)Wv1BP*~QAlf=U^jAkq&2kfeww)jM0#%deoL;P}G*QwU4L z!rj7z(PXjIBJGnPk|-Fznud?Ng+60L%9OM~>gXxLtda|!^0UIW!i8eW)Mh_5sezrj zp)JM!A@|n0UKTdqt@J?D%f!flw4hm#b+Rz{Yx8RTRqD*|3{P`rLtgeHen27#St-Kj zdfcc@zh8IxVgt;J3oj4C4SJ>4)(B8MaFDSXclfqg zy4ofb%`}}fjlv3XL(8b)iljVpHye%yb}o)A$}|F_B5L%gnVDI4yMS-yCqdJo`U=CB zZKG&mbq%6Q0PwDQ1CyOGv)u0tzct{{21rCmS5(^b2o(-pxe00d6K|$&7HS5CxQ_GA zzE9`R4LwYZtO$ydTRw|SJV$jjHZ!4?BQoL03N;FXz-&xBIBA((WgN8`4q^fPp$Zm0 z9(!v_E4gHZ2>bxB?6{tvC$|qJ8b_~mzXmQh$mB*x=xIjL+umBDp9q|X00ab`O;uIw zttmr^G!ehUdvPLFW?4krAga9An1UPJY@O&5cXIffTJpw3C;@iU;^VgnH2D$kO?;71 z0+e|*RF0HWN|+UJT-d8k`m=G53pWdbIXlbpIEYFup~7IoIVoyCGNSt9AVIF5hvjG0G_heR00jKw6(i#a(dAe@1ZCXOP}y-KDO(M#r5Z;8 zy!TJn2r|>+qfujJeF~7>GNoSG34mM$)6=sCagi=0z}N8pj~@&7bS5f&%f_d zIt|8SfKL`1X-V2)~@y*aRbrjs4^ zE!4uixDA`PX@BR^G33Y}Zezh>`dKr_*fFd*HTEp)ZKM5owZZ#+*HyPHCU}U=b_7)} zX5~~pEfhx<2afxWcbNwlTZ~6lX4MJ#WKWMKw0Ku$`WRa_X$N8~VyU zq>h^{-@EetAxN_-eocwd+tq_Hbm=Ne2!1lH{o>-wVf*>U&g-Y;$u2aw3DGYccZWs3 z&tvbrc3Z9Qhd8W>cjM^UL8&s)J};BH?f3hJMQd+kY-yz@sJ(;dd!|+D;)RBwgs!ZY zomCusIo!ulK6c$zVBGL5b)HgMJ2Q()%J1o`SNvQm6N@h#OujtMZ=KJ&RN!EZ*4sa} zi!Z}0*WYX3O&z?}@n34sn?Bfx0AgHyZw|Tey$$BrSRk~RAZ;`X`v^U1Itpv1Vqn`I zrOOk-%B5Y_Tdv;L--H{_xj>`Tg-L^%4cS;zJsVrv_|ZICk&C!11?XX}Z`HIs{dCIK ziB^8I{3WwSqkrV)7KiU;v-2VQv|#=1b{&54lnQdA4J#d|#p!kF6y9+xG(vMRVb8ph z=jkQ=m!%L=TSVPsnE@wGRw{1W#hNFk?^5IS?z%^H3u`U_2+%)#5`2HSIMNGVAcX@D zm8q_&K3Ql-KF@d~e@~|eUU4jls{ewI2zWhzdF*_di0gdidkZjK1%oakLd(KAY~{=_ z2po!pA(J1a8I@gWOYuDOeVrRlO!P77{cdIB6=gu!(Zud`z4y!axi=@cE6n!w5+@TH z(yTT`C#~7h_$LL%&wRjFSco0e>3b+e&*+Z3)AjP3md`k@=O!lZ7Xfm<7nSeEo0_(7 zD$8uEGnAStl+|wcQ@g_;bUG1{0IAuUV)kT97ksz3lTD9g)d3wh_+a&BB@c6lt*2JL z=L^%Up>^-q#A_~#40ITPU1>rJ7FMQOX1!6E0ug@=ENv_MGMiWXA^zKSqxUA*RsnPWAIi`U?~=0 zZ3X;f%2UGoMO$0>UR&{9OP?|u-ydXjn*oS==FJaY>$BbHa?Hx4{1jg`q-mp(M@4aT z9?JQU9`I=Saaz7i-`95foI}?>3bkss^TWc}Xp9sI<3`nEU;zA!42V@LwU1t%vUjZQ zgj_l{7POb0kD{he^~n<%yVB*TP2w*XV>@EjULVNMX+FQ(j(2Ei9@ArH)9RCAiatM! zsa#PR27)uB?nR`AdCs2?bv|OQyN}`D)i@lg7~B+hNW_4=@Cj8ReS71FInOKvnge%Y zv%Fw)9`3btu&5bV?(L}0Qv#qRjKLXcqvQ&o1R2@K&&mS1C4dzcX*!sAtb${ktMtJF zweyhP03N=PA6|1bSvb^g4nBu>F}!V=5~pr=LBzP$V}cet@gRf6X)|uCxSUg-^s(A` zGykwf=Mw|2C*0#VTG8 z16@~BR@&*R_KV*DYK1oTJut>GnZGns&Rf|YYVQkcXD{nrbg?uD6B#+vku%({rOVgH zy0)}{z&QHP7r-A+HoVNWZ8Oi8MU7-dD&ewxwM`o@0!;EImDbjt)kI1pAE6~-;=|NK zM|Bzq$eIIY=?WQupqWDol!%;kj$)awp*F;{MfSSsh5W@gC1d}d%E+Uc((d%CaDFSe zKgkx2SFtmvbw6<$dzSfa@q>Pmkb}PX7p{;9_<%e%`j>oida1%?QiN5FXF_G%j$Zmq zi^e;g-P?HKn#+-yuGO7~oonfn<;2-$5KRAK|3Tt zaRTMWQy#^|vQz-SA4AY5sl&%%J{|FWK?`<%qtC|HcMPQoKk(aME^oVoQk#aQoQLpD z7zpq9Pb+%Y%a+Vh;bFiAQf$!4o_OaJz0baC`z;y)hNooU4@|lADJ)W}&oW~pe7%F( zFJ)feO!wf5yXDrK?WjGi=9_!>?^=~YRq|}w>!@N`z_USu5NMo|1Q^2d zbYiSi`;m^@{xY~)z3G2!-2Dwu?Kd8Cuj_uEQM_A1Y^r-;hG=nEor#05>v@h=xw^+a zb)Z0c62ivO=RxO?gx;%+EEu;cskfHnZK}f__XU>Q5)Xos)F- zCWEt2l!lo}rGP^uVfMfvg9bqBIG+3N%Htq_fO+uTMqziEB}!x|a7r8CdHwF?LqwH*@{2wz7-)Ec@X$SeDi-Kc&r=o+CSV ztMpO80fCftEX~}qXvDw@C4XKbc|ruCuj*`Pa)tk2duJ8b*0#lKs6nAXTeN6_7PsOK zn-(b++}#PT!J);9Ln%GxwHD$~( z{$G~b%r$*TE4gd&_u~FcYa`@Hta^RH;Kgb$15x|#b0M)Dj~*y*HYI7nLui+;uxB_C zBj4*h|Hoocp@USYMqpRK{GE4#T*(ODQX_W`7J2I(Fm!MbR>QQ(t$-ULwcfBDMZWHy^uSS|hbgZ|a}N+9Cx}|KxGvFpox8gcb(CbWa?=OF zc}4K`GZPc0v1Pk_TC{__cnRgB;C;x0pH^ow!}f_Ifv_VJYYA9Pj+*X{ zH#Lk&=;!B~G^m3{gsq7slB|tDriCsQ_;weoPp>ZFrT7(~p;7^XD6e{ZRA$3?N z5*dwy{-{fvOX*T z61E?|!`{C>I6y%1Dn1%oXWjKy6yaj*Hs!2qWH@7=;$p!+fY9up7Q zT`JKYhs>sxuWgF(OMHXTk;d6kAN#KBK+^(OM5;kv$1!$p1Ch`ILv{xaEvWc&$E_Qu z7mNB0+kzEk1K%~IK6}q>ok<(C_BS%n8uc%3YAvSsiKGkP4#$3O8L0lQ)vueg#Ur90 zRb_sR7*!w;`6%@hMp*4VC8khdO^6IDHX8G^S$1pk3rOeEE-3JA!ipN}{g}!A>S2!O zvlT;y`@R&il0I+;6pON?pqG4ll95D7(P}5OywHGhwriomkm^|b{=sv55dyMqYhit% zOw(&o6|S2~x!tj7Z487pH?fSWdM~=Zkk29 zoLZ!GI@mb?f;FbHxL<|(`saZPZw8hw4P9nsm>$1;_O6fg*|UO=sJOuO{x6r{Z*1;$ zutN4bSh!f1I=r+->D6=AKj{k7F;ZF}**I}+ry->WbQJZQsY5;);&4ulZC}9=?waAr z$xj_5OVb_!PUYj$UO>|vBoM$&wm1z}J%0P^%Lb17fz2ZP59O^vcszhK9121Xbf9%E zgzDV9k`pw@#}uj9iFv?6;CUVQTy*YCJeZjCRl}dvjD}2 z_l>eL4GokBFILLLCdeJnRLnNF9F548yw-=`RO|l`FU@2cX?rDQ98;Xo!aYMS#O}Qx zX;qvpX`}0?rxZRl9Ci;|g#P-XJ{MUm6V(reudb>}>YuC0zNZxiL@uI^d&y(8>8 z+ihWIO%P;GB`DBavbngLcbcW3fIQ-Oh7q|)gY$`=TFjKBVTCLnCi9MbW{mQL|HHRo zf@d08UiLvW7hSz($2y}MnP$^=;dWgdGC4LsA?h>_7ADj9vSJ4TCsNS-@y9eq8O3K) z!lT+2=B(|Z=(vSJXsk8<;mQ16$KaWmpUj(A3q+2j3B>JqG6E8z@Swug_J}Eym?e)Sxso}K4`pn+PbLFVr{oo`u*{?0f zE;$I1&EmzuCn7i_*64oIBQ^_3%TIbz(iZoHpFNn_W9rbs3E8h)_@>VLo29sIaYGVy zgV+Ko1;qCfPIVt2{OD&!;Z|82XpurlW4@MPaGugdK|ibY_UM<7oEVio`$9wl7VfXV zCKPEuF%YLW`&#SeEWE8;B*d1<`wOe5V_(bUDDnhq-I-I=`1To=bYPf74r4WUMMJB; ze^k=_ga~M=2%qb1_8_~vaf3la6*Mo$id`DFVKbpgz}H$BFewm5l|TLn1N)H)iG0LZ z>+8u-QK0xzpj*A1q=zpneGs892_xDljYI<6Ba= zHY>4L^sSM(g7=Trv0;N~i^$&W*raP<9ye@I@v=3Lhs4n-uyihse?SL*cU{ z1BJP`OeeeND5A)}mM$`85#oxK7 z9snbQLz;H%9dU+6R1WXWMWmEjJvbq1W}-PUTj(WWWgW-_P9x^_>wVD-vw-~W!bOi= zupt)3{M;VZcTH|~RrwKi!18|PfNRIMwacsEF_!F1Y#vDI1)jA5BYw?f^_9hQj7gK~ z#!bn>JL|hVNZDjk` z0kxff&?M9F@kV0-hw#|j*E-v?FewvWq?9L z*=$c(MIa6=o(<{7^Ct2|(C=KijP43QlV|GJH%u(IIE*RMFEptonkD)vUtnlRJIsXG-c&x~n_WW|eO`1qAr4p(USu4=j8g@~FsOVi+^IULEK^ zvvt23WGxG^DkwNYEH@ikW~fO$%MJFV;RlrTbtux1WDO?Y`ZN(ou{64ZUmeT zC|YczB(5YA+ZN%>jFc(98>nRpj!(rFWIeOg8eZN$ zXSOI|*ls*lWZt`vH9QO;#=uVUO8}cYwf=~jyfWBB3%O5f$y$XuE;!Tp2L!SX>-dYE zZivQ#u&e7v@6I2?Z2HPk#{GerKVHkg#WB)Le!+wK(2)Xbt{_%2&qq%my)kv& z|IA4e-zFl=J?2!MDL6*tj`l9eMYf#wuh!KUm7GclGJUj(xVQ?15LT||qX8=xp1(o) ze%aYCRWx(QrNHq+u0x0w)URDV+61=b?~Pm;K~?!m@8=KhY!44hW=8I-@ z&x`;Wo$b6on0s|%4g@^7V9{EL;A%Ggj*4PUPQZ*lE+DAt{DO^8e=O;IzRl)2QP{*7 zzD1ggTHbqcl|h72WC7Xn*-G77sf055>7EpN^!-KI{-62!kwiQ@#uMz*-|06h(J*EF zrJ9ndJJ@O@D(>plR*60E9ImfXQ~Ux$a+JubDg2ohH)7jpNt-H-f|7mLY|)re%pjXK zTFUQ88M&xm09uCnD@E$}c-eRSNrl3Mo(8J9RNCZY9bK?NCu!yat9#uZswJh9>0z?X z$rdSaRFzdimH}03fN8GxGKgINP;p`P#F_6uyT2IhsXwnTu%@@P3awIJN?AY_Uq9%7P^gmpu;6+QA4-Aq)a1gp^Q(M zvg@&rYrYOd>a}_}{Z1}GaVWkMZbJYKJ$`ZVeys1IL56<0y}jC5{2)Es(8j#eLp4&% z6#SfdS>r7-`{}%3Xw(ockM=P~RTtxaUg;&)^)a}_=DdMjx0d-eMo@XlPmL9j$&y!!3_EjQw1ldCIzR0c7`vOn^ml@9*Yt;9q!~Rt3G${^Zr?(ED>0yIS z!_R{lrFERe0#LmzuX>g$_nmb-s?NxiG&@B^{4o?&^IdWD8@g}Xdz#!d%bMpF#!E)J zi!E_5@Fm|D6`k>3m*`JCemvx$tRLLRl1~zjRd|fx+&wgy{<7X>r61o}lF=z8`l0g| zqxLq`6F@2~oW2KYlw>I^7hACtE^jkR)|B3Q)Ag&etDs&qz-d2+-ZWe%jbg9K_0(Eo zOL@D@Jv5ccs{X(gV!xQI{aTX}oV$6}InMP~|AVTT3iVU5iWrhd-`D+v=}+Z$c9x!{ zzyjK^>aNo->m_5}O;o9W;P^Wtrmgv`Was%~a#KlDGgEhqCt&hRZOvyJ;W3S7Af|o! zx4Xp&`iplZRUQw}t)TZL^;SPVKKJ{J8%GJD^DiSe+!*;jfWqG*e=E8x^2Gy zPKuk_fsSKF@QqF##mEr|aQ%*4%q#IV4e+ks9x3hV#%*hbOi{43=MFSn*;_`-eXq*p zw&!Ly{`Q^_QZ<2Fx*DW#${fU6UdzB`_jL@hcZzk{DKRK z({rWrP2K6@OOD2z=AT3a7wSPa#(!|}i3U3FyGZpXK4Rl# z7xTV8`|aKcX)PY8nEFL1MU?Jqh;ts=;M*e52%9n%ntWX=L|SS~TY{PZO4p2(%S<(K}1dqkoDfy>=9RGYQ8#CwufLOrJCDk<{Fx=I1Y|<|#bX)d{d=Chh_Q zcBgjLO;aY!`&8^L>03Tb48siC_>+&ZX=cav7zp?y3nU#z5IyU|JATXUHDfn~f#VBg z@raW5^@yK?wOnsSg!yaWabPWM2mX3C#}rju zY3Q#LRarf*IC^r}W2-%FhXrKW2+hdR4G}F$C)4`Sz*W~=;!BSb!FLbM)Uv-n>Naq7 zv2raQ(6+1_5*(cCd=a3wIlVG<+YZZlGktvtEr>reDb$gQIBS2u&KVWL%;B`Y5)pmOoS= zwgRKSK%Gzw6nQHt>;KWqVMVcsZJs9MSNmb)aeHxH8psaJs`IMUnKj-n?h1Uhd~2L- za44eN!Xx=j8av6vLHyG%zed_uBv_(E?<4{=evQ( z%6n(}oXxd~ygI1C?Wwcr@@}3r{&5*-H3O-D*QD9ccx*^6_o>a7?scra;LVy=F7jw) zFW&v2lvH@{pk4mcccvV4xW+!sqE_>L+DGf)MkBcq0CKVUDmd_*9uCP^YH6FxF+l)X zwSOu4{UhiV%ZG^m$Pz9LE}p2W z;OXN=eBNw9=XWe0V&p>PqCbDnWgBH26KV_)C4`)U+pR>fAGWSn}$>p=@m)s+Nxe)m$rDG4!0p~k&Nx=7rKO9jVv zkF0cET{3EqNyRVs=)cXg|QZK8FpzE22}>;$lOPg!Xm!ws0S#c z_btWtC{g?(?m2mhSjd*@>0oVK6P)4LBRX= z^Or}GQE?Y>i%Mfv+W>bBEv|mmRFzH(Y1KhMwRax7Y_>mql`11G8Wy|lmqgq_yJ!-}kcaiWwWQ7~o{) zPjxnRHWj;7b2d#)mL{&fzxsIbRJLBH|HTrSVad0wZ$~92ICr;-@}h7g)(Ir^<&8Wo z#*-(QXIGV>w!m}ssPzaNK{aFc;=y%^?*8HExPU)A>Mu0Bi--EH*NVM_v!|=g_%r1T z&ciGp1u+q8q1E7SHWq0{$-|9>kY}fN2u!~%B<^z;q1x=WHqIvl8#wB9M_>J&)PK3= zB^EuYb3G@OpsqLMeoTy4>;55Ksc)jVd8E{jAwiKC=cxkcMrA`YG&Oo=K^gQbH$WS; zs@^A@F$>9qxP2a~=6bD9vDpVudNHJzW|;P-1|Bj8rc$;H_Ez=rI9h@eY^O}h{oxq8 zR^(1qw!nRwr_*0BUfk?h!oeA9cyl@ou3|As+%yo*Qhe{)lWW9gl^9_GwMZjTj;Up; z-Y1ciuwt&R&E+J(=aAi-3nqSL8%$WPSiXhgTxjKURL@g&64xBw)KX7X9OF{hwct*I z5xYZEB16ABrV%XnU{}p+qEkU_$?N&8_dnj&zt_oq$2}2o-QZ+k$N3uP+Z}rJ=S;aY34Nj}8HLyY=GpjZIDWHfJqrsk|E=|!Ix3PGea6p42eR_JUFHAM# zO?u|!9)ty$O9XOyFk91KidlDgY}>^I8!dZLI+#==^3^roF}~A^uF7jY@hIu3v9xr* zk$fNFfCY>GWV9J=qGEp34RH8pGmm;C|ADPNxK>!WN*U@?NNHsJrz@S zL{LeM;(a~3$qhnTO(2wBpHm+$soeY8-m*q))# z#qpKCbp2f_PnkdJaQO<*dQP$IcYe!LNVinP{LRemvlsaZ$!O?bqS%*tY*6+-Ev!Pk zK!4wUlBJ#Gtd#|(B%HL%lXo96BfFb9D5~VWSzZiKnEd|1;$m(%F@js1%i${;vdX@~ zp8cW>W!|F3Ebu{)@A$9?sU7xZypLc!%WTH|ZAfcm=~>u*_VwPu;8||fzR7Pj)eXFY z)?ytX6zTd9qELICoPtHAB@SW7kAqoilLlO>Gmm-iZ9ruL-}3Nyo+c`g@*z$dQjC|> zXMDCx1coo#R!{izMaH%li7B{Hj+;|xPD@XMBq={fJ%b=CZp`O82jiG2U^QuzMWjT zwcvF^baLzXE6rQ4G-iIG7TzY_inQM}_dCm+mRU~w3&sglKbdDKq+)7npl1b#!U7Ya z-$xel2N>4pAe;>05(?9Zp3v@Qs7z;#(LlprL4WBpvm=lfN(-k3nx-H8Wo-)HWRqpZ zuroT|ZuIGGSCH!Y$EFp# z&RRSpJF*+EhlSzpny1hvSo7lpd5%}<6vlc3I}fYj1HsoHsR}POMGjsE<><2};z9=g zszgz7-kD_+=_at*5KMhT4Uro;ry7H{wr_eAW%yT{{}*r7z1$ATcPZk*7P956Ah@k?wB zBKe3U4VE`W?%wp#E1sdo^6IdLgvg%lqYqAY=AUXD$ez+0>rqqve5+Z!ok$lRYIk9v1_fc@wu{H(FEG_i z$;q1oX#5rL8vNo;neUzYTri||AQMD{v`;Wcn7Qz~>9Q$WN$P9K0NW+y%lf8;`&c7i z_pdF4^U}e6$4LeWBQ0&H7KWk8hH|!ed;TE~o;O8o3d)7x{&*w9y(T}z8qT55QR+6b zHtaz02S7PblKP>Wc`oCno8y<2XcYx#jFG2?O$${(s@m)pf9=a9(~TJDK&^N*ggBy8gL3xBJ5T+Zoqb2u8~fW$VhX zK7K}fh-pfnLjrxb#*r`SUyDp;EOTEL3~?t}MtHN(p_Z%j0YKPM>0!>XHZuVNYNba)27&#y%|ZG}oPr1vX$w)OQH zKqD8e+w`m2R9^p}$Y4?%$WbFEOCru?imvwumsSn&Px?23y4&({cPP7!J5*JTJe(vT;!$IWQ z&VsM_ffCS!oB@kv_Mp<9{XZX9JgTD^nStpUCs5feCyFGQ7;RwVe#(P9UO8EqP z1kbVb0TV-@bXZD`1N#vy26IL?oZ*F4i@1z z>TiV#z8|nV^abAeinu%OkT8C#r2KA{ z_o?@A z^@xO=-GS6-fk_cE9p@(*v;eNxr=P)3+7;+(ex@06xO1h4mm~0ga`8+&7ARBdfy8Eu zFR19xZ{i-658eqT$_2cK8~a)JY`_>tvm~^RRnI@R2t2?Z?gsYUu;T3WTwP6OnX=`E ziYK;Wr?iDK`Poz)_o}DIdEC@lcclt`ju-UxSzE1l_|k)JLw;UL`g83-+%Yk9cI|;a zR#FhODA$hR-FGO!^I<1=wpUvOSx1GV4Wh#H)X}=0`j9{*9_YU}-W~oC)@S50%K=Q<(1AI%i;bm_W8F+4U z*go%gZu*+15<)1il?~WLW(3?TdML>elf;F&s|UReQ3DdIo0pq$5GOW`HoLYqj$8L( zII%>X-;Qsnac8pb6xeXBURs*MOnZPsj7BNV}@H zo&{bm)o;&+b@?h!xgQEmjhX6Xyu>g8Up&--X#R4|j4kJC6xm zAbdF2*>~1e9M-ZAcWVB?wLqmNOtD02G-oV&cjS!-Z=D)4%)E)=nHDd7uk43%Hhz0( z86&zCT>c~Gwi8&PW7pr?3)?L&?$xW_9^?z`Jo{8-7=eEMaHYS&dbifP557k~A&x_K zEEX-kPO>9iYSrKKU2Pu|^9{=qnXx{sJR%0d*V);ZM9qy-?T>D@>=-@8=GXDaz%Qj{ z@CuuCezMQ`YSOc5d{2t9t8dliU@L`_YvnnkMlKG96;WGPA-NiP;`CXEN}oQ4F(}S6 z6R}2B#Vj_+?c{J0a8#RhA^6HC(OR=)9 zTZ=P$AhHFygyYY4OIG??p9`eJZ^!p!Y9>lJ7Sb|OQxOZT-aIcW*21WjknadTv~jTY zF*{DC6yUV3%IS#Kll-o>+ut`oZkCYXIy+n4mifB0u~OR_aSBkrZ=N`y+>cCWjhcqG zx-nGUrORG=NKY66tXF2vWs}A0osUwe<^woZVSzsDXNz_%ea0elq%qDfJXaMPo6C(# zN~iWCxy6L|B(eb%b!7%s?%AA)a_@br0+cjEbUkp+6((R_28d@gX_51_qOX8a>V$<$ZSex>o_S-#A@H@e0psMJ0_!VS@aTIZyuC1vkVB?| z^|*@*>#wWrAM0e}J8h@!LPZq3OM#w>IZylumX;Ks_sa<{!oh`vF4?QZhq-VTp;?`J zK_1UWmuXg!0(vTDharKYJT{$2s=7|TW-m^@bH?`p3JYSaj`5kRBKwYgcL<~syqcX*&f3D{ z_{@>en>8$B@2N{BYffVc)QCQHoY|B^AmBJ6Z&`Rj?EAQoTfhPTG|6$+ExcjVr{|~| zQlV1b5O8@%AnxGh6L;*#aVql%+bhI9g!xv+v0Q{GKu}VFRu49HbE`N{wpTqb^`yD? zG0!L8L+5v?3Ixy02IP2m2&FW6Vb}p5F1u7aS2Z~JW^mrokHs6Cehupzc0I^1u2A0w z0Fvmbn<(8IHLXn#S{``v`R_#T2WZ3zTS$LV$!h-C*jdV|KOUnUtR<>hyBT&rbgteL zR68e2TlTWSjlGP1G5ATC`T`u5`#U^K;=Xb%#o7_jEaBVt(n{%>kCEX09&7jw_vfVf zbB$;CD0Rt1+#EZ(N#V_k&T zax)9#Rf$&BXPgSaYUx9^PM?WCIJ7!zyot5`{?WhGotNZ{RSMflQ1D{#ysT)jFa_X$ zHIYp|ZtJTy>bIWW7Z>xyfA||>rpzzjAic?B>p*|{Ts2!=YtK+TPcYz&F$yS6aWeuU zkS08h&&6IXl9B-(#J&rrzRSVk(f4 z$R0dlHd#Jtrj}C9LxSrW&ZdE3ZPL*U@4%Z&Jq6+zJyFU^M*}f2F-KXqCj8d3(ruZK zVa9?|6?}ob(A9`AEAz`c5&`Ysre@z?#TwU$#Dr0cob}cj`I|CPo>a(CbJ|ATlwR9u2f6??@t6_v zl0*Clp>Y8S^*l`&)pb`&+$No_l4?kPQRJ9NrP!MbUpQ7cK)&z8&~@kH*QEI+S9|?a zA3uV{WHGXcK2&(4`hZuz^-ko=h}OgK*uX;g{S`U}&(U}yJ-JQ7Y-o0xDnuS);s@>CS$i0qEYNM~I$GsWTQq}eq4dwIhTiR9Z9<3NjAFib56mii~ zAmPm3!Qr!LM#KR$R!(LOl@*wuf5REzh3G;~;~DBDAM6F*^=L0IOr&FD(~5pM9BsV; zuhEEj?=IGnK91BSZOH43rSkS@|2CFwI1>ij+y`;S-kmuP{n1adV%al>7~IU3^_J?L z^pw#^{7^vgYQ8GS)6+9UNph?rCMoP^MVR)Pnb#V&XqOGy3F8$=Pa6vxbg0zB!~G z{zNTYwf9rYc6CaxYbT~Kaj}9E7Cprxpb$EX7g#Gng-Zse3vr&Sk@9X)DdE>ocGp;3 zj|+Ldo2A%4O350x?rgwD^7xA+6H%kKZ+Z`UNHME$NC&Tu@fjw?+Ui;Lo#uZmZoRL5 zb%8T$%dY^jv()can^o*C^KWIKq~&Kt6ACnCW00QtkH*3|ji?|KOeMtN6N-RWc&yD9QQKWJSC+-3%4h1bBr%eTRYU5t;$>^ zmi*w2oHe0s+{;eeA5-#>q_nY?QbwW^F)s&;Ic4wN5um0;=dQi6l(z*{Dk3HZ0spG# z;US%eoVS9gazM*c!0Csv>_m&C-`>4$fLa7CpGt(CCj)E9BR+{sJi)EdWiKroP69gU zlNlLK^RCVmSk^kQXM96kdP@6aO6wfL_U3>UP}KKSgYm-sY{0LLZ^{j6#)d`Y_s}Cn zwwG!&5sX?@8Jr~hnJNIMUZ({JshS=F;VRyyXItYSefo@Uz@8!To9ix0->D4_tbqg3 zfaOt8u3ECTwf403K2;plrafp-K^|9&EW`%i@J`DF63#j&6D(>O-<-I-2%seGOp@=}N66%jjZR6Hsu~&TKsCwL zkUs8l1vvAyCCaj6cL|DtAYl?-G?^J48*Wx{fB+$^=>|#nrR19`R1FPvY;U?2XBFw} zv6=OfT#L)~iumShOLwS^-+haDjN= z^=0L;`n8BBgj+%zcrEV$(bQjTa{{rE1d^w`z$Ev2LF)Qs=OJ$Wh#r$%|6mW9fDyyK zHH1F5Cqz#L%psX|e$0~Q3Jz_?IDcl;TInRs?Ywy<&4&!JCCGnO)c6TWO4Me4tl<`S zu|xNu5ues<4Y&`+{#E{3d2XpUGQk&i_38;amGkezOG*rm&_&O{{=Md)mHaPB!{9G8 zAwe`?;pabc(AbZ7G1~C}s3)2x#E&sfET{v8o;=3L5=P&ut*XkJ~vw6;$$kVturJq$c<?43_y}2{s zeeDU}?9o1k=1}A)<<~kc!)d%O6CwLqTrW+&|M1;OdPtTO+_0CYmAM)OzxTzACrIg`XIk;?W%_k<3yxE*z90q zs`pij5?P5^xoZc@vwhMG>GcGqpO}>Poo$5*VbcFv55xH}{VzKaw)`$J%h2euz>s^A{T)`%GvA2h2(ctnOC==$BH&MYCH%j_~oB%JC$ z8RaUSpRFeYsq+F;<0zwmRRp03UA9Ekgl&oyI(GJcXTfj4oh=77F^`3mco=N8UwE)H zT29!y-Pk-ozV`zsE+Xf$+xXM4C1}qzKKI$P0XPIc>NS>$fUmAF>He->J7F}lAcnRM z>}(FTHo#jC9m*1ZRgO(qzHURf)vI56!S~OmIL+AyTa>m_BN(yw-d^hKi{w}ii{?zw4 zQ%Ist_29OVw04?di$VN$^kLU&HvJc@!J z8`a_pIBx6*FW<(Ewi-$B1=Q4&E8!FPUyf@{fK#;XA~|0XVEH4D{wzR~_TjKF@}YN+ z!?cr>eXDf9e0uB(^04)&9?{G&^XGjufF7czR@Kt90?veJGI8`k`^u`uV8Nb6Jdlp$ z)COo>r8+-TRHvUzADX)p_9~iTE#M?f zu7p|jacon77kn+IPcFaIPqT)+Y!u)Tbr*H~?MVvP7*}y$?eFcp$Onjf?XP93^*#53 zfeCj5k51~018oEO^-`NqtIGjrtHrF`>vt|K)el$5<%PCD!ON8)K`FOK{`0W}v7R0g zxH0PcJ)la%S9>-kVv}xVd{KL@>$k@DDG!#S4>b2z@RrAhk-6BajsBOz z58lVx0}>A%ZF7t1)pMIo+5s1N>V*3|4o+;D1C(N~Tit4nBCF#9AC_C7T25_um&O1k zab(%Uo!#R6ckr5fcW+OW#4Vh8R6(}w0ipfc$a;4V!E+qw?BQQiYNyr3>mwy%bf0&8 zwEGH@NOOBQdp^zZqes6->V9P?@MiI`2nzdOLI;2eZQf_uvT^U}8j(kvv!Mr0WMmL*1PajqZ16`f1%j$&uujiVyIUl&w@LSw1WRd-L zefSy*cIv3hfNeX*+P?6nM{&*3TOZ8sx?Z7WDP=wrp52$?A3qpRRHGr*mJM!PuSXZM zon>zx-#dI9`*U%oSEWTlcioe{OdqP5XlGtmK($?F4nxy|UNQVf7r<^7t=Q?)1Y&TC z29(AQ&xqxvsf4_Yr5B+Hl@1(o%^0QL(#iBJuHUN(DHyt0m(m?+2 z3PN_Qi79{p_p=Se!G3rXmT};zimgM-?HP|#v%BBBrR>Y|f8kJGX0Lu>3fOeWIc_V+ z5+sXLKggxQ{hM6JuB90)FNe%;^u(U(ON}X<9R)HPiFD+u8v1 z9bVxlX6A`0V9#ROzd$FAv$G9b8@!xl+rUgx zOyH9+!bS0HXuv%XyoHciPyGem&9pMLy*y~xPms707!`N32eG|%J-sxr<@TsqBiL^Q z&!jnCnXmrL3AprT+=Fd{pm6o+s_&ywUn8;mRZho)08U`|D#eU+c zSW@Jt)_zutM}X%@h`CkqkCL+2SG1Y5y`%=vIdwEuEm>9OAc^{b;#F0;1b0UZclYVm zyI}tYjhPcKgrwDU&Xr2jRC%d z?WLw9HPIVqG=5TzDVCZ*#@A<>1)c<#4y+t1^u7JqWK2GbR+LJQ(W@G(wTHlgsRg^Q zo@Mj>JCc&3N74vcvKEoJty+n5z0&GRE*=R!5s5lM@6FQl!hk7{o$UK6LRnDkr;yqY zf!k-HJl4k}+WNJ9>n`AQ*6XCs_E$;?Qhp!A!sv)L1p+Luuh)sBvGwI-h2!o+HA3iq za7v6b+uJK=v_h`irjO6P(__NcLwsC!9d0HIL5}Q!tn-B|S!Dr?k@XMJN)t;Z!bX9@ zbnc^f$lQ0)CUocfW)*ZHbZiP>aMH}8+b?IYt<1RlwYFHsM%RX%RYPCTV+u++Y5GRG zBZ4z6Y6H=KdT1~1|FnHM=FlIJU|(Lb)C;V~V+P2}$D<&YO@#jT{pBHGIdw zI^vu$5S?RlAa-;3<;eYywsBE(b^!9nVr38MdT# z*xLzsGqrQCZS3!9&6`_9e)hJktrLq7ZOGihnh-={2!t^MiMpid}p5Q;Kl_ z99kk~4Q84v#?NSneTCu>eJD}nOFgbE@`%D4luvlwv4LSmliPad^6iNO0KnGdWUOL? zUef2r|2u2X@_J&^{#2_%)Z@Oij^sk1)!%(tvdtK|*tD4Cp5`H2oureUvs+uEYs;HC zv)Yl>QQwCin`8W3&UzOXzDC=Di6YT0>UT9&pG(M}fIGWE-#lojdq^`OaQW(?9{P)l?b%Hp*IoL@ZLo}w5zm50kn zhY&Y1%o2fjTi3=JqJfscUlKy&$AE&zn5t<;hQ>=a{34}mSCflzBf~$oA`2Gn`Rr|R z4biO0dLx$PJ-qXTru*l#zs>6eC~Ed(^0j7*Ci(3UdUw(E$8IEQzZQ^$Sm` zK3zS%$R4v*XB}0t`)`b2Ep;If$f=ho*^7Y7ii@2v&a4X-yNL;q73owmsdd5eQfj>m zVl8HVXy}v_kl}S7YG@b{lw+Tq&O-=FmPgz!EG%BqE8Q*&`WLhJUwW#E(0u4wuyTByLuE)0JXHNonWLPqlqK$?sDS$ z#srRx>g!clMwFMEML$^ATn6mJNI~|aKG9gUw1pETMmOo|G+N+1*=0ZT`aVdJR09AT zc8r}Da94QDaY3^`^xJnmtii`EpjEWZ$f0Z$rZ}ZAqyXl!zC*Pl>^ZXjh|vNZwiYtg z%q=Y~9Xx$159dp2U-0cy2wy0+aqc>>%G+MvWFoZkHH1M$pMv!1zzV#V$Y z5ABMS1(wjV5k-JeW|@sqjh)4OD^=I$JoO^`DfSi!W5UpHJY(M1v5fwWHL8$F0ZTgZ zvrb%2(R0PXx4*@zp9&>w>h}wLY?2{35Upq5^4;JF1Rp&Krlx<1g9idxx~A&PoIE+u z%PTi1so6{L@A6+1k!@0MaL9W+Cdss+kS}!~Uzf#Mmc+_keYj&&R5fNajb^!g^p{}* zKA|(EH*p|}VpT||5}O3sfQ*kN16ii4*r(Hn-@lIdBGAYQ&`Zry{gGI8U z%)VbLDk^evaugZ5`qa>j?4ioT+G212#_a%#tlNve2h=tVV`AV1=fj1{TVj(p|M*UB zWg>glGDSc#va*B`-4j>$4bN$V`i41R(CEzD+4?dhV%lgXLD_3AMn9#Hlf%)fmgbGx z3mOINzX#|Q zy_J0Iwa_WHYrT9MthF&-GrwMVy!0cCv(>!Mjxh_3C@f_io?iOLWdrYREab-b)^0-h4H`z@CA6YJgx?V)dmaqFC%6;n}U3OF{M?n z6a#F>YzU6u>i)MV#-Mf9Qedb0n_sx0k17ZNX#RK_%&KsLnIsA+p6JK@k3~*&b{dUX zlI9LCC+2@;S7MYh$qQ1V<>WS9Q%7mCGUD=0$u?aZRG7;DWdNR#eP;GY&w3?F-Iyi( z*U{>Ru9V>b>Z}sH&#F(%v;Vp2$|Gr2cf&w=tiO2vFQ%fMm?i3;KY`{1JwyAc{yEx- z|F4@X2(5AZ-xAvY&)Wq>oAKX||MqtMb6)=yP#JeEqYu@+YX2$=Ck_#{!QC literal 0 HcmV?d00001 diff --git a/docs/img/AllJobsPageDetail3.png b/docs/img/AllJobsPageDetail3.png new file mode 100644 index 0000000000000000000000000000000000000000..75b7caec119b21a34e2df555a5107347d367709f GIT binary patch literal 94804 zcmeFY^;=xa5;h7UfnXs>aCi6M7Tig2AKcw_aJS&jgkZrn=-_Sx1b26L2L0H3pMB1E zf4G0az5P5h&#cwGR(DrbcUQe{O~hA4DO4nUBp4VNR2gY;Wf+)utS~UJXYb)(@5B#@ z^uAt%Sc-{#l@Svob#Ssbv$QdVfuW7}jTe;e79s2fsPrn+6CIIme9z_6`c6--s2xJC z`u?|Q=?Cgm;FrWu%HG~@4op0E)tasjjx*DpY9HSUi|FcRW9jAhycC3Bvk?>CZZ5a> zFY}97sHT}+N5f^*GL_G&sfD9oq@l?chOqx8yL z*cj6B&nX>Mr(K3{YM9h{?Hy<6KhLz$_!-hxRx&zg@31Cl_FzzkfBhOjHwQWVM9cdEHx+@99_&RTRzS<&sOLNG*5qc(k_VSHOeYEO7p`(q~s zU?0ail2^_(2s*B4_OD~H1>bZncJ#VRsoeMv6B(Qhepi@Je&h@hh4m25vmb(mYY=fL zpVsjvDMAwoZI)ezK?v)xh_9IyTp`*!IbP*Beg?wyXFCltsSvKe>IKn3TFV&*hMD58 z53EOlu-ofR1Q!_v353n}NFU!3!RDyEDh>wEMMBd>%-+`4)XoJ)%*oWy#ngn<-O|N^ zR7ytSt46>F92gi<7#Z;|svZkR%kD0!y{rA_w}pwKUZhw01r)IPGL?VCvBbsE{ruk# zh<)3aC@6i;CDIv*^pjLN`KNNyhrp|kq8NX?YmSBdZ1za?-T?J?9X`x2>$sVnEb*^q z8?mEwi-`YQC~=&WF~a{X(sl(fh5ufJZ)qbLME-jvUGe`{{l80t`Suy>$mS8qt#tAm zW=%E!{exEsh690jG%rT)qXW$X*QgX#2WgZR$yN$@ zauBk2^WzO43+&8<=N$#kci6Zxe<=4`l~o+M-9qM5Mi^>8oU5@13n|BDEBF#snipJj zqV=b_&=SCzYu}bYq$CI(&&u}R6F;4y#D_94<&=)}OX#|UMEpm61GY`Kf~>a1KF&){ z&q}kC=Zr(FcAoC=T8!H6kxiMK^L{{XYM<8S!NW~aM2{!`RoSIwuQ`63f6^nXhHqRUvG=gy|1jXWGp*0~dG ztzb~)2eZ|SXZb;yM=a-g>&D5;{gTHgS=5^b#Jk(}eEauufa1PbvSeJHfusjH3H3KB zZYD68$I18PAaBfwGIZI_VPU&bgZ*X|dHy6xj|5a2H1464;eG-pFkgM!=^7$gJW)*3R7ee*jPqHmbJ)b*^72>`eE>^{RBq+!*T4| z^fBU5)b9kt&+F@lGRNAv^5p~`rVIL%+XjS(HJzh_iNI;WvGyEpKT>-add0CLj;Z-n z*+0`mmEx$h-|7v~xU1A^J1h~uu*yaJ1 z+dmu{PhfJszKgKm@;Mz-<|EdDqW|9 zC5Z4K$<3lt*ipC&C&)ePofJoScXD8^-JO|ft*`rAC7??%$%w{>;<#nUq#Od}!83=x zQLFC`eDUWXB4byGxv?9`b~Op4MsTQuW-2o*%6ly{2f|_<`^UrbP5P_=0Ym#^wmkNC zecJCg6}+7)w+1`c%o1ZTyN5mm8ro`CZuz$2F~r7X3|L@=d)$3fnmc% z&K47qh~RE18imrbUt~j9c_V&B9k7DE&*bf#5YM#BmLjKeW^x=SX9Vei7_vQ_@#f0vmCp}JT4jB z*Ww18&laa&!`M?=guz^sS~OBD6Erh_S@9sZ&LUW_H9rJORfL7-jp1jc9m0a)w|Z{zz@hT@8tuii7g%M#4$V(S>)lIg!ViH*r_ z-T%sk#^E<+GTTYXZoSq1Gj%tHy_c$roLD~Xmu>?L+HP=?U|`@LfIjf$$(H6=oP6;- zZ$@Gfs)1BcT9oH*G9U)EXrzy)vV?zlRty;edNiiN1SR>Jr+u`^bNxnQg~dD)iG%!d zi8x=wo3O(Jdh>3{exDzY%#+qeq;YU)kdhy3yi4Z4_F!$t>p$Nj%i8fdDD8z1?1Dn_ za+5}(nbr_3ob04nq$WWkNhsHM?D9s~4C_Z_5h7d^#Ygx#upq4~kVO%TVMnQ2hsO_- zxUOukH#p6OC%wll(87Z{zT9%n&SShBY<$wchlQ-(VU>Sze`gB@&I5fgIe3wHcm0qt zYI)&!*2Nu#XXR=N8$kr$L(Gk^&6x=z^;oYqz8CYGw)%5QkJTqcATwDbeB*p%Tpy$x zfeMB2`VX?8#U%~*pGiW0dB)7;(@jf8qRR7p8{fo+i-S}Id-%uOe$2Mo$x-@E|L;6+ z%NHXV^%%lETi2-kw;dQWog^b@Yy`#kX5?16=Q*Gk2GqXR9vf+}mkI-jcn1iMbgI7`4#e@ZZn5ri>PBio+TbX^2D9>}P6-de!yTL7gH}pNkt6Gj zthzFFyPe?R3}$-S9iX}tK~Eu8x)|}yq3Jp&F_htZIkNn}?ugx(t$R$1K)5GG=Wx_+ z1{fPIh@u4}qHJB}i!SZK?proD0gPH6B;M}gW5NZJQKi8(;=O*kZ2|j(B~*a$!WOc* zeRPPcR3a_m$@|1m^;_Z9VGh3^1k5JuMvW6qYKUG<8Fm({gHlR=99;#>F|zg-ijX!% z<#FevAx>}K*9vA41t|HsBf*4KJChH-k$||y)H$)wp!oL+qfxQ&+nA_Qt3vQ(?T0rU zAPH;0m5SZ4+7^U|$%YtVU)ZvFn|bGi)D#+Yg6xid^T@N4=}#?ow^Z0c!~ ziC(IIh2L-*E3Y{BYGjyvNsXOr+XdbIdL>C4cWJCYneQ9Qt8tk9y;VgaR$I&h+B4On zO>|W8TKz-;f(aXVP?finGm>Y9UXoQIy?V4xohi=_E zPOAOyKst(5_)8LJ(3P}VXS=&b4oF?uiFfZCp50Kl(^~V3!8ZB(P|t)0WpsG^#j#8_ z>>0oMRI6O(G~-&Csz5r@@O?D?z*xJLin}-Gg^Xsvny_tX;N9B(59=Jc_@OUJx}ez4 zT$zdfDRD7_JqawI&pzN ze!sq!-`!4c@f;@knKxnrYv&1vrvgFEl23AOaG;;>3IiP(nxdnLq6Q7zp(rkPQ2D{I z=vo^RnWG_j2gTcbbmSr!Kop_w9d}!?j7{q>sgD0JS9N}=(L#HOR6$Kb{efN#0W%F^ zhGF;yr#poF3#57I-=(vavK+LXTW&MAi7RBcb-lY!uF>}u72wy5f!Hq|Ch4y!BR z37tn8DO(qFB;j}bHc#o@s8VWWhf&!5$^v{L>}YjBR~H6kR@Cij1}G7q7X`+3&SP5I2&aF^szvy0jlKP# z-&@hN_Z?BG>w^CSE@pfew5oJkR68R%NtIzx3x3AAC~4b+iDOXK3}S7rROE=?A&90B zr0{8&xXdOfvP}ApMdCk-*{Y}Yr_*5NePAn- zB-Ztx)FIb$?e=vkx6$Uzw1aR9Iq<*~JtS3<3G>M0)Y9&)}pVpL#h3`BcN%?Rn zn`Qrdo1N_i0Mn>c?0RaIo z_q>#pltvAU@87?7psaiIA8SD?8yJutG2_V1&24OKe56?n*j?Co?Ram`;#C}UnpHpk zo+|0s2O>W4=f~VTD0JuEIW{hihKno3X?rNU9imITg(9n@MC14Tc)NACvZ608CAE2a zTDey6w`Vy3$IQkSANUS=eP`zebkbvl3$LiCcz1t~j^}wFfMiD(NAf5D{C>$uK>-I? z1>g4SrnR-Ty)GLA+&<{xQNRZSZN?PundJG=QPim3E;zp4U;lWly040f0(<&CUs9Yw zeBDV%P7&;4rMJ(6K>cVX&eg>aRFd0UTfxhA?OQ&2NiqLf9Sus8VS;!2KLphImc~wY z72Onvg#KeIO^H}b%EVDiG6w;4|E{GAe-$+Of8$HP06}M_AgkZT7hhU1C~){#b2$_q zm}oRcW19?76!4?Xhy4WXA?4CQ;m4LGrrVA>$Fy61p<#1#n_U3MdZb@O2`3Q91DAo_ zCpKIDk8on5RtE#o^U_*sImH^bxY~~mdZm39 zHu*TyMBhdqbE`NdX{PX$^t&YvBipWTl!HWKcsg%!9JY=*UTQya_!AJw$Wary0gIbo zbNB!_x0sAXvb#OYx4PLlcGsi5AQ{ViXhPVrMJ3~2m!0o7=XjbIg)z~=KC_c2ALVk87wadYdTv0Hjaa_u|?e2G#?H?n4t!}U3g}|=( z08;?zHKo25Ql7(mD(#MPUMPA4C#wB7 z>h)|T>b~&LX^y+>lTI0OGD~(6K3G{6*td9qd25%r9Mzi_*N8V!zP) z0ePcOm@>b%1AQ}G4!ZL}vCDtP=LV+j*?jKRvyh?dB^$tFQi&0uTQ7G0G=rW;qX(;y zkkDJ4a=Ed=M7)_7bf2+LY%K&g;oH%^^`Eb;r}HfPJAEEBb_z^eoEfC%ZRgViCb`jJ zanV5&wfD%35za>gW{X1u@V_mEtq!5Kn>g*#lf(c3?IAaY9BKNhd9=Y%i9FIkKSRw6q1d<_99h^`D5{=>t{u->9Jgg^Vft-N`KE>K zhJG(1u*O?)q9l;_yKoz(27`IvgH)gO1tRt1vRex8Ieb0eYB@~KHhiIfXd{%oRli0= zM$Q99*@5c~xCb@(i=#rnFV?8DHhL))+Uk#3v$*Y^t1DE{3a9Saa)qTRIb=CQ97AJ65ePu^_=30LFW@p6}R zgSkE4c#8sz)|*Fdv~QS+Y*)_?&Op-eS+t^)LSqv(8`)9U88zi^)$Qa``?k;7TrNNU zwROh(g}INq)v@>?+&?Yn-lU<|Cb7Z!bzC-~Y1MjiSYuyx?WQ_LKObPh7nkJ-BojTe z)L0y=*H^q`JyW@{U&nwdhy@yhP!dDL0H;45)5bKLFP!f~ZH#yPtB>q%u>p^b37$&n zn!=lMv2W}`2~zUa%6P_8c!z;oMcb3_qcZ!a1|KSy9Ip*WPIc13(ED@fOy! zv4<;=kI9VLaB}B2BcWzO>t@*bGk{z8)~I<{49w|EsK;>ktLg>L`|G#JJE)u$kE56H)V56jz3&nB69Q*{v+j zY-&@G5;p&HB(6Q838}YT)MVde`y-s$@pz~UqV7g*S#;zAra&fZ+twD7TmcLK$ zC5;mo@1v>60z$flY*+=tOG~dU`#$7lS*NRq*AdHWXW((4iYzU4ZmIi+R{D(&@!Fo= zFU3=LCqG{EPjJo!W9F*TV+LoM`|?dY5jI&&|MBRRa;|7klvDTAZq-Cy%_5ysIhJ*B z>sOD}=N8 zY>iGnDhn6GK=G*`S%UIZ#s1)+Ee%Zgf-g*iw$j)6`pi~Gc(>S#hgIUF3cZa&$YR8s zDO;6nn2Hf&{%?f5w$y$F?2Ujcs1emt1e716=q@q12;pl*(yc^w#2+kF>9wBfI)q=b zk>-osI*1mlpc~#<)vwvR``UB?!q_jam$UA&^!ttLiQFtOou5Ek*}tRkbUXJ$58`Fx zgcaX$v#yK9NXdd&gf(cVqM*XS&O-&H%>e!{pBBceG?C1oo88)_+7I;hzjFYr+h&N0 zH^8Z@Ze|IHwR~$zYQQqg5oo_GsEcI2?*Sa4}YMzhYkJT>NYV| zud9rHYgf+dT zIOmR7LdXb0dy-F3UYuX^(Ns!g2Oly<#k*G-=1 zBZMl`7QKKG-G@}^LJvW)?US)m=$-b_B{+S*q8K>y-m;mao~duM{OyUpD7d!>V!P5t zCQ&3>;&4F|Hk(^;$a6$8uDt`dE3XQU>uaV;ru{>iw8z0S?Ho86=4T=L(I<8lAF)Kv znsAnuSWP>dAl{0^&_NuR++$o^Tz^5jEX_4g3Q7&8f=7P#kCb=JZw22|a!v*1bHtl` zQ$U_wq4H!9>Yh4n#_+Qc_9MK@&Tg+X8Ir4YV7kSyA3Kp{gXxa)>BMpB7#)3U#<7!C zW6h$qCYlcOW(cR_=II-`W07(zJEd=Z9ByRQOPr>xtCiw42cY^{gqle1Xv;v-GpBUV z1)*}lhv%fWaft$x>IMkYVvO6cIzY%LqWBB zue^|cTil*YKh4Ow7S=Eqz<9Uft$O2E`!zxliyxT$8_KK>)K|f_0v{@FMdT&)^k234GZEazT-y+0nJuVY3&Dy&(F3LrCb%JD{D&7 znrluWAHky(FT;#zYnE>|Z9%>oBL@i1pd@?uz@h0MYEAfc2Pj(TQMivi-XmJefpiK) zAT~SpK${x=pQ8F|R^RTfvyWJDjQa!LsHX1?i=lUptUhuh(zWpDln|?(c8M2YCkwLb z77LvbtemF2L^d3WFHcou(aJ8SuTGwAeE>fA!dU||6x-2K&Mvr5C#u)3F&`v{m993b z!8c9Wz*_7JaZkWe@TKRh`J4yGwPJk$zX2Z~$k2F3#c7JL&3V0 zB$i`JR+ku8*lm`8-#716+DQ8ywysU)$d8$u;c_!uN&kd??zMv0+RRuF#ZRdpek;aM za1~h_yUEgP*?md!t>a*c^99+jTFC#Yp*jNFaT=h$`rJE3m&afiWffEbU*{bUXKusM ztw#q#6*GZix`q|_I&%q3=Wm(pLhQseuiVBJq{YjaSJ)JZ2i^T_B*Y)#T%2UAOSmr0 z3$Q}vsAI-)F$${COzQc4CQScH`rljtF)UQp_#Ce%swC0j5F@_zJyUN;dXPEmHj5-# zM@}Wa8WN$=;?OGZ9w3H(g<7;!J(z-ju4?=JyMt+Po(fsoS?}1exR>NTRV58o!REns zSZX_oej!Q%CNuLcUJo_?-MzOktLLFOP!`9X+G=)Ct`JT-CZ@F@q#8w8Mq#k4NE21t zji76aWu-_X_lZ%9$WdyH-{-+Fi@|j*!r2+iJhYaD?}4Pg>ZAMPh(z*N6!Sm2XJ|um z3!M}_-m`gfif4aFbGpMeA%0hGhuepR^P>=nedX|R7>1&oP>|o~ymC+gGS1n&IjGt} zQbTqjn>C&JZ8U1j(@$Q`7#*RTuE)TvT`mdiE;MEv7ng zZD|V5-(Kp2wwo1Ub#=9PpSdxQmULVc)hom?Ri?AFZd>0US$85bf2ZbiRlNCB_KRs` z?~+n%mDhi?K6x;kkJX{m&ZyQIf4zmG`t6qcTlhRKcJCu|8>QA7bStzET}(!(m;J!* zn=>(+Cl}cELaZW1+x`SPC_}Ije_QghnUw#=-t21a3zt{7)+MoVZ<%Wr1fIeW6T~c} z=&;PEK_qmtvmX2TftF@NWq|s76^&IQ*pPXZP`-~e^VhVqrUzw`dN#wajurACn;Rxz zl%+6H%fZ8!PQ_rBOh~xn>Scd~G^DWCk@dueRSrE`xTCu#qAa4j07LOjPsS@v!Xj0U_6`1 ziR3FO%;tciTM<6)ewfz%j=T7M=0JFK`wo{3KX&z*fs-OpeR%84&{T0I!t4*2i~Psl zSj=?Xp4!XA^s=F|;KsK`>-7hw6)Lw~qX_5w+}5Xizg&5F zzh{a(GIm>>uyL~-TAS7R{&f9faVsqs>Vw{aIi}Y4c~>BtPV9wu)3fLCQ)}i-pKD{^Q0RqJ?# zS!cVLq9~T?IJ2#^ax$LcX3Nj+qG~W zYt_|+j0>P|LG=29U(coGtQu`iOm_9bEyIcim$T^;!ypoUOSzzXh5aWWrz?%ZdJ_`& zYLcRoJUQDw<6wjtiZuq?pbRBdVf+(_;n=Kew$kXha8kpsMHR5)bB9r|fz5FbrLmmN zM^zH!kFBH_9`{@4Y+mUYH&AJ-kO#I<;a%$g)OxzFJAPu4Oy$OXQH{R@dHw=QU?@qb zJg&>e+GfdqYZc*Squc(a*SVk4rtu2gMhr%nXm%u|i_0rx$RN)96R2s=^tvu;_|$w3 z?8&b^dh&riR4Mrbc7#z(DpZRSeQdomp2?P`WV>NyFbB%1o@3Y1Zn*$<>suXg%S#HY z^8KXlfxbx{JCz*1U(s*mJtEDT7|;X2f%0sk3Pi_0tiEGw>PZm%ga&+U)*K&Ri^9lCTgV6wqlzz@sT z8Sq&_y4j|FL3RV;EZ^9rlN`UC7iZCzC|DoU{gGftrEgnD{a2?if{vbct@=BBt}On` zc1u&jnWKu;#4M9STWBlC+!$hd3& zjjqNot=ZM}J0+tUTd*%c0a_aCW5O@_Ue^aayn1*JHPxwT-ttD9rq)0n9og~gCvFSS zX{R;I!m*p%$&$=y#%GeW(<>!U8*!gLhDyfW5mWo7J5J-JrIn;E`K?gTk>?w3#a?_5 z>=MO(SYHdZrD}tiY|ZCKlMl#T#AJ{S(SXQfj|zJldT3@+|E)^XSUQQb+=GNh_U@(r z)bOs6NsIewp?E|BcWY~HEoQ}z!_3%IlD*$*cXh30T9)rQgH}$Mj$5uy=D}P1*r8GB z=j2;z;7DlDd79o6tzJ0;%ZF2OE%#vs7E+hTyH~8Q+)Jo%$4QA}ramfy3Bf79SJe#V zeK(CGDv7Ua_@iyua&2aTQo#*aQwAsd??_W%4fR-ej@bJvuSz-)#Pk!Cun}NNzZ8_d zdUF@1)cCFH8(_s^>bB&R4_&@o-}u+GoMo{W^kjNC)bUb{_AZEM-Xl z5%Ac_Qw~~>K30=0Q2&S2I%Y#hpB{GqrQbIFWsWQJQ|72w@7~7nt$DR(m3Dc9FSyjP zX*PXRt(z_#fDQ!Ps3r&#`PV-v|2@7{ZkBG@FpLD^hRGyuLVw$FethX!)#C z5EYsFPwVjTfuLhzmLO-HtR&xmL3#27o&SsA8h|?io&-Qe5Lor}^w=^-8=IP*4*R)Y ze>68A9Q=(yp+nKm&dx|VI51gQSg7FtqHDs!!v1D92A(iF>K2b*&5Y0DF@RX~WPe}A z%IZLn*G*fSFDECbneU&;KEpjeJ;_K*H&%M{EYpP=eospaZ*On^`yJx1w9UZCNDphY z?#roF=Q#qGza@M6*~JAp2S;LXu=t4CWxJDZ?Yz9GC^;V9zK5r$UUd)np9uv@~gT^%*~QQ~?2jv)fx59-g}#`Q1VRb>iUy=Gb@%2?=y!VnuCj ze6CC;jhxELN|plIkE#%*i!@MqwctJr`+Gm|Mja0=kG>(423rx@T@G5AKuw#|5f;gEBW4h|x6b929H z@ciluyw&=e==7>uMJp?M&x@_jSNUjYXcSzv@6A;kCoyU%Ba)Gq7>m2u(vpaa&o+mL zFWuU35u~T5zw&bEul!$X>MZout_i{HmBg#4sCcEa=tKTxZpF7!c=`D^kB?DBjEpv@ zU`KRv;5c6G?v-9FsQA~U!h0~7DE@Cs4!L`9V&dc{>*FOS zm`S^a&8>Wik^jB_lwbTOKFyg9y;P+Be@qW2a%XGn zDlweKAfsEV8D@Gd-zcl_L+lC(f8Y84aQ8FvNxNh^785_Iy36)~p7b-$w}H~$a+qnb zT%`Vzc)-!$Xu7{W83KcWtOI7VJD{HC-MV9bsh&BIftIm6Rr_lz@IIv*yn3!B z#P59EA%7hpdnQBDI~~9f4k=DE4Mv8qK;Tft4;dE*>DQ zTlxo<_uN{Y;jSQd_f6wy>pjoz#ljPvuCln4DyRq#;=b9Ez~#4mS;6LZ0sk4Qi@l%x zsjK~Q7(v3xBa_7>^@x|hFS=S$6JOy9h-*dEFlbVuVz=TQtK+azNa$||MZVsX_N(ES z_R&FX(e%&)#2(SE;`Tf1VWSUE^*N}j1Pl5xcNnkA_U3;mNo|!meNE8q#f{^UL;iP)uIg0)Eu?&t1i$ za9f+0fejMSFY%M^hsrNp{J{B;Ue)JE>o1@A&Ye)FziE`_G5fVOlH^z7cVFfK$taW% z4+Iv{-<((yF5e*QXtedlTiCbV|Csl?mwH5=73Ex~QD*zZppedLpOH_M@J>_d*ic6t z-!7!&%;B3ojDXcA@V@!Crd#>Duf>o&0gP-!k&(_--VB59mAZPe?=o1|6?+q!X!cAf z%eN0dS2u1QO5d6RVe6L8N*ewy@_k9zWBNw-w zx6ciP^OMhj)P!vM$Z|q-oj#Rg2mmpLTzq+&sqdEXJBXCgDg ziB`9O%@;r*yO|{*&3hrpLCs_^S#80{OeQc@g(C(ymPT}+6TWuz^2-!yoe(>RZYcmq zIl|w8c1_KW<87R+kMAVRhH*1c9`SC?D#vG?KReG7QBM$eb+<~gHJhu&D)>aV4I6df zskiE;GEGxoToPhZ9O&1fbw~@3Bn)yu?`NOH^9k;7%0ya;gK+;;zaiG;Nrm4u`|3T- zDfG;d<nfkhLVdO_geI!2Qtg=G zzIbIOy9Kj>f4h*#|r@N!D9IQSzrZ0&T?H|V3v_po~)fD?F2azM^!*m+aq570iw% z*jJkeDgss^>dYX8P7(D(*jZT10rZzeOuHh1Um~>Oa4!otpPxymzVXxCG;TnRMiEAt zJu8}cd`s`soN}0GF7C5X#q&MB)vZUlpy~F^*(<8NTu;mUj$n3UKr#j=p?eK_;`z-O z+F;^kfNwJMr;DX==jp;PF?jX8@`>jJ=~Ej+Ilu?tqC)dvauuv|+IeGp-UkJXt`S7x z@7PAv*be$PND9=?lIw0HXUhXt{>He5tJ4uv1nu#bOj*^Ky@l?p8=J}eyeS4Bzdjl^ zUlDw%%rgIZ*y`81+doAFINvF&ocCgckJ9*3!c8!6D2D$rq4snXvyWqQ_ldGO^y_hm zIjt8o?@9RRMKb3c^#T1RWbqt{L7|+h)pES=-0O~J?4kz&VGwX_Y8LIa7rH>jUwOgk z)mlUT(&eg1iyEfzlPq?>U;WB%EZBp)-m9b-6N>ou$ZnMz0x=@+8St zX9+=*`9_fb+-Ea$!YA>wcYDYQ==h_z8Pwo?wcCRE&SN78ic)4}`ht^~UTuLaW&DSA zn@+q}Qi)^jcdTlo><%!1apBe%_p_C z#fNoqEMo)>-<^U5h0@jqOE-{bTf>&zmOWaZ`cF+}=FS(=T2XA}^{d68JBD5z=lRea5jnAq4)LkUp-2}3Vx$0(YlO$MXvZRB<2i#{pFKw{w z9k*mE(Lf9)NTkL9aAUFF$sZ2hG3*|`_@&1;hQ|B$ogG8uWrb!ysVy^|4|i5O;5&9? zmW-ml6a4OlTaCmbEp9$lRLYY9Co^#KUg%COD^d1X)%h@Qykvfqkc3p*i=` zxez5e5K`s)aKrF?d`FRkp%%63o^I2X{nIvVqD7XttX_EjPOionk_!~ObCRR}3?E~7 z^&%+0Q^5=7+w+FJxyc8mv2OVShiT<*4J+9;3Yr^FwYnNpcO0=;fBG3sAt!42(xQC^ z`9(0hURL>S>!1)C}uZSI1%bF!iBY^}uy z#Dk}`lIJJ&D3NuiN)JfHX$}S2MHpwTG(N1NoN#yijc$p?}v5K4^uwT_Yzs*{54to zzVH{0HmH1@454xt`YF+wW6#6(cKXwMLUm-hZVreR;xBAZmg>a}LE4|-NUQEu9#_+D zuv+x>EEapcDOT2J-v=d(K8zPwu>%algh!% z_DH2@+qowlY$&$)BZCUHcpRn#^KZ`_+D}x+jJxJ_1I@(x|Fq_=eri!zW)FF0VcEK^ zedl@Alrc;cf`82uxc2kv1$FljQruqT|S$jyIaYXl#w*C-7gc8L{&tb#&=*f zhQqb^dy@O`v3A=kmL}|M08?qQN3-uiSb{G{LZE{*xf~Oo!G-#1Dm$5YUzeh%4e5(iDNyPv^~^sH|_wwah@Nfm$Chpu89{Aoea! zv;zntDQ(v`4TE0udVtHfmA8);J&Z3FYV@)-Eg>P8AV$eZ+ZKfj|+_+6X zG%{_U_os?5p8<%fLOyR9GyY> zKZ)phpnRiy{ggJY9Eo^%x1V=jbGX++**mFz|)_Vk+WthX$qZEMc z#XQen95zTnB|DwqZKBZj+R(JXV-M){EZ*EwTsOmt(N%D1YC!j5oBt44>z3d5fmJa-lpFQ=qg8l7QeT);{hrKfu8_HC z=INH~1Wr+%AK7)s_!DS^cn0DCC%OFtm zz+bry)!c!#@s2#F_bH#^xgM=2>y#xP|M+~8%5bNu~mA#9HJDMN!s@!-mwm^bC`|!;!S^hVE?OPOJ+jiE(a7o%sa_2i1#=2P1OECq&J^f zpFa&>YRt&P8CBZXe&X#D)*(=tm*zs#mGsh;@d928TxZc^oG%*81e2_vat2-CD7cz9 z(aUCgC7-3Y#haX3aUOn+`$*a-`K7Q-Aj1CKOkgIT{3zpU%yoVKlZ}vdf9qVWJnnQ* zmq>D1zvnZ*0vbz$+e<4M@2!-x*(ZId;~P{xB0)S1fLC$}EZPJZh`4lERAmH3i9K0V zqWn!aS%q#27Bcs<60-VjFCnPG<0G?1pQCW8{?Yd0=lQrh?(`C7kowzxAxMv~`uS)I ztZD_wS&uq+{HB(BUu#@E;}%g{;W36{@rC-Zn0yxX{W6-X>em7W5KHtGCB80&uaXLZ zH>B2deTKr*+2HfhRDbcSyC~z{WzuMD^4yg$ZSHVsp#YlIRvGMCeDv|aD2d!OUrBmk3J=SV3Gy+ z*Vo7U?(WK2W>2%N(bQz+k{hRY4eZ%?_u5jCGnO*RA2FfzOy&AMFEcsPHdd zUjO#(fz|ucaelk9?L^ipivRtqXyu=lD|%^`$Vl4AH`pcqq|z*ZGT$SS2TGqR6v)z$ z<_5zc22*|y{x0g|K6rc@hfJ45S$B$db?KGkP1@k@-BMf2Q47jWx4oC7B3$*zp4AS` zyT21ul`Nrj;2_6V(1sQ8VUi<~bnLJoo7E}&8t~G&-Cik@Ii%z!7EbtQuorz%YKk9za?=lBtCB`>_}rPl zaDKl=hf-`G)np3%sU1D)j5kg>TQ|G8y&b9L9HPU$0gI!{FK_CuLNUV^CY2hM>~*dc z#Sy&>3@?8Az&1Mc4pdW6tlEG`w7@lX`Pjv)M-qlEb~fhcDCL@aUghYcK8lThzrkwE zQa!;PA=i}VOtkS}W15%YL)m{lYw44v7q&T3ptio5o>J3Z`=Yl(SQU}~&`*7UpL?si zBzM2d$0^)keMLaDk~;%;?1E<0ID4QH+Hm#LsI@fbQg)D4>zCv6!$04jUJs;x60Bv|56HqdXFYY-`2^ZFQEN;kR4+(8Mt^!V~Fpe z?TD!6Eav8_UANDc9njqqD*iVY0F}dXn&4aITqR>WYpR_B_PTOD`E=>$LcPkByo8qv<# zI%XC!lF8U<|DExZ*XvdgV0eAeCRA?B_Qr?N)JuDU{jQ*#jG8OqP-|>ic|S%OjeAKo zoL)6lnGZ%o_xO`)45A8>!Nb-=qK;OY$8gP3Z@WX^_cZ^D-l`OsLXV;QTbVhmV#VtR zG>Ix_F+W@3(w#9A*#z{wiTCP1xT&^^XH(NzA))wHvUGCLAt-xbDJ*Qxul`DhOm2`QMQ{Ij4R(AUYi7WNsRp9j3G$&X)hvV4oQwO<9;UGg~xC5KP z`h&B}_59zSJ}vYVYq{5ljgP96k$T5JN4QK&|@1#SrYk8M^`Oe8{REksQG|)aha>`;VC+Cwbar_vD%>Ny`hXDq6cXxN!!QI_q;N?8~>~o&`>HXbL zcfR#Z_pDmAq^rBC{w1lXH2wZ|RT(6sH+uLqyxbGhD-!-_*pJc9X*iVep`VJFFV>P* zVDmgRbf=q;O)H0qVrJyST+;)30+hh@pF8BU#BU}>!q6N3{zSn9@ES;Z>=08-4iCbu z=zw4z_bp=i4Bp=S8Q7(ICo26ipuFRK>i`dtvv*oQO}!3VCI?2rC}eXIB=_K8^d!>D zI5AO$9Lgf7y_Bp?n5P=?7pt0>N~WgxQU#qFR>!Pdn#^0;r3LCjEiY}-sJEzvUilQU z&q;v*vikl~-FuujCAkqhE{hRv4GmdFZ}@vY44UK=U#qI?A!fjnXa*CcOM7-Pb_ro7 zXRqWv2+&~@t!t^~>-(yv-a>*)yiYIj>550Gk!>y2+TN@9F7qMdr8ZU?A7Hk>8TTn2 z{wKI%V6oS z@?W-XwRGP)X40Sb++%swsZVOZmFC}SW&fEfK?Qz%$lY&T;w4kud`P2I1L*u1WZ;?I z;#uf8gvhG)#Zy$8d+1OArxL?f*`vBT2zv8nxhN5B!d-_s#cy9jC(W#hapjs+lPeC= ze0buOAV+n8gY5}AZ|8A2T+TXln3`I)Qr+hr zrJzEoub1SAuTSAO86NYCFjaXG(zmmU?aUaqPZjAn`s_u(m%UQNWyDFWX1A3gM$BSk ztNi4?^DXbIP<8RV{QSvtxFgksz0)6sME zH=4<)#|~_h?fkKq$9t*qw`-~nnAfh?jt-M0SEg4(t>~6^Wfr@iqs*od_h8?>oRkMW zCHCO_DaXC4f)Zi}nSyDivwLEV&(u*&jHiSM3)$A2TD7Oh#g~-YL0*H4BEgzrRO(G;osdn;Zii&3$*SC)cIwn4POCs+o|Hh5%k28XCw2g3c$NbhO>b8+^6CSKM zt{3KBjXv7MLR>4|;hBw*CD$QdVRbHp`Oz@dtnx*emRHUixG)^7{Z(N_RkEdI>qk5P z?nu%Ac=`8NOQ$G!oE*3|JM{DXjH|KQywbH6P)uE#WH`^*rU9XkPa=D}{icP7$R-+{ z&~e{$8)XgC=1_J34VS#>=$q5&4Ml`+dXlTSM@blq_8&I7hS+k_ZS?0jR`n1f*Pj;hk!Q)yaa>Qbq$$9*et*E_q z5)omt;T^JePhKj^nSjQ87&*T!{s}XVRLpG=fsHe+BL_N!Q*VJP4U$I->V=t+nT?z- z2P#c|XGqD>?SUHeKp>iRYeqr&F0erOBa`tbc+NBF^w`&jLe~&?Q95}w*=X|%%xoP) zfg(XyC?)qu{kC4RrcTc|X9)vA?9e56*7R*QooaVpV;8 zKVC=+?)d7>DPlX$+oVfzOqPzEEK|l?nrMdW`tC>qNfYDyLUO9?npuyPA2PRzXVM2b z>_Y5WalYF|Bb7fiSuh{l*zZ-~SQ#G*iG;p@3jHDsSiCiKKI!AHt!j;Q_ z$SXqL0Ma_lU0+R0Pw1x0b;aPH!3JTRKWS3YLmKT8+&U-`2?WCP@b4?z0Wh^kn8-v; zF}RCW=A9BM12YX#=%EWfKi5oUP>e-xIL3oCm9|$6W%V!cwOz(D^MZr;g756Fvb;>A z0|NYu9kddYN7$&;JCyJF0+mElxRZr(l(k+lSiK;-M<#<%LE6Gw;}iNXd49- za)9*f{psV4K?bjxYh4eNRbkPtwBtYT!~@5o9935){xXbFrKF~0y*nxx&~P;%ktkma zUU{DD9{OCWeCoVmMovQkJuN@BLX?bGgC?*eICSw7rke2|VY#1fx(8SjX0$ zJVN^-%@DQZSs6ca8eZD^LNT9j)(Zh1LQy-N^R|!O?Fe=3ex+K@P;|POy!X_R(m(rG z!&PL0*?6+B9+BVr83V#Z9 zV7$edMlBOOrVcTWz0IZh#-jCox+pPam9V+Jd4g2u0)agpnW1gMT6yxdH%r}N2tuq5 z)j?P=OqHhN>cYBfKj)W8xYISitwW_UC!T-H;M}cT5WO^$gvRqW55rWZUaLb$0HjN9 zEZgFybYO}xooxj7B-xd%fMSv16>4e1& z;bbKRRitG|r+9*snM^Y&%P<6YaSwgjIy}}7Q(2Me89ES3X4`^g#BS{b z8Q%|%>UB_Uq%jo41&`mvsZ)^qcX{tw!?SSDPGEhp#m@2ANXE^Kl)vMNlMmO8RMg!{P zt+dZLL%3WMvq1gPi@w7x$i0o%zS04gpHL+fNpNgxq)4-6Hebnvp_wn?izFn#(GLMv zfGr9>#&d}v1BE%0_H(?K^Wpq*uy-mlo9zY0){zCz!wm0u|Kp5#rI57XIzduF!g$kY z>4PcXaOqRjbw`9ru81dXkj>AvCqN~@n{|Y8uHbtO>u3wkU`AXJ8(AyAJs+1H=dlQ> zgL#Jfa9(6ZZy*|$F(shRmwyaSi#q8zR-#VKafB{#C70mkEuYtI%e=sj_L99s-HL+W zf$f--aO!K{u6HW8ZhxTulh+v3^TNTEo~5)FmA#EVcabS}>qw)Sr5`o-9Atz(%C$QbC#|d3u_gVzW93>0lt!|^vB7=Bd zU+=EHS%-bo<6sT(^#W@J)k~?zhwFF0r%S zSFYD(R;d!>;lE|%nI{nd)me?^MR~MHQUGV2ey(S2zc3rX-ve{)4aT5G9)C*z6{$G> zJ3o^~Gm*4;rAGeUIvqo)ozYgK_D#jL(OvF;ku%pkeTXa>cS$~9-P&l*nf1~RQVhes zz;Jke(q}RLhA!Qd(&%u+9@9)ZdngdevP$o$P0J{U)8a*5ra$B0Gv<_EDu|e;#q#ic z*_tA_9&4mHg%>utd%e8!tnU z^=N+L(90^#rm&=dR=Qtrl$;y1l)sN0kAxThWUyKoL*IKxHH<=ALtz&ym)P) zS4;TN>g{>a_g+J@ggx7TFegjT8|XTISVuA!n`$H+#S;Qf#U*|{(aDJ3^A^GPgvtBH zM+tbtMUr0Io@`yB=kbcJW(c}kjhJ`m75{Ole=M&tu>I2di_6dyphwId+J#Bq6D56r z@|^-(pvtVi#zo^nq^*!0BY(u72hn0vqWh9Kh0~a@%VqbF*@H0$$UTC~*z1_@R&O+| zV0@s|!ac9#ce2`^ek#G3g;gdsVCa}UTZRhnQH+8s4_3CpVLUU~eFD!COeL}6$>3%{ z8?YO(7HjfxZTH+&dS9a5i7qSO-5Pp9rmhe9%D5NuWJ*3Ek&$S<|{lHl~)RPm8H?~d9lQRd6>}flKs{e*8}j(IS2pi zi(svPWEaJQo8m_ThanZv>C^QOOxxNn{FIsvoOa>~1NC!S?nB0E@6GQooNib zqY=OwmB$t@BFA=xEdK5)9?CVHYMf)h;n0qfj1zIxCt3c2VIc%vl7j0pp@g2a><7EYZ?a#x5Wm1 z(t&zCvM3r|F>}C9oD-pw^Od>dtv1}V?D;yX153@g(_pa88>o%1?|4gqUFPJbaW$ds zGt}ceguag{Hr(}Qt7VPk5$zTiduIP4V2hR7oq931WmtPNg>LUggD-q^aqH^szPkMx z-^{1yz@BZ<8yq~ZFK6b)0hSQMeyq(y$!|=jPazh)Y)8kv7XP6vPjB>6s3fcH_BnO9 z4VR;BgiPYkCsn#AWb~vcJ+J)MWKW*WZ`&jv&38(uj25ig-*mHP8r{A0M(|JT2-C%v?iw7S@?2jkyC-VK=<)Kcxbe#dKx_YwrdTrFPUm?kjm1VtV?rv93l^4K6`G~kTTv~Z_FMolJ4063wTs9}-cogW=5y{lasR-B388-=A5F-h+> z%|X%BJsIuN+V)07-X}c9|!?DU> z8OTHo1_3L6_66Vod2*Hlko=Bk5gu$J8TkH%WWO0zI0Rrq_$LAJ|G4!3zwzo=@_!n? zq~%HZge|KzT?lw4<6>Q({=@rjJ>Mcr}-Y1@MzO3(hPCoi7Z=RciFWRaLAHw=70fdeikfM)#3 z-v>br3~f$Z9M-yXyP}`}z;N;l!h^k)!5B>eBWJqGr|-z|x;vgjX-(}1!K=J(C^XYj zusGrv_nwc^?a)tpxeiHkr^sCyzJOGuD zRrOT_D(i`q!Rk)Ga{(uBRq=AF$jG++W?;v^Jdei)1uh9`*TioGab< z=~0Agf1WYg+O#@1k9=*tK8|kPuW&C1Cl)xNGrc5-7Zj|ydo_JX#K$fer&K8gz&*>l^l6OfzLT!!2jaCv>3UPeshhEs z2{By1fv|w83Sr2}feps5X1q!4ctX&hU3Z6ltI|r^|4gr;oexRJz((u>1x;&kBgIn- zCWP)kOMZmt)49#um&!kE6+tJP${|W%Gwk(x+D>*- zRbVPYcLB~QvkkP5u-)qd9QI}%PgcF6l^KdRax|`L&j%z>4Ey`wZ&F($fJ9OR>3l&y zKALT`>{=FU)9L6xa_Cn~D0B>UDD%7#MvbQrGO@W{EXQNvV$TZm%jrSdCq$8-Z#Tij zdIJ+Uz0>x_xP18^Mob$zH`jk9o(F&j}6hB({0Z z^g+Vvn>=f~4cx&9oTwXA4(lU)=ch0s6@lZQ;-cypHr_ zI2Q6AFZEErNAbEAML=G>j<|JcXLe3!n*H8;!zcP4{HzZc%{HhX0n*t&m9U+@E+t_; z|3p6aFa?DaQ-&I&3vO#HNI&o!>4w5dEk55GJwU@?Hlwa~g z$JGPF_S~Mz67xq@tp)yga8crcbGYD-ZgLxDl|~=xCm4qL3ml1X`(%qqTnm$WA5`t8 z1^4Ag&&}8Ehxrn21P`qEPncUKNTTn1J3q|!W6DX257dNT_%BU)g_W!#0z58))B75l zgtaN!rw5*^gadzu&o(KtI{I8diaP%|$9t6%u*YC&^j7G#BGJdT34x=WLgdDl z7PD3zYo?rC$A;Ui*Qu}o|3#4%g7jj4V&9lo=s)n zF4ES#YL95S8hpNAtD`?Wf8~*BbNuZtvbG%=bXy3n>A}LaOLYo?MK@J zulh6FcDO$Z8&L;qL>A~jk0a_i1^tB7oHjW8q_@_RGkC5q^18C&TKlz@HcmV~u30XA zYIsGpYQ8pDy(;h|yFFp&=z&LJXEdkL3eJa##N)_zUuc(IcG-;qR{*;gIjV>WQ@OI0 zdiI=-ID+K%cX&Jio>@(6C#Ln z^c?#sfi0$4?m()A5O;lx=cet=A1U(sZNc}K>ulpZHJlM06BH`>@^JB$ZC1S7Vhvq{ z)aScVo|pZI54nP_cQzX#N)(qCU*M2;-E*RtIKQ<+N1FebN-XTO#NGcj*=4{#9N#4} z<_?~ADp;zdNLyGVlmw#*y16Ch^(CDvpDa_>gFKyf6AR0RLUtV=GySi75N>Ujj-&@F z*MH!u#-Iq585|bju?I6cdt6`3jWnZYqj&b90~$up8=Lry;)1PJ53>w_uRoZ!b;mD9yAV3)L&WxG)N0{NnSCg_%HmDlqvi zT0F&r3~QCC46#YnW6*I}hg3{f?Yo1+k}*XeGvXCtzLNsb!bI;jJG zQmt}7xy#ei-9GVsveIqvgLvcDcr0SYq9AQ9l*hqou-cjPBe5s%!^H1+TdKOJh3Fb3CR!E*okmmoA?Q)2-5`FJiDAX=12;@NLU9 zyB_qM1b6^Ze)Lz+iBGLaTt_^Yk#RgS&FdEqJY=X>`&_eI3|JeyHscpcLwn}?3`ixd zy|nH!-MzCVGe`>_ZKjdWXi;0->Gj`|MQ3O_sy)xu6TOt-mFW(iXJt0I01;fhFGJD?3JA3(3k@ATZc@j(eibS9 zvVDftIhqD^7qAbr=JIjwA%|%gdZ~Tk!w+iL%(LPu4Qo?J+}?H6QTi~^+`3fx#Uph9 zgSVLDiGFGJ)Yf+DS2I zX5%Jz>fH(+Cwuiju>gO;j7;7OP-R(#J0x~ys4+{(ldH5wnc$eqgY)4~D9G0W(DK;u z%Zfc&q&+1ZPhVCg&ZJYgmd)^=Ub#)JaXQA9gC)+R3*%nh=SpZ-*KB2)ZaU;b#Ut!J zao@~c*M7|hMcJYl-F4x+uW(w1)p>*yAwjNct&w+ulX+r$f8GupkADE9vqXE)9H4i$ zendS6IGf5lN4vGI48!)3XU}@Oi`Z83bCC@d;G&8B=9gpCF?do>ndka;|N@$N;()%J?Q%BhFGV5qPLihyuYS z{TN5lF+`ADVfrDi@RWWzw_ug3&Mr3KUMNeTCcxX!&bt|E@9oB0LFN%Mx5l}5usfrBhw zMY9!Y9}qY_HVQth9Q78cDvX3gNsfYToK+&YJ!X@$`1KwygHf}Osir;GnIk$PJlt&6 zwj>^n{W+6Xp7vMiup--E#8cQJN|=y-x2~qS167K; z%@I0-L3QW{XO1!YLY4{(Up<>jd_|TLTUVQo$UGH`rS_AB46%{Ro+k(OzBdsLS~J1rBg6*2oIVhWcx zPhy(T!hG_Ymg9f~BGvHb1#gt?21s7c@TPMi7Q6nOeME0QGv=?SU!biN?3^Y)37-qT_!7x2 znxMrCFS^tH@R{jM$7E~TUeACxNvnoIB!;jW>NZlCNIz%SGz@+5rP-tJNOC4juB@{a zo$^R@r=Rhn-QIIa%1xzdVp1Uf3=qmJF61Skb0O)RxoT;4n5*KDh_#auYpTvTI9Q;5 z@DV#1^|(?ykZD@5nrY~%whpUjE{Bq4EnM}MM%3q2$wp^l8_L14PhdNT zx3ug#3*HCtO2u2xU5&w-EkPSWfD%-MRC2^E^A1U7A`VgLP|+6(x}+?(uLZ$ZGi>J# z2?FKv@VO}XYGs)0m$Y~__((!0o%LA|a{#?Bt=h3-gFQN)#_jilP*VEt2v3!YQ9fu> zuGUtoM*Oee;mcFb2Sm|g9KszlI1P?JKC6>Z;tl$IEosIqc(6>M_TUW>nS#S{wCG%2 znwW)BfkVU@bp0q}oC6&O7de!K1!+-Q)BPBqH;}}zcyp(REbbZ~Y#zE#`Z_E8IdU=d z8kP-LeGV71hpkNE{Z7P*F1VlZy~e%aQns{i9OHO4(c&?HZQO4pIKZytmWYV9cMnUa zt=0cK`uO#7Lq}z9B9rK;+1YrzPo9%Z}{|AAP1Q za!vpplJ0mAJplWRD{K+aTxB$eD|mWqR6kaBhKiLGNqLuXN)Hr@H$=XEC{M3=Pf?Cf zG6t!vlC?o+MDCfQnq|M3zja0;s7d|xu)q?bW05;E8e03uLpdag&$r0mA`wCM@~8Wr zlDh=V2uF4?nk)YoFfFl>rFxfU8f)Bq%&k2iat1(LwIb#>p>)2%%n6ySN*o$9xG9UP z;6C;fBA3s`L55E}P#4?V0~PH$=eoSON=G03@RY?-lK7YR{7ZRp;h|bwY�AZycS= zeNt-KlxBbtdp7WTxAi@xJDJwt^bk~Kok*0H7|AT&|L15YFyQR9UVB@oni;sA0?yy? z9hWCz;ME%+qu(JBbeAEEq_jP}>4r`=SO0DWc213j9Lw1Hy*YoDbbmx!N%H`TK?)p; z_I_c#sN=ih!7>tcSFwc+?<{0ECp6ddhn!?R~hwsDTRH5gAiKC7DC=p(%^3Q-PBdN57Q=o8dN z_c)(@u)8hw$M(aQ4+Zel1kR9M$SGGyU|69UF5z4y>+U;QPznm=x)QyvEoIunq9vT2 zdU@SZ$b9U(t1?0CB?hm8Ia#7?LLW{}brs``LMA*Isa8maPJ~}y;P;ctD6%ze_vWfc z{iOVwy;j^A2(fKZXo2L)&6SSFABlW`f-<(Ly3vjLmG{ZcG#w^*D8@ zMdOMTuyAIRS1+!ssaSXucd%=%xtll74}t)jXkKx4q4z#IBTaV@N5tB&eb-T9jr&nt zb-86Vp27Dm5uRWC;UgF0C=e$ql6bV&IPl=QM-uq_i<~6T?J!4axHk z0Bg(a&F*|Cc;351B3EyyzP$8UcH${T;E%h3`~1X?eG^6I^F@8p&$qVV`fa5iu;Zc- z2`CtQUU)`_yR!=HGh4{$sQ;>ffA*@k(}FE_a5}0?c2l8N3I-r}0+`Z_$(pb-nq-@B zyUS{!F=z@im#GbyO>eT46Zm@j1q-J4$KA)N+*F=VwHga1n6R;MhTB@zxr;wi9JTc& zi{Wc>6d~yCCtcbXMBU&w_;r4|O}XRa3VLu;zOnf<3y4z?AKgo`#Y1gy{^*C1*;o*C zfRGXm_gBV2bS-k+`-QXm~Ka)-Cqo zjZVNfWz9978xTy;iCpgIf+S@f^F;Hs(VDhoay%#XFWk|Z)$o%`q7wVo9cOWOJ825LNVG@?TEE!nudQB2WPO$#*INEe z%EJee@uM)svXIYXW8cO_AI%US=M!inT~}bnR|j?Ml{d`0uQ9WdD?U>|{0M(|y5mRE zhC#*F8K&{RPfebVR$Zzl%$)dV>+w$}pdRT_cVQcMmHOD38SKb`D2JH_JT(VyrgvlW zRgW`p_j$Xz&6R=lE4GHh&4fExUJj4}BC8gX9n(U=iR5`Vy(i9-_f_xH5*M|i9B8q_ zu-NU->`-m$Xw6m{6%#%|7a*H%J%&ZRURuKG!ISiM#H(2tu7oezz0XbMnRS^t7)3J)O>WLh-`ef+VBc5O;s!aP`%E^1t z32Xg}*2_0?r+~FrW#)<(1eOKFIa zD*4)`yg&!weLFZ9izALrv8Z>gCdSZhO358&iyMi|buTiJb^W z5=Al5n9y-f20Y^cCSxxZhw9+BEFw<0H!G9JBjXiydZ-DcoISg`Q7XhjOA#k0r+zv? zF)?Gcgz?C99$t3>e5SQMm)Wm-3hQDHb9+r^HK;r+>wKLjaKRU^ZXe`U<5;hBWu5qG zd`FIL6;~T0f6SoRaZmy$A&$vyuogF^=5R;&XdzKil$`vaPi1f_R=; znNm-;qHz|eUqPzCxgTg^BsAO42*&4*l_>fDu*FbRPXJPS>djSTX37JY_Oaxv(o|yJ zD(zR4g+DIOkf?y-I%RFe>@~s06G`_Ij%1X^ONCAscOpk(R@alG|t`keK44BAcJuRF6((nI3WKiO4eR8{Hbc2F0&EcU_8-~v5? z;IuK8eU>nZMPyCA@t7w5!rsagTwrU?8j{g-_ev?dbj$f#M1^Jd(uo;+wz4yC%hrUZ zi{8=y$dWEJERcbRG5JHVV!1iS!-T;spK(|Gz#7dqgTiSX+J**+T9s^FlLcz13#?waxl+EV)BedKPV zD+`5*xUX*i_=?nA+D0;V|E6Ap7&_t2b01tQwAC<2ugfe4+yl7eR?6>*Q{xPLH!?6s%29wnncdJoCj230d7YINmRSPJn5^R)Q`=|Ys(qMRB zA;GnuwX%wBS=DSUMIioxh0VQXmpX#iEKI{ADW*q%Knlk38|+ivTQJSXwH>A;=!f`I z>49c>dMUQuriKin2@C-Ko_{UDWeYFfTY-Ez zq7iKwo0!~Xfvna0`*yBl%)I20-X$ZyS;?glGlrJ*g^e>kE1D zc@uZ;PWB}|*lf?tw_utZy*OXzS5o!|BlwTSKlX+qb-3c?9c%b^EWEg0Kj|FF8EiMc zC%!w7>vjN#7aIU%YR7VY;`%H;zr=2EzVKs2XqTa|T~+p9l# ztZI`9fJdGaGTxfx83j$osoRTVl)>g50frm>)TF>|L8*NfCivn&aQ$ODUt-LZN^*ve zJZ9;l-&Yl|DSsPxiR!A7sVm_iXAdM$=Arf!trU^M4FDlsyPDVZFdV?Cymj*ar6)qM zg+mA6mP<7)E8H;P$$(Wz?Vj!uNJ+@u+PcS+qmypLePhXM=y1I)OjrS6aFJ6zf|5!5o63*i`Z~RLBuQx1d!!ToG96BM?-=C*b zxCj}~1*Ctoq!A06PoVq&Hn1gVW$X?v&g(ed_2x;U zNYC1BZMBW^&|`8u+g-p0r1CImI6j<~z8#eU6^qWLkY6vpS}rVZYE|=5@Fi|PFr^$X zfP}gkRA{7=#;Er5-3&3*yqixj(a?n7IiD0TZy7orKDyKTUr00J z^vMk7-xvTV>R$9A*t<&Jm!R2<$?Hv+{_al^Iue~7R)D^6S0OiZ&eW+xC-d}E0UpNQMKgS+qqkCeh7#MtlEsxZXQ!zt8bTAyeYBfAK=?Siv!i#KCr zuryRX;UAv2j1p$dtf`Aq3F#ZYnaC6p_LVJ16?d@eXx|XY7V~w%OE3jZ7 z#+tS3R;#1OZzuD&-V{eH#?%wj#aZnR?yraMchsP6kukhXdYzT|$Xw&T`{Swk;p_ZL zsIm6M)aXusvhp8gB~ApAkTW-^?Z`hu1y zEL{8<)>a~@fkqT|f!*ifk{_LPG`H+AxVz80X>uyvD;nIjgHaDF&U&p$(B%>(dvb6o z`&`y$7a8lR8nm!*aa{FDczZh27eR#&BR5pcv<3kerDI&Kum=;c6Lq<7ACp` zN-)cb8?=T>=ki(*r85FcUMT}7ijAuUYe_>e3k7#dvMS8L3$oXAHY)K>Lt4-jw(aCr zw4Jnuotc3g*U_Ryjkmzbq?n$NNlyg?UPX)pt*B-<4lB@YWaJ7WkHr7rW5^B|NH&js zBMOhkHNOy7S{t^7IXJvArQwB=JUL1*XtWEH0U9~c7(xb(^Vg8J?kT{MMVFlEbU`(y zPwP8?H!?il-Ad(x*m#B;SO9ToraYak)X)FS)=!6ikG65Z!A1^40@{tSe4kzJm9T+WDd1 zT*{Yz(PWx+X5Y|PJS*Bq$HO>@PfSwAGk1!6x!jeF0Cz?d+Aa+jDX;cZPA*K#HNZqP zrw39UZz9DFOV>zzqg$diO8$|5I3l9j!D2nRnI~pePR`Z)s2pK5Vo%)1$iFO>(ECBc zrs)|Iy9&;s%uCmN($ni<79s8wNAzChG5~|?*{lVU^J*K{K6z_)Dsohh!H!fyeRb@J$*3clV#XI@Ql~1d6+C* zZa0&mRWsGph{ld6B|kXtI9PjLC#y+1?n8kYn-WhqIZVagL zpO`2;T(r)Y2Es}Fs*^@znT#MtjJm4sno4XHKCIl}0RKI4dNBIcrJ;&On1!EL{Y}wn zgy%1U)ChM3tGq^@X|B%chRG8f1+izk8ON2)O;+qrA=yRYID|Grq^Ez)0lCGt;hozB;j4@N7_E(z1Tzf7U zCr(}~C*K4lIdD8_ZV4=7aj0%VEOR2`Hz6j%%q%-!D%*VpfwK zwrVarGsgk~Ju59^Af;(qIkPkt=yNLV&}^&p#E182BZj&=FM(d-8^o{OhSRxhX&6$&H}mVusqO4QXcDkpS6C^AkGT|oJYmKv8+%wpZ@EFBh4 ztt5w>mJSBaFh90?)mDL9TyU0w7ztG%*j_xgYTQj9J*gW>gf_`XP-YXhzU`OJkw`(d zMMm@y%qfqZKqykEIgg-|SZGU6r<&o(<*kW|2<%nSH=s1IILF(O>1dJGR{(5Yb#SWz9D=>x-$ti)-LxnOd(VU&%V>h; zZ~1tS&sG@g5W*1_4&t&B%)Myo=tE~X98%ls@gvEz#CzlWWBC*)27HEkEhPCTda!|6 z0RE*Y-bQH4X=qZwBhgouH4K-w{!87icu%&CP?WrEX>6T;x>14E6bnhRcbVR_Vs;R` z^dTP^?aNFZZ3D(u(Nj~f;pK-)QPO&^iB>udHm`#fC=w)lu2a~Q9JxZTY37|O&iq|H zR?dap;EUY7vLpPX!tz=l+cj{Joh~}LUa$!FGjewCsO52H&DPG-a>-LA>#OsPzWID^ zVCZ_~98aF;v-?Xc-KS(Qh0TlpGy6jn>pw%eiD}l4+Px3 zz7G)#&nQcyV&a3bB(UoI^YCq5JCY1@*)`kp+2er8$^`uAsnC8E@0qK4ShXOLYsX5*yO>TkM^RN z&N-2K-_P?4d`VKsViNz-_8MeRUmNoLFw+lz)}Lp`Msb@(GEtmPezvv+Q5Qor zmCUv25%53Ifw`^z%kwi+E|Nfu*gwC2PkLfQCHN};{!7eC=esDzC8<%SQ>zaydH%;c?2UT^gpEdcUCDHdthsKU7^s#O^N(n zx5!Pp8){25mxjJ}z&*Ye3$8k-t&BHtJ%7?f8JZ#po8wq*cjSTgH_k@Xm;DseM}8Jd z{OgxeQ8g4ZML9a-w0+hwuxxPt_t_Lc$T%^^#Zyn+fuoW99{S)x`RmI4>iMjm5lOWk zj_n#zPg;tnJERTrNq|6RLR^quuQ~P3lXTyKQ2(D;fFB-ab8Padg7rh0!-z-9O*aJy zLT5tn8od(E_(Jr=7=y3QZSR&+L_pLT*C1+Nhaa&w_o%S>9RzBNlnm7<+i5xZ0X6<5x#qQ~QYHCE?@Gak9x8buR2m zGi_rpgeDZ%_+>7b}g*-BJmYeIlXd{K7+vONGdE+e5@DYr?SaUet(lwCZ^UC!6q zdV6~dKUe&DN8-+PNWz#dy0X2kirHN=Kh4(a0*ZDO%!hF8&P5qUt_eeu<@eD)QFgKY7`=tQLR&hlKmJ$%>J?vsRP7 zQHOCZ^FPp79ahupp_pA5*km6c9}EnPLh=8QUw2~dc+)k@=Hv8QHa^S@48X<>i7F|@ zH@J#Tt$Ufw&u)@%CsU5yb4tG^5lNtTwD7EfIQ|64oM;rly2moqhTW5dA9 z+grioJRAC-`iJk8yhesfY5yiFzY7~vEe-YGGNbW7;Qx!F%^g+t{V#vlB=LX!2Tcuz z!QWLc-({0b$@^c(+F|;Csrla@WmB5}Mf+uy`G3{adrkKZ#Gb}26Xq1zBa?rmbLfi? zj5zU1PcBM{xN`fnKgEn39MIaz4-XG-$4!vr-JxEB`Cgh68XEc?gvozapVNCNe%Jk>b5>IX(VSZuZm!uIj%8{e< za|KmZ3@9k5&P&-c;m^rbK}qK3=BmZ2p}xMp9Xq|%;M1MmT_sgjQ8%|n;lGC~rUR;F zbsQw%S*Tis*52O!4xNTD9CsZ$DQjv5MBO z8Wg7f@9|bPq@bcQZUBcyMD&ilzdUy*cDvDAIdqBz0>kDk^0ewKER5@_;{70O_~U7%*_AVCJ=kL zT|jCFA7(b~bgHsO-&yCuJ>Ow;ja2_8Cd(`8=*aF4Ljj&~-6Wy~{_5=LF8f-U~p8uu376ftI+@c$Xa#cP}|{;kiv#j?h%-j{3T&gLAb5&s><%4bixMX>1? zdp-Zo(1hdhDW%2pj+Mhj@YeP^7@&3fN|vD2wem!0L~x7$Dg}+9vtE6C4-_4?i|ZRX zrtw$gh;~S9j^0xU-{=l^a?@W)y=YKgzGw5rO%TQh7Xi=_ibQVkj2~*p;+bL{2xQus z^Nz}yUj4KI+W$~yb6f6bTmlA8zYyStl=N_$Dm>GPkLN1W%qS2MI6w$#@%uNB2)C-u zR@T9kSw1&-f`jauRYY0h`fep=E`mKrGz_#n+Xm=_R`c*p*lA z4QSRjBXuztC0qEdiIvfMlS&fn^n3dW$&|!}$t5XmzWB3ewcuhHgDkLdOzPxFhbtaS zSQ2?ZrI{EmtzC6i@fWWh7d87LpP`M%(&>cE(XNN&*$D)Fq*zQEjAi zb2;K>g=p2{To`@ynmp78Go@kt)+rmYAi6!ZS5ai6UBr5QZQYY}_I-{8=jP&*D3Pb= zh0Zy^m>_*bO|%pnA75Nmm2e{Za*x-P8tH&^9f~XJG`2 z!7Sx#m{EXMpzO)fvc}S?xvh1MeAqpQpyxpYz3{m`4Hy zY<5HB`Y51*EgyxA^)cs!u=Ji9QU$&k<8Y&MaUW|Uj^Ec1&j9k()Z~YAPuyWW67uvf z+PaJehSyM4vpavDR+pXi=0LWA&Fkh|!sYv2KCU!0^q3C;FyVI0^ZC2ISd5^^T&tyq z9;X8^#}rSjN6VLVR{v!sU{Jq0rv5`pvmmSk!8-KBQ~H^&*-B=_scLv*!C z`sH`{#ouTq(|XXtv317x;M~Qvms2rp za!^RX8o#7el@Tlo&>x~@SOK^vUmK6=?`)Y6jSEH^fkk@Ib?6>x6bD!ozCx||XOm}v zyea-fJnVL7>a$+gINhQ?`WsUYl1R&9rEU%vq^||SpzRPA<8z@~&+{i1UQCTWmJ%%> zY+mMp_!gTZp;GUX$7X8ol20L0Q<7P?fL|+MRpz=ck@z4`vDq4gLQhNMVgnNjmz)DDC$QHy*N?| z3*U`fO(vFU*M<>AP8$!0Dx9y{K-;hMf2-h;r<}fcBt|Z(eMWOJWt{PiRpa+UzL%D7 zQn|MH(g->#-qyPNLz6m_`7FJ>yTGW$7K)nN0PF1O-FhLuK`JhMC-Q&-+}vw@1`#}; zWEhM{1UDI|Pf+RuL~O48loHYp7Kq5hpA)PO=`l+iDi;){oyyTfGQFh`r`tp4;97G0 z_-Ee9@v%ynPv69qJW69I1@DJ6aULuR(7rY3p7|YtfNFzath>vq0+d(@?M^YUHdlB%R-4g5!kf}*l~cVg88`agIg@UHzy3B)3<4m z0-;*h?G?)wIn9o{%Y$2ceVf#y5~52mhM@1Dgp6T!uq874uy+#$D*>5aGC?Hnv^*X)n{b$QN? zoD-N8v`^7Igx8uXlq3Fc-xG6ONkydtAwIb5SKe{Tk?Ux&rxFN4G=Q@|EID{vA4rBAU0n{&W ze^sg0{QU9QJqJl|RrkD90)lGO7~pis zXW?aBWBw%mq7u-xf{DboF88ya;=&?f5KHIDeRaEHu9yzR`7xHSn zlM@`TCk}>zsFL~XERKeMU*021E&$0iQUy$R7Do~=ka#?9&RhWd4W&-n+uV4)^*r^I zeu_);a<5yl7K2UIu}<)Wb#fDU0(}(v(%gRe{lO-lJ`f;R<%tatUjAuZ?p1lD8>n)ihx!5zOobvXp8Yc!& zXkRq#x3u)%f$4KlIjo-O1ef?w06;V=M?5kM()Dp06Y^`q=@I}>V#S<#y@XDRM#E`$ zA0=bmRy7O3`BkkI8Vn^f%3^ngJxo-?`G;rp+;*OWnJW|C3#-%laz2o%mF>hIhOH{Z!|Hx zlV>nmVDTIfWmZQ}1bQMv5ey)l0Wsyu1G9+gdDD!4wKG5c9YL=hn@{?GI2YI&UaPv- zlX*;fx3%RC=DpSe0DY04iN(Q}mu6E1NMx-}n|<6qqw1z0y?zNc_oXsy;xF?P1eV;g zy|#|084GlOO0xS;$Vz{rb*9ID;y*Qs#ITQ*{!)`Ve*C>(6srWAXis!3+c3ne7@zES1!IJA+T%%a^Yy@y2V2G3ikl7%nj=ppjxS9kVRFF9V z6%Z&eb8SbLMA=t4t#l&Ok~R2bof;m~0}2d>1l z7rt52-gIr`TL*!gwr_6zo8JXux^j-qV^nPOS zxR?Vdh6~ZEgty%(kQh zdwy((hHha97b}@ro-xta{-6EZ_UY#hQ+t9}Hn4{LGAYXut>@;9Z-=~=TuqzD@*_l` zH~_45a`<}V4%{o!Q+Q1RRBth@LnEP>>59?hmX(5!6Voncl`J}G~YPw6&r0d~jHthnZeLOJ@} zXa{H;>E4}NtOg{R@YHl_`(iHTnm*aTmy792YS?k9iLj(Bm-kTD4DgpG+xO5_HM<;u z0_Q|F;u@Fp(&h?AdL|tv1^LP6JFrA54EMUw1aF=a1s`q&F3O#+hJ@#yV)VnQt@fDc1Fa3YghMcKdqXzw_>aZ6F}j5<3_kLI)MgVeFDDWQ(fy z)RFocA1-a0;sg7E{REZHl+aX`1=m?lU3wM_j|~6*W7z9~VUANwMy<`!CBNDrgDHPG zvE(hX=2MSPE_NT`#$SaavSU4@Z#^b=fm0AhiEk`JvV9Sv{YY5sQY$L5wuYGDAfPP! zENIxFl1Xqon|d~hwkIIz!+^j&RU9SBSN+8kA|sFKY_dRBvFZ23wklRqu%h%bod+s` zKN^$EkgtZJeN`Sv40Fe{C0++IEpPt>fBN$v`}Qq<+J(EkJfd>@`-eZiryTEMQl0cg zSg3`T-m}iPch!|AzuKRzi>le{kQzqZ$}iLz_pQb_9}xH<`zgG1clFt>&+DArH@ER@ zJ0?#?{n#(lSep~e?=d;Fi@o8#?+aygIJrg4q=sMmt>6-5i zjyyaY_HOWcwZb>|NWFf6b?%31U2UEaL0(25kN_#7xEYS#Gl76=*Gdf)x4&2LIq7qD zNd*NuzS@`Fhgt8b{)6?Lhp^cC61a&dxAl)sgC7!nd=A~(d5xIx0-t>8Ru?0N>m#Zg zmq)1JxxMxecRUauk%Xil9igmRZ>xdvx#DUuPxf)uH%lp*WbPdYZ|V8LmdHZbV|0q< z)EI0aE=WAO>8jkIb_MU-VX{!96#*OZn3@|)C@l+A7mZM@^w#xQxT~UVr*5j>N_dr5 z=Bi2r_KN0b_7U09fYvQV=E*>BmR}r=Vp6q0%#8*~akut+U)+Wx`?%aaImA zi0oQ1!Oj`%$#C+VgEy(_w^(I_k(jzy#fj==*?rL}RmOZ=rJGAMH2ADp3U&kT;@rPz z%OLFdB2>}5s5> zEEFwLa;KGYmmkd@o|e!Kw02I66`Wtq5XYY&Or2i!n-5ajo!}hyDw)m|G9e#8HXk#m zSU*sBZDNS8x&t<62{Nu56h++9v|ZY|&JDq0lNl|c5OiP>vAbE}a^P2g-NV&^9YLgA zT>_o9E1ei$xwo_?8D|1~Bs;7I64rZMmTxrD;AKb_rBy{miyXIjxDQ#n-p7UmbMjHp$#**;O^41SNjVXh|Cydsie*UXq((SG?el#-o2rl^S3GQgzOyBg=E zQNI0}U*J^|))CdHQoQQpjc}T_x>h`jcbRx^qSb$4V!vcFoetnk{#JhpOtYMew~;4_5g zMWm(dn^$32IYn zc6rD=&aXRVwKiJCVE4<iBi~sjLT%UoXHcLJD*-;o44i4D(t&c2fE-2MeH@Ib3Pmxa3s6HxjK-58CGV= zxMMh>3d@2&mJ&|nBJr$T;sHn=(Y-b>04{6I}KJu`AE_7}I zopQE4$f?2;=fKw1Tl=+e$6f9}VA96(NpzPVLPsJP0O80mX##Jp9mY9rgvKS_N)hJ7 zRAMARfx-&p)jajs5vu64he95+y{CvgF9E-&o0Y|Xc%Z&M{c2TtVtIzYlG`8nni2Qa z>_*$B{315DG8v_(!z}INB}xn8O8wb31UyV0++keU!Vh%5Q+C_r=N(fJNC9r7^%x?#NK4HjT|4vZII zuh#G_TB-qHg#rEVRwH2Ezs5mc9%d6cz16a#2O->L7I-W!K7K1JeQb7LHD1&to}Yl7 zaVKDd3|GVKS!Muo(~?;EztGl?H{s4iKtO2`Bj;vkvV-e4rSE#l`Y7ysp}&33(MV=0 z0DYx%SOQ0_#$BcyuAEjVrOZxRAkR6YZr*1a!gP5a>&ksikqxmH!5#lWLw@ejr9H>4`DTI9 zVMLk|ZT}dFUK>zSwlM4D9Gv^hcR4qwHEE718WTL*dAtckDx7kgxECO+WLV)vEI&5x zqH`%t{F^@7^*)?zc|uO8R2B#w(CL}>vE%$4~!9Rt6|DG~qvFH!ZQGaibQ^p3xyPQ4t?idk|m>40onS0ZN2XBH^_ z-aGNEZElLf=@U2wSAvlRXK3KYj`h`xszXudxVkMX)+0Nybm1zy^jmPs1Juh!kSy`FAP}klMHVogxLrb^;=*)kpAF7W^C2-mfCqOUmR-_QU zpTY7k8&Mc6x2PxxPPH65-|N)qw64f2D|7q%Bj4QKrk@A>oq6CCyLNST4Ga%&j%SO$ z#m9fLI>@f7iV_WcW4ASc1OMo6ZV4`G)zj14+S-arPY)m7{#d(i1oyC%h#;j96nw~& z9X0&f@yyTbJ}zH~S2!gFrw9H0{dq|vhCw!^Ga+!_PxwozmMb&jqtnJpeg2#JlSDRY zT#|z4JOTo#aE!Lyi0tO(2Ah;rR#kQWo!Xe3RT+0eYmkXWjwHLZuF%m8CoWBd)T=TkjqVsYT)q1*RXCutCv9v9Jpi-U zzb4YnnEyogM)a|i;lb3j`1HSim=%5u$v>B}t4q~YqA#$XCa|2|xh75b@+?RF>yw++;=ft| zgvPqio$Nc%&yU0PjUFtQ!lND5H>?+M8xcHR{z?O#Ynj(F<-Z?vU>!2{X=oOW6L7## zS<#3#!hB%VghAf=<6L`k#|a(K5UsLrNN-~Lc6fhXFrPGUbK3AqpxhDIik5qIoVOV% zX7bQf>+aNWVQW8pA1KLP=GfXu^jv(L&!O|rCi_q_*Kn_y7J$h@rC|1c$u+kF-NYGZ4J{QzTJ_Qk*T!=#i!Ny3|0r?rDUPA>3@+xnE1mSE^NoR<=Qr(Yo{7_X}z z_SWMK4cX0ZDTD>SeRXEL@w>B|!mI`^4gdfl$q!GJK<9awnv6YrxsD`lo)%Jr3;Ija zGL6KJP!flQ=;NMagIxI|R_)+QaSxFxwEKaz;#(=tSpCBJ{#55tgPV3(MxZtAhYoRU zTF)_CNgDdSFzx7}4c`m{yZfc45_w()$6&!?!Tz^U*xC74+DGkA3Ef}nTa<`g960S( zB}CQ?p*z$WPxPVXd-AW-X+2+fna$D0*v%Se})JS{Zz#Oysj#|=6*}Z zAOEHZy~46a)2_t8z`BjKO{SzdZtPRP#7ywwSo$!>LfUZ`{?A}y#Hhr$;rr*u@HYpr z2ro$gaBLruKVMi<`hYs)jsCZ!RB&MCi9yeUR+v2Zs|v#|ElU#^);(bh`1rPFCX7c{ z_w*oUC2ZL$$H>yC#Er<1d}OP+`2O|^jPh|Ok3)B#e05PxGNK)Dw6ENi5DkXv9Ogk1 zC}mZP$jLq(A%Od}gt_aD|b(`=s|N(4!zmKWS=B1$O*- z0n%s*l-Dt`E0|%#Jip=Zba3n({R0Myo7mw$t9!ph#jiNP@1E&*Z;HR^dvZk-mRcy| zc0B(#2*;tSPtW?>UnM z()ecXYxL32Bkolc&!Boli#vZj^%`eWbZ$Xx z8AbG_2t(LS%yKMfla)`BI!5W)e2^uwZY|b0LoabhC@?wN2;t3B|K!oV>uiZ3T8u0`l7$-F08dz?lCg$VN;XT35mbj)L&LU>ZQ24)`?6Y z(1?{u+8rGJBUZzqM}^CY9!}>RDRoqz3w=<=ARdb+_^lXezlhL3$B%6X&sfh|!CvZ! zZ|joVI6GF<#amm679-BD_>Ty!@$PAG$Fg8GT*y+qsU-Sjxpy$55ir&B;^t?8g`deY z&Fz&Uc>&%y>&5&lVU6%_71yw^kq0B4Co>tdZygUklTJbh{-pQTuHD#+>K!qu!vtm~ zxjR@A8K$5%JZ!oXpGuVD>`|6avO!J*%z_#0fbXmM^t-~-fK3OdE0xv=Q2r}sV`NU9e zp4@`W*492n&=J<5-cDZhTz7c|v90|YT>I4En|omNVjc+?%->=LVEB!aBui`llI$@o z`8@zncMZzx)Xug}7jaRGdw})!7YF1s&S^glog9txBn$!1uzX2st3?E8V;_^W{^xWG z+E*+=%8OVS-CxU60bLLQ0+O_P$Vv0?UB7%&`307;5TC_3>qEcK-y>R}A$>8DR7_+j zR6IP5QG?ifV@%+%>nAvCu4;zu{H2L-L)6u5Y3nT?qE1UAIt@2PxZg6BY_f|V{V)0= z+&QkIp@Dj&0b{3lT z!Bx2buE!3}-_gdBzl?~Wvmlx)7yA*2&5zP}CT{7gJ}QQOkBJ%mQ40_%O^7cP%k=8W zSa%Xsg@wG@+95mu2^5w&hV47=td&lV4sWPg^u zrw_NmJ%_JJdAx-|Cv#-FD^Ha8kF|YLh?ms{F76SdR(xE%C>`SE^iVuqcUgwTFCKLeAw$!;dBsbHM zM%^M^#W-8Yf$8)1uNPiBGCmg3mj_YDnv^(<>Ym7`^9|CAjhAtLa#~o3*}OH0YQMao zNa>YUMTo*$ouN!xrrO2O5cV1;{N5Wt(CbRXZ5>|Ot9!5~K`egbCLXRlL{Cd3Euq)4 zU^joPrg5PVLGJ=aykj2R`j+7DEt(4Z(u9UY(A=Am9Y-UKuOLZ$DEGDso#)l&oV`4+ywTgaY_Obpk!xSSgasmVoSt zzxE`#H=SK68GUg<+VNwE9s1~5K4il*z7Wqty8K$?&6MP;@%#oHH4=^4@IsPzs~}3ui~BrfW`>*zx5p`05*(_UNfC4=nYd| zcW;hYqOg+8a1M$ErlKur0bcIz^0cuo=|bt~7q#Ewj;ZIAx=i3ErzuK3V@HFsB8P4w zoP8WaQ?D>7gu7~Ypa}phGLIMTQsy5|s9n8JZB)1OCSsU1J+BSi_wf4D%d`^*5j%`s zr|A1$R|u}A)Wken#`Z_SxgO`f{X=I_&hl%rwM41~jEoBgr*4YJZ)j@rRa|ch>qlQE6%m{D z9`MkOQ4uY}lo9JmR7}(oUujz=bpB$ZIedaR6F=|EeErVWYOkq#wSMOcZUWCIe0RLD z<6$`0DobkiPl(n|noM&ITzbFFXrU`EeroE=Nxrg2&BB)4N=FCq0avmcR9v!@`{N(I4`s^^I=Y7m;Hm(>tFdXh?nlIDW*mC$lDX&A=virw(qwd!yJrEcGP<9U)hW z+V_R*FDy(T8TAhT&04OGn&wKNWcPc%+9+S<(~wu|PemMlk?Y>m)V@}?~ZKr2MA=` zl^ZlO^o3x3(>z=Yszf`@5ry=P9BTRpd^Ucq_8P+^yhC{HGK!3vJd zwIqvWlO))8UcNqQr4NfmFqbxTO!X7PqqWt5$I**oJY7C*s8g4lg{{zC`spW{lcasd z=1N-M{Ttk0zFTimnV+TWtjUI#v+EU#QB#)!vmX!6d?sjXU|LOHmeW=^!(AYW$R+7U zQ25Pmi!L!TVcroKT8QLtsOo9D!k^>I+;L^9yHJ0#5|&o7r%3zd6x7K%f5NWZNlV z@A1=Or{|j&t|~b)Nm3^5|7@)#`E=Pay1#LyWl{SIXfF*D<8dA>#n6m$&NNxt*4~B6 zUEGfbf3N=td9LjnJSfPB_uT{7>=eJ`saUK&1Tt*op;89_`=|v&<#&>9dnIIxN))fF z@{P}CDWJDrLOihwc+2qg@p19nzba8m%5A?Fvh+M*Ns{CdT<~n`#$60$5c(iTSE@{^ zkG@t$JcL&d{F(gkmhJCv%erzDX;5D>uC01R6<#fX_dI%w=>_Y1f6r7+mP%X@%`LqB zcZI8w_}@@Maz4)gf70du3orP8Eyx}&r39qu5vl+&;9yXj?7tveMN&-^9D>2);EMfD z9=Rq++_!EUm6#~;`7`31z|ESIIrI7rC{*IpCj{lv%edEn=cv&kgVv3)Z~-t}+1uOO z>od%MdUi$u2g<6dSS`6z7genojXEa67zG3Z>*~7SzI(TB-#lcf_W1Z191=qL{yhS3 zMzPWW0<7nhorx)2I{qULHuh;^nkrmv4cBCcgoM1p!cqzP8|ORc(QRB>s5QZg|ESb8 zH%Gw2QrT`K1lMW&hdlZCF-Ja~|J}#6A0#Rd+!aA^F}9kT8Z|8~E*_rfw{O_^q4{v^ zii0B-3H8mlMr*B)Dis*6a72nkmpI@vyzvN1B|mS6hoj=-!!HzdW8LBwXex^`AH)fxZ@U# zYu9agi?U?G{zh2`&_Y5(i#7SyY($VwcB@fe-l#NjQO@+eYY%5A$zAyAHdq8dFkszy zpLgzFO!Uu*1m=tctkRpyEN*3?IFoI+sC2bax_>5GVhp?~)dTr?|7*UyOB4SuD))fA zEH>;~C&$uvY+K4cly$_iBAtjz@U1|(Uz0Coj7;0`2m-W<5RL}IRRX!_zeG} zAi6X_M7Tut=?e3iZLTr0Ki1!y^RH{GNB+GZt|CA5kv8%^$td+YN?o^qMN zAD32*x`ZYW;&^N4+qQ)IL~h?7n`Ls;i1D>mHJ$UGcH@HXnG)xZug8l6P)h#l`GIYv z8Y`6Uj1&c37~#X%I(aP+5}ABju*p%Iq~ZEY4NJJ!#wc4>qg{6>jgVNo0HId*nzBIl zE^aj5+T`3%*so|Q(_LW9c@<>y7BWXX=AbAr@qRjRqpX~f^!hbk0P$C5VksEAhP%KZ zLm~T<2m_mww}0^UEZ<(+V`;(pI#eV&3SJY+T(JcrlBSV@jIc(VjtP)m->h#Pv~|T% z6N&gub-Z1&cHTH*d02j{s3^A^&eG3cN;(690>2(bxjnM$rB?l)L(P7iwf7X z>-CWJG)7WYD7v5sv-ys?Wt<48fs4591lik9C%FnX-&=sv`mx@Kd}L_+E1*ZW2j%j@ zv9kj9Vjn9@sB5GfTp~5eGq!eyC=?VLN>!oZlR?;?QkSbEuQgPPwJcxA3vM`RI>A&X z6>Lv6IH~#oU-i*=Ddy)nO9;*)nS`!t;Tpx&d?c{$E7ZFcJ}86>&F zOPu}W`jL@{D-5x1bI*SBy|$V&A;Nu0;hy8B%*d_zTO{|(f_}`EJsy3O)Wbshkv^Uv zz!@ez+KIhNtH7;oCW)`0_w9ugisOkaT(+HVLo~#Ieyu$|J-i^UC)9aR zcC?3pxK^iJS|LMJc*?Oa38grQruV%;Bd3-F+v@!eWQ!9f(=7&&(X@L6EtE0g)j2SY z%=<(Dop5iGO=!?-0jHvDuYOKi`32FD4ua%|6Dh&vg|V3GSk;70dWp%hYT4CYFp~l8 z=~F6ryxW5#RwqM2pU<*8S8)J6%qZ;eiIhRE{nYq+Oi=Ke6~*mA*yXLS2SBJWfJ5&y zZm;bswDB{!hKN@T;+b8Y)lx!}^`o2dow2=wN)cW`RBH&qnQH0hFddFW6LP{1tFv^4 zJrTM_D>MND;1<2Zo;SHjxA!tNYp#w&fPzMkUv}e^0Ei>q19M}0TdH3;w$X|kd~xy_ z%57!lqU91GC>#ipZ=%(jcDl8Eym(Py-}-u%!){kpcEK?JUX%s4TBbIV_P~gg0U$ZI!zeY+jJ3zX~UpGqN+sAb(0r)Ab2ZaERwcimNx^>lw@_N&#D%O)a2R4?%C z;rhN;7DSh+u=z1=PG;JvASy!hVDh94z>^6 zMY}mh&Uj|T@fA!6BkK=XLrto+Bs)+Z(RVEm5A$$Z-|P038DQm`=kq|qlmKs)_SO>O zgEFM3`l_1e4m{a?*c4AVtMxYu4-qW`FeSzs85dV$=^-N~>wnJbMw5n`s+Z(fY)k2q zxyy*+ON9r@oHE(+5|Ybo#9*58EZyW6tM6jti$45D4{d&}+4HVV}PWpt_xYwut>!6ZVd`3%Nqy3@|WNli(b z9g=C?0}{`HkY6r>Ue}Q8jpr4Iu-gwfF0u^jUk^8-M;TQ^H4Z`o-MMFwnsA|fy_u{3 zD3vR?)}d1@^uO+0&h9L(Cv(mL62yzQPETOo#nnN08HJaq{!_a<6g?^{qb`vA4Sm$% z1d|^pWA}TSxgR_AE_d01SoJZ=rz@cI*0I9gX@5IK)bGO+DH6|(X^l4J(mOFKt`FUw z*5n6#g~lS^6Gv?X-4E0FGhs`0Cis=WRJq1rbk%lmAszWcxz3TqBjlYo&0O<0-eBwX?Pv8md=?Tk#m&Ij00)E=BS)WFaZst$wRlx)Ip!Mo->euKormuKayxX#rCtS6e? z2^Y#2-gPueXS5e7ddn-b*RCdzCoYC-on9>k$M+*7vfh`w_iN!d6&fS^o=62K zH;aj%@jeKkqf37``xt`!^AOqK zCA#U?!L;A2lJ`b>*UKXKI3(x^s^@Ls z?zU(n@bTElt?|whpF=@~K71zvF7srXsKEkNEDY!?MyWeVXRl(eabq`Gp#N?b&bO%j z($|pMF06CEO=G)4{^vuO3NnK=SsU7?(>-+6pO#WYk7i>5rG%%@@p#6!XZFL>_p?!f zLzlw_;94ik_emBmj$6z}5k@KPEW5^1yYcTiaT;H_nkrOxahuuWiG9!o42~XD1HOfY ztnHRi&GQ_KZ^k(cRZkMseIy$^<09bj@fHbMr@)^#9!B}4I5;Z{X%5h);Uo&7$`^Tz z!^O|i9hOiE$!o|oX>!!>)R7_DN{wVx-B8UDo_Q_2cHB!)=`r#h^iJ-_mBpx1CB4z; znEk1tpt0+?#SyJB#(B>mZq2+(PIMri3s{?W{oUsIQP}inV-=kgEtBhN+t-{{Qq7Em z)O2*Ya|aU#m(&NGO*6;hkpW`WKb0;936h^Tzm5s`bXID(R9lR`-7h*IYw7t{{?t?Xs`oj;B{wuk^X|ZFD6bk{pI^B`IlH^ zX?9X-B9*$q9t(d=pV^)SuEbdz`39m&GBBh_#((mBIVskq>Vy}j`cFZiGsr~&)Sxrb z*Ui>S5L2jMvBc~Ix?O5PLpK$S& z>=3oqybmyBxaGrpHWlmHhLN`02E_(8u&%dePT=21nANI($Np4Ln>e*eTvhe$Q=w?$ zq%)@fxu7-PFRwGf-ML#lKbri~7QD7C((Enh!x`&qo>H;f*5L&&ii`qly5_ekX=5~8 zm1ml53KydBq-y1vH(qZTEMmUo28XwTMQ1_|ugn}sm3bHFHG{&CYrL>8&ENZ#b_Q#z z&A$QrD+n7a@)6n&Oy634fP_>sTp4;Y=p-Z*1tm5DE>owonygUl#Y(vNw8yF?ie9ALB*u&0%YlnpRV7+{R3GDa(`q1@OpFQ zM0IGG0}*d!y$(l?eY7JFqCY)~Cpq;W{K&mHKu#IQGGPg~nKXZ%HO6H=5s z@}2?3KzX3WQtTX)e-KUUv%Kbj=)%MTz0*=KvX2hMc_;nxy9()Mo~rL=F?B-U>f=tT zLoBrYME}(S@L4&Cz@mPy$&yKnA`;7LNz`l5@w%F_2yn|494yy+wwu$+!BUtH!x}Q9 zxVmr8T#&SUv}K1k7TB;mPRfN%bc{FTb3$mcZdkG)RV#0a*;Ab^!%IJa6&~U;csRVv z95>G`M6duea)Nnf0Qp(+9lFfyL#436Zw{hppa%o~%P(JoYr%5D7D(nbcJGXSSQFT% zkWuZvS$i@+*^Y1j!3y4#T?M)grt+}3O*$qZeAQgrcm%q-l$akf{_dD+43eQoB$3b< z+5Cg1AZ;c{ZGSmZJ{`N*SB|i-S&4_8K`UB@PWJ$^l>fKln2o*gj^+rF=-MqEOML_y z9ix$%!GC@TJFk)4p31t1JlHUr%bqEhF0@Vl)eWNQJUX zG?H$_q@x;Hy-gdA_p_jz_qZW;D^YeW*P!@dNEcY&)!h`R@F2{8N1=W|s=Bn@p&q>H zZMf)6E}aI=l=1(pgjQ-kJLOA56*!RF_k*K{U zjP33EW>`T8l-mR{)biA_t~OtG-`r`dZk*@dVOaEOj9DVAh28J47!Eu6es%l1oJx-e z6B}(B93=WfiTj;PN)Q8!ijgra$uCiGjbW_+e9q#_M3B|PdFt7j-&(iqZ3&lGq0XmD zk**tKx!vkrPQ76I*wd-c)C*8l355lpq;637Y?C+-;pG9HO`gZru&1_-I0C{&uy$I~<;Dz9Ta?gg)k=H%`9#l^f_ z<1aU#-VexAFn`$a2L}sR$Mhy17xq-er^zvk<`1b&@Gr(*c8bX_he^=-xe)KG*We+Wu>R zEyJjs2d9gTtwJU`Poj(9?_$Ht0*m%YwB!Z+Oa*mT+iEfLi_QK{5sdMXmG-BHIkP9 zV?xifro|jil?REyHN@?y+rQoD>KT-4dqW)HJ&_Usf}bI?d&%%;d9T40x18P9tSYAT z-Dcqo#qr$9DeS7?bK*=9Qy+Vc+|zVB{kB$2Q3tlB2(y)EL;>qI%n$o6ix8eYvRpRj zcY!$yHfcnYWs*9xQTi!Sbw|QOk~I<7`xJHsNm@>#+kJDdN8Qd69ov zrNw)BNACn#9T}Lheu?BonY6TurJ$hjz|Z(q#f$Kfc*nIoGh6ch#B8l9^7zuT=y}2M z#p4}YgBQ)Q&0cpSdZvp2D-*JX(&kecOR-u#xeoKxI*UJaOuTK> zvbf0(-i}OHSLn3M|9*kY%E95G&sY9$IZ*V~9?!${8W%#4=;%}Y{l^CwSR{@`hdg{| z!7nzW_M!5T@Jh%Dj8#j`qdrO`v#AMK^_l z?`0I=d;Jc&$VRf95@^@Ps!i z)By02V`$uIP`X9>yJuCL=TUM?ixxUiFaz^lj8_Wn5Hd^YqI_8hOUoIzHzRF;AKivN-`LFit zo?(S5^vlT-6O{hSd0owd>8x3c-IM<=X+f_) z>D8-0laNH#)zx7lzz*c%>D6;1@*?_pcf2L^@R1SNlBt;RUo)9V8>*|nQBv-ewYpfH z2*JSWo&UYa8(v=CZ)9Za$5^rzp~c0*A|m>xrc}=K&h%?v)N4#AfIwnp<@NiFeTo5r zh4=Z{)M}cV7_fU88Da9Io$0!3r^$-Jj_vKys_N^(d)SbFlL6#|MF*xpWp{g~4TGLv zYIZ;4;0&y`xVm3#yoH@o98etCBFdyjf*lKsiD6)2`Ri2pn8qZ)mTvgmE`FwRI|BcJ zKr?6;C4OQgJlxz#u(cuBh5ngprlC0u@2FQP=^GwKqNAgOdDfbkm{5m&3;tKx_pbBM zi3c`S9t2zZA|>4vP)AU&HtHH5N2Q`FXsAw0nW>toa#LI)RJXHZhG_=DHb`X8AFmJH zrto{ZAmK2E#l%cwaMz)6_#*aO?x7&`U?lBwo+@-DCnaS~nJHE*CcOU-ui!h*wlP9= zle`4b<5!GCb2{o%mKmRxP7(?R3ip3I+OS+%3us6p8!5%vodGVqH(5gRScYf%Z#i7p z+1}+VVE!xcj{kp@h5soZBnaV{zW*KYf4*;N8^6%XMZmIvmKG^ZHn2-C>hFKL92+Hz z=5%#;zk>x87Up!flRPF;pSufE003Y#mW=gZc~usB&AS1U9l)ej5=RpJ$bAm;sFo0a z{zytL3cH1y?E%zdYkM0J4Q*g(2+`8=cxFJy1Kz~Ti%+90{$#O63h`f^zJH!e2bLvx zFcIiCu!6(?r_<|@pLu(GyQ;1(?l0EAB>|SmR%{g&6>zpGSv^Xa`jL$dV;=4Q zCjYk^0~`Jr>+O>J;rsk9z3(}HQWF33K5K+S-~ZFzc}+d~|Jpm(kpEv9Wz#lCMCrJX zb9EJh#*rMO9r|w81N`i{q&n%!_xdS17SBXEoF};z2{=F?8TS)*Bx!G;S43->qiBQm zo3-?GHev^1T17AUa#>ryf&!{_#!r8`KcA~+bSs;1lu6IB=vIHz^0wYS+AvPjdA#G_ zwlzbB(t2#&j-eLm;gPjHc(FP0udCd47rZDj9(6`}rSGmLZ)r}&;Ma*hTRMgpJSL$P ztxH^F2#rMY7cGO~8}jfv9Ll>tx22cw@)yk+0be4)?)p43KrrvA$ll(TjJsjO7f#`d z{TBRft}ON!-cU%wQ*-+#|MlL`Q=$OZ*D|&Evx(&Uu|Y=YgVkq=1?k&iGR0$|AE8Zc z{30PLc@hN?Y>X+8vNlc;(EAP*>!I@vUn!F%)zc=f@r7P-xtau?fX}ZXK{50xfXKRm zaSZ$-p@KTf22BxspL?RTHtn?}h%Ch)P>v?^hLa}vHHV!WMA_t+0v_^}tWjU9$eqc% zle^WgT1J>O(|gAsDVy^L`=q2kGtD9J-LBEhpDDFlr}bx`yac)4vK3gTHUZQi6g-_| zm_{a$M_ad%0+gY)DG^^Sh@GCTaRsQ3blHYquDG20`=pE+vhnUc7+a)Lt!>|pDetkJ z4o0^y$&SXJUs5?P?SV~~uKuWZI6Ua17NZ>rBYp~6z2U8SZ3^wL!`5xE5YM`;WlTRC zSfd>2&X^i=WrcHj?tk(Q$5j}k?U@MYZivtDQ6E{yc^BkW*wammb3dYzw|cu;v>&f5 z+bjn4?#Ylpp)N}aYuE=nF%^Frc?eq8Zvgip%p~D0bLdZqN0>>u=7O)Fdm=lU=u3r?)5S$@Izd@NUg* zNOlKL-~T0=`glol(4&OTUA2!XA$d8o&JCUkI~R!N8ImsbY(58*@U3u!F$J8EGL{g; z0nKHJN7SAo&o&3+Ga9iCdajzlSAE=>=z^VNxUs)P>~3qjI5|Szf91YF7PB2N*i?*< z{Iu{UD$ig|Iug?RDAg7JHx8I3DbDvMD*<(rq$s*X%@~Q7np#bVmJc7FNVf?A=!(kFbviP|+HIL`aC#Pl6A=#xUZlKZvtHTj#~)s4F% zz37_8O9gmSdZo&0hVYnRL%eiWxzAhHZia}`;c+HCCZWLk0q9qVJk>l{WajRbpr6;&Rh- zXYx0oE@+-*+wVf)@i~(Ja^gNhaCS>RhoC0~%fpsV%TgsGB1Y_TD92%mE4G~M*1>Dk zh(jqwQiuTl7a9bTt8N86l8a4-bB(OOCZ11speaG_Mq@BoDUC}-3L_tn^7Asbyf}^Y zF8ZU$iB?`_IGiYQ(m^a3?Jg^+d@Q`~D`Q{t8-gK_H*zFijCvY++HFrbxD+GOVQN@B zuNR>)!wP{XCrgpH%@;JaG@T6PKDpO2vH|?)t}QjE*P?D zn$jT`rtabXe7K;s9FQ~a^M$N{3k?z)-YYNP=E_Z@8*N^CUg81ZE39J@Z{^;gIbs&L zcPLz63{gz|X!RgO=I;Z(zNbG)2HYgP#7c6XYtR$*y^b4`no1*koH6Hsapn-*CLx3L z$;;G5bPG&7oaI`s-k86tso$SJ1k18=IP$NA{EFiN=vmQ8s^ywb_=zbxJQ`tVpi)V7 zXHuDkp79w+laqWF`jQ*7z9*$*w0ohs(Z}~geb~Zr0 z;aU%z2vFu<{JSmZ;SbGzt|pKxB}C%k3#DP@w5Mf`CWoXpCRhj$Wa)04=4Q$*NM{U_ zaUH%;#5DI;>irdE6fSO5J{hmH?+0E9ML1K5HTdM8uL0)Ig8@QI4os#cxncXFtY$Yl zHUgFT-|H&O_ba0}OU^d<&3G{g)UznzD-EBFz}#*j%K(j3?UKJX8{aiz2a~e^BQNRf z2d-*?=Y!s1Mb9Ua9gZL5Ao5dNi{2a{U>$4!mbh+y#6!$NDo^*D8kDw6BJU_Kl>Ms# zJ+w|Y?u~}!oI)N4k{F8={jZ77tYCw+_!zB7r7G9wL~QL3lQhPy`bYL4!@5&EMNOm>N`2Kf@yBBX#Z^SCjS0AWHnFh zEQ4ppyBW?Gpq{>%#(10XnZr7(v3L>k7`!adG}W}YEXb7$xFKB!HY^OVFR2{U#%HDN z{&kf`321Frw|HFVHKb;3NZpam4RKyWpE>0D-OdhbPefW9*r}>nfW6X%)QA)e;O|Glk0YTtXi}_{p;-w*Y(8j!3?U`qL569xzcXls`8=kAfyw%O$#+U zEldm|0vyJ-wHwJ4?`m}=Pq!?PEXM|LAbqiH2HAE4uI5vFjsUwE_dyZ`uY0XOC_KY` zM~QU9>1obuuA(GPlp#@0x+3RJQ#%XImBHM{)6Rx}YH-stZnAv>9gIH2cu&v`D1ind zxrQcs#HwGE-tW*0tuppWDE+N3`)C6%4BL+nKaXT>rjr$Z>H7(zX#Nn5c&}LMbtnUJ zP@)UnRGw?qTeJGomVU;bo=WAUQxnU$Cef zE>wHeyjju(!<#R?OC%Ro7RFC1I!x`8`5lp$ z&<#EKEHv!5+w10*d#kx{rAK<88XlmLVE9;Yz4ovv|J zd^Pc3)}KQJG6Sp(1{1ay!Q6CM?ROE_aWxL$5xHypO25aS{{Gm??t}ii;N4V5XW*qH z=?LpG6JCXR&`L{GxXReQz8cQ3I)eSdKIteIzSv9us+rcePe>Yk$tXL4vc?$@28B1r z9!`c^^^Ul-90i(@DzVbdqXG8o0Jm^J+RH&(Y6g4Us=LMXTEoXBPt!ZY1L=qZa;~8H z&*#>f%&iZ7nPv!#eDb*xb*n>#3x zY>NeVI=9v6CZf|L7!Y{2yBjmU4s#wSYls-%9+1W0%7de)pVXixUwiZmh6vp^%3$!F zZZ|pDv0%(0fG?`f+`SugUbh_N#EB-X_xqnymCWz)+K&+StdLl4AnM;K0?2Ya0M``| zBgoNMWmX30?gcg9m0l=bwtrgtPohrm8*TzGS>=>Zev|%`fK- zoyI-5)z=>p?|GT^%q%5=d?ODeV%03-eczes9#9O&>{)5Oc%$C>wa9Xi`epr>H8m#|#@b(Ezk2{i$a(HJBc!yKEZIPU3?(xL! z!;LrcL?L72u4l1U%4eE>Au%q_flAc5;^#^;Y4@h5z2r?o10e5l#dHr(uvK>N)#F1Q zn8rtpU~t}!ob?f}Bd^>7LwKYq;m&dqZwYNY1AWT5v4LnJO73*hmHN99wEcv?Ce&}U zENuNIcl#*+`Mp`D{PT~nb0s7=UE3=O}YHFdKl?)!5`#oL#vlV*N$qrcjy z-R1lDETYiV){%*tum)7JNSI3{<}Z6~sRs9{*r5HDn(U_3R7xJi)B6EEwKSS6PvxLE__LjgzL?Wr|f^RnJ;B7N4@&?}-CLV2`tK(4wqk+bu zbBtB^h(;1?Zq!sUs(aru| zqW!wD9uOFNiCwx%!&{vClxOzY=XRm9=R(j)?FC+$oubLR5{LI)xjltSo?CKR0H(}a zT4jdt(;e@XhRehGp=wnrn(^-Yc^^g0(y{sn!5~Mxz9R{ zswELE(DT&DA(6g*=b~c&5;^k1Zws%-CW4rj7T0OxQ5q_RGkr8CAW@R_{^Fihn(Z_P zVzIs4u4sFw!IICGjxo(SXbz_fSm;YToJn`?8HQS#BK>^ zqCGz!^G0~SGJ%P}OlDzx;iacq%QGXk`qAZVB=my$r|Br-ksQ~cuIKqSi0>Kj>F30q zz+x6$*(u~!o&N8Orv?#T_plV?VA_Nj+l3L6Rw4RmZLDy0*m*lu9QCM^639i!3Lk=r3w7tXl#ZI9mqf2aK!FX}M zG1?HF&8re%ZyYp3$cyH@ z{@Q+fE>YJnOo*9m&S4?seUgrEqZl8*!DZuYblrce7w8RZ=AG&C_<`bRymj&oO98-r2-dH++3{pHT zf@4*1O3CuXIEASC_uNxUhatcCQf!(ZJe&MlxqSgw`+(%dySiXn>(nMUO-$f+S7;+> z`{i@NmfGm-PTqa~-k58$YvvUQ@}BBjP{}Ki-w`*c{AFM#>piCmv0fiER0~Cw#)ipW z#_KcUsPMb-_t}|!b<$Ha>)6yPxCBsi{RSHc62o_(oDs3Zt3?py{Jp*FTqPaj@6Kh~ zL~%WY4}sa46t?P|+f{b?g)MMe!K0HcGdqP72~eN?K@v_yAJhECsnWpP4^0ya2Fb4x zjb8+Rze~mS&zLA2h`|UoDqja=-d^_=Tp-13rHc@B=vF$?1)BCHvmGiTjj%yesEqT9 z@y^(AS>h2Qqep07GxoPxC`bOT)O|mC-?KP|M*u~h>T(?##7_wt@qKUE*(6>x^P{ZY zO#&&API#pyNZ`UOD6POsq52IiD(0Xe0x6Hd&au~}Ml@YKrWG(+JKhiY5jQPwx{8CQ0kQ^TB^Z5&2_i62EliP6SSb~s* z+(zt9Vow5iS>CNbFfjW`^S9W%4G1#TsT<%M^!@rVgvkHKGTddx!C7NqE(nHJz)W)^2QXPnz=~khp5EuWDX^ns{HcpH50v);4V7noS~W z=S?~K=qxo{GE7baOZNURy_9~)&YazrN56q*7pyw4O+);iu_IWeh>sx=A2tfl$Fi!@ z`#46biB7dYVLIMh)k`fweyD_5OHP5SddOawmLNlb3aD0R&)=aJ4Gsaa-Iv1mRYx@k zkV5RIhgdOUI{nH0r2K>n2`0j3bUqrt|;+ZO9t_ zP7f&-_hiuB=Bzf2={`Z7?OI7K35T)+=_m2rV$ccwq)bZ%NGlU{6 zSiHcA?~BVk_!x0j0KiWor1*t=j97LqkY%rAD-VgmL|0ELpMCmt0czFe^^bb`mGMZr z0{H>JHh1v;m^w7oV7g3r6g^b_YHD|~>G=Qx8Xg;X+Rt6S6+VEyzR5?+u?}6o$xF-} zuw+kJYx|@WnBC!U6klXrA>SHCYh#&A9*(BMMpH7{P76`$_YxK^JGi9@a6%a+0#Ub} zV;y$m++NWWg8}4fOYI;!#8_c=kdN-x$i&6cwHsDGC=bmRnrHtPYqZI+i`vZIVFV7) zzrKIh$cfP*Su<2);p1+6$$En~$sFuH3SJxON_B1{-fF|mFe5?9m<4GM1+iEj+29rC z0PqNEbe<7GKh#n>{Wk;hadps7RU*%WXlW-Nl7%(z!`|jI;8^-SWM27fiLefrs|X&p zlY0~U#-EvNBy@m2sd38h7lk|ROuoDS65EHF!RN52Rsl^W2IJGBcqi|1NA%bM=L+&% zfktMiTXWUCOph}bW^L>#OOj2xxyp^DwLR%s@N72|#`5!god z+Cxs~h>!-k33!uR{GNJwF)xJIQBLC;$Cm_K+?q%I;d|52fsJ)Fb$4E$ll1}r7#yzp zOB`9NC5g^oc}~LU4J}T@TpFp%ri1|=kI-6DX=E6 zBQnh7a)#3fqa0_<9X7ej6%WajLTA{6oWo zBE_H1@M$#%ZoIO5BxhRFzAgq%RN8~{k{w1+0E_W()WA1HB5bo7n#@mB8&}#_GY=78 zVsNT0Zao5&#v_1RY=IT#G-{J-H?EB9(s;32#_kj2UQ_Px9t~L%h@=}GS)3z_H47@T z;Y!mWxpRKS2kGJt`MdJ;0^Xrf9KLlFyTt?7C60?imuVxKN-4-L|svC3F)6fl@inkQOxx!eVK7FG9dC>A>Mb15AzMRi(&%2oP?Mj$T zFvN2J;ye&>tqxw@**@&oS*G7{VUGv3uKB{EBi|fGw}|1aXD)<&LSt6ZmDx6c;$zrF zH#OggeRT&(3^~p3pGA1$#>o_;Ik0i4V5h)v^c}6mx_nX0uCQlV_}L`GoyJe>z>bpG$f)36=#;1Fb9Tl;y{W(cN{ucR5nQB-q`vnqSPK z9XF?n5&AwgXa5#yb>cRuO(ljJ=IiLXqg0WR8IxexXz?2wT0RO(%er-$u5dm$9(gMU zlFae7i&Pr?3Ctev-=|vtARST0Fr8NOLX?1Z`^)5j!PfCmP%?wKI`~V5YwpNTnfLor z9p~=5i5A(mAfmJ(>vFx1(0xhCjLVDBBDRc)#_pLguUH2F5j88TEtpMCyxBe3(j%@^ z#M@jwi>q8#Lb$)MFj*MUVk_LEjE)(Lo<=tZ6I;{8(R@#}c2V``PKl{op&5;80}J=t zItgAcFhRHPSk9sKbjWZrYpIqyo})FqMm?Rcr1%l%w*qWb13_uyYGR$S2w|*c7U!a5 zUt5>e<45_8TY{A(wgB+`(YAv#foAvU8!TlBaEyY;LO}&w{7yBy$D-3ko@%pT= zSZXMo6=#U^uN`x3hIpTvUMTUsWq}VfZawK%8kS1C#A2#4CPnR}G8TO7ht!MFbA23u z(_xl?Taekwm$CH`PV^9>*$yjV;NV5bds3Zv$bimH_ey_0Aa)M>v6>AwOM&HD5SIR4gW>_cGf0tkP=1%*k21vDJ!<6rbVD zxfpcEUTZo#hph=m4sWBs_REURY9X72gS3QPYv_#mK4cm}bJffE zPS#x#)x8Tav{`w@5}0--tySCA+oM#n4kOjyyqu}u`n!^en``bAknc<4wb1;Kr> zW0Se&QHFeJIm|C~&Yp284*OY4p}INuv9VqFp{hY^nle_4J5Fju6wq6bxBKqJ;N}6c z;KkzGw_T#p<|1LD%iXhf>6eJD6mW3rKPJ`W-AUd1|j8NAQc3)QV)QnPoW9RQ~fy;zV9wK;v&WkkudE0e7)Jo=6A z{S%@%5zn^@wz^h@K{_J`YEp7e^O$|Ix~M`|0Kit$Y}UnyLiL_v|D z9GHWRD5cc6v&|La?oDqhO_#o&{G$Eb;7zK@NI8O}6V&lE*&4C5$a%-`?rnf7XT zDvHNwL)o4QswJ}hEmz|iZrsv{aroAl+o?KtiiH46Ss+OtQFNpI5gVxnXR%At=paXb z8v_=vfG7i=pkL%@s2oupfx}&iOO>eeeNY=-Q#WW|Z-oLQ>+t=JgtJUL??cMbByrZXu>RG%# zV1}&qhgJ390l|QI2c_+lMKUw9oJ<%V`rDnqNQXW`sK&>nYm-ZImZ)>a9EC;6)m$As z6F;~3vU#iPf^KHF8$;4RT~T`Zw-7y}p{Hj&day=Qy#;13jb(_BG-+`r{V>NMJ}S@w za9`pnS5L^gzfebpjF6hP->FhJ9kYU`#Fvz5eu*65x!A4_&*i8pd{W+;_GQIerxZYn zce^1G-rhnb*IDepg6b>buf#>;8}doNnLHH#TcoyAIHo$Lcl-o@f>f0~Lj{NTFm|ji zSGseNuk7twjc-%w+Q&2QUu4#%`L{R`&SY~yz3T3B7v+}UKagv`k5r!)(tHEMPGqS2 z;Yu412QB&04uSfm?X^C|AxkqqmZN#2iy4iNHr;T8cZs28Hp-sO6+b`SPS?qj?r5qf zVS;S|AV&(m+Onb&>hmGEGYvu4EoS6f#E51>_4b~hX@veUd*UL7K!IX;id7;Y82<4L zRH?y9$#DNm5&bxQ*>#^;;Y{DZ{htUo8Z`|B$I)?HFGew%-~3go?6VwH5@n~4ycx!) zl)H-y-6NUW-4FIU%Lj0q0Mhg-d$$9VMqk>2#*lu;(>s@X>&1z_pi;Lx@4iNOzwm^t zeG9>kwb?;ayfJ}eReO3(O^Jt5`f8mb0CMYjRnSmtOr}t`@V(?dsIn@_ zdo<6xPsjlh|gXvijJGNC6FMItax zBHq>-|DHr=Fk`-i#mPbyXy4qUJ1xfSHOl4^u$$Kv3BpyHI&xnWX)j4g$UQivPjvkA zP20!7C`XYz_0GSlCjfBoV>+Y=!w&OnJDS}G|%3Bo)IEZwuiWXl#cxlrk zu3fT1=BR6NSyOc4AbKvA1X6SFjTupw@AEWeP2`}4ycGn}?cj13zXR2QG{RfCO8Y6) zY)fw$xPpO0I8arO(jvjd{!Y_}#3Fxc5Urs(;<5!_3-Sgck%X ziSfIH!Q8hvp(<-RitFXEEUe&mVeL0>yAVo0VY-j6#Z5?kOiJez;FsgFoQ{8C5a_1a ze1A;Iv^O^(!0f5z7_uzvXtC)K(JT25)S3k-C@yB2ZT^Rv%@Xv_eZLMM5pGMx7iz0` z{Z{m3<}wP6k;HOXLnPggzC)XfltkWiB@1F+0F6=*&;m{lW)t;&G(om68OStL^PfMF z0-m5O!nWC}s|u;rHp8`P∨PI zt(zzt6EPCRu$;jTN%4;29KjN7T?6-Ng$Vx_i`i<4`MQ2Mx5sH?${-p;JVuS}8Rh?h z*dV=y14UB&O7Y_*!j^1Aa_vpZ$W+;~THChe@M~bj3ep<$uQ`l}EE=^?##3Pb$%Uc| zY`%Cy8EryUTBo?T(r3s-uqsF8ktezpQ_v_V?B&0xw_aW`hLt4598E~oG(U$d?ntBV zxB)1OyXSQt4($^S32E?oN&NKB-wXj7jHIuX;%y&E_tQQ*Fd`O_kpAxd zFRoCr;~U-oAq~&zCG`IP0d^AaK>w9(|L@~++#|0g>OYaa-sAWi#+|(ihfJ^gD+%*< zL!0$>2I#lnsjh;9P2{;?7<5Yp%Zk?dXGskQ#HRnG9MdQe|0Oy9`MH-@3jIIv*dW*V zrvJergE)=NfBHV_<0Dg>VTvHZLiw_dTpWUcMM4F4iezMEok#vZ32prnYJ8%02Kd&5pIQZxlM4%{DU#-%*Z)Wn?0QWFJrhB z&U}Ey=CBVBgEEACH~kIzmSEhD6khYK@7a^*gGkOJgN8#!>^b!2Y|3hppw(PeQ8N z-$=(G%G%721z@4JVH7;~>!GQRhl_V~fnG8>JneTgW`j6w@eY)GLsGTT6E8hu->{0z zN-{->j8$G;*ors(Y)|*R1Ux=*qKErb{6k!&3O8^#>n?zU_{J{3y8>y80QYOi$)9iJ zp401{!=AsloC!EG^F5CY5NoOR{WM`Onsvd_H`kf>ktAqsYs2hX32>@(wb zyy13cOz&0>1x9eE^iLheP)*2@_I|+aP*w)->I23BGvq`ULc++a%l z5K&kO+ZT&|VN^)z*wa`ST@kR2Vu{Uo-H{V*{quhBbXN$kc`Z5uL#6U263=PaGu(K( zVW{_q(Qq9KM6HgF8Z;sCk$Y$ldhVk*o*BHcqf5X|kbcP)YYHmX$9A)DntSgKsHe6*I zr^aQGd})uXGckQcC(#aBZ9ahs;Z)?Uj1JXwJ!V2QKR!Q#Xrz;Ffb=@Aa~+#~FQE z|FB)KqpgO={>8NJPh7SyC-~R!0;Lk)RBb-d@R7paXn`w8*AuyUcT>mKr*AIDQwy5e zSE<4>8-VKw62l~A3gG?6F9JSNN=CBuV{d7M?qNagNa&nN(KUire=fc`mpV0&6~pDI$StF3smyRvKOQTL4BG44_snKk~bcQ&7N zoU!1BETxmB)GB`a-5j92k@w}-toCL>Jie;*KUF{`99#kMSLwGRb>9(J#s^&(4z9Ot zty-L3llCbsBLOSR32QXQL3FrII{K+f&G!hcpaP@WY--<@MZ38qj^|`h|KgW{Z|sLtW%vp$uR8~?i!J9D)Wv7M z+v|(hYjC9SA4$IpGQIu%wJqf5&#%hxn^{7{6gl>F1l{4af`68yX0@$bb2 z@y(-TmSfY_NeC3FC6wHDZykUWju{^~YKBgdJWX82S9oLcqCM!4%VcD9tdD%T^%U~i zi7YkIah!O4*f!(Vh3qbG2Sgaas}3@Zm7cno(6y&1M&%WI8qG9R!?oW18{(S}0_{<; z7Bh)*^A#Iu8aI(IlcR{Hru$DO1XXD=surw9ZVaQ48{W2LR1+bmsr9s*W68NS^Fg|| z*m)5t#b%4IU|yZ)9nKz(QM=s3NqaB0ath@U?(HqC!Hsffc{0I9P>3~kx*}uTq)&$1 zNlQzLv8B?I)3v9U!s(`=mfqL0CFgyC7?%VKrvhqYQ~{^A?pKF zWvmAtE1Un-NzA9hnyxu?FVameRuzAAaCJk;q`Q8J_IY0}H?yj`?Tyir+Elcy}9sNiR7PAOg1&5hrA zgtNtWVQO7>qVciEjX@;0;XWtN-;_m0poRpAY`LG>2F||bFTGK*UhU8RQ3n9krl#@7 zSM(47%%4ebtR7+|H%F41nS+qSH%%?Oi&<@YzM#HV|3QQr+&S#gCV`tRNh7EK<_&;qUka z;hU6Z?L01FWqOK2L~5> z%#A^nmoexu3M<3gS%STJA4k28ffh<_S1_H*fp6rK>7*GfOADtlBq_qt{@oScgx1sQ;V1P>tP8)_9fVgxe!f z8X2@6neKN)WT6+KkAHS3z*_k9H6c`;Q$~qPpN3KCvW7uT*_7mIn8NN6(C7B%RTXA_ zC^Lj5(5CQSQ-C*MWaD*`&DMRk?5!>Gkzlys`Fuf!$>J|MHe7+kjov`WU1^!Q{$)?i8)RT!F0kP^t>052T>@dU!toGdlkIMnd?9lDC!a6uAS; zNGG=)7yyi;PxsnnzeUcFTa3IUkXmqk;(Fxpkt{oyNNU)Pc>lsgO%qu01kT%dCti+I zrfd5Rr{3S~2h1JGoJo;&?5a0gf-HHAFrR(Xl+toqpVury(C|4jxo~WX`@~i z_A;=%gLJ5LDs34CPqg=T`Pm@fdr>{sGh_;#MUFnYX-;_-1;5&milj*LRBEbJ*0j`eTo+9?9`cY;ut8G z7J7jU2_wKSWlbO&%vU*Cpb-muo5F{$1t-5AO0AB@5b_z~8W-saLg$+m#*_e<7zB?y z138Uq7K}tY^Wr|ukv6_9NxuwEM~>k%4V-Soz@BE9COeUyh>(-97KTrvt!=BzDJh;2 zyZa#qr&1}m`B%_%6loj7fu!B@sfy!wzi9! zjLzZ1>>U&(7j$Xoz-*;a-RJ8k>|vPG6b|;YD&r;Upf>OSKNysjUy}U6Cnn zi7S^?x+)-rw^w#ei3&Mb90`QzFELrddSB5$R!1;tF*$ROHPaxWRK?lbM@PF_#V^F? zGu4k$_8=uzujQKpRl3M*DqYw1?K12!vt@TaswYDQC=NfB@%VAk;9*ZJ?Y}{K}xOz>1a3tD?Uc`gC=8s;lc;)4I3yW8?!Q! z%P?B|=#H?R)1ie6ynjqQJnf><-jYHSo9^#eBLfVE3oGS79(FoDmJp#BT&dLY^f!^e zs(*#svM$~I^0<{dGim-i(iqk*>QFG8z1_#@Hb#ye%&Hw37?3<7xg7(>C`2vs6Spt+eo&xI1 zQ|Q(1_!u2S+53OFo$M4R;X)V*LV3J#SG(nd>aAH*+dVxh_d1O~evBe_?^hi685heS z?tV$j3$xXW3^^lt2FjG`7yaMNf3CiYN91>hFGyJH7V*!G z4k^hZxHr2|J{r4=sL;Lo*|<&r=!Q+=Q}@#r;eHoK$daEj0k+iP(7@Y98K2P(^mn|S zy~A;o64E}y?MXq+A>G9Y~RUK&KT}GHB^++}!w? z$>=S}Mu=WV7!4t}5N-GSg8KD13NtI?!-Jm>od=dO6FUYe@qQ5wv$Exf((tm%S_W~R z*~QT6R8oEscpAMIUMjRE9jDY+e&5Y~F8d=~0V1{T!3vklwSYb+mqFf5)>q4A(W#Wo zLX$hr8#3c<6!e~RhcVz!Fe*^(q0ers+j~Xs_twqdAhD2-#o4SWd(Rm{iLbQg;|m< zNN?0Mc{B4Vq7}IU>xOa^t-5=OpoOx0EZ#39yQ$&{GNrhL;}BebB=0g8AoL&1yQghukeOyKa&K?eXig2WOuD z_MZm(Xx4R={X((9qLuIAPJav6u|1i#r}P@QEFUW?@!W85*~k3u)&qKbSDIxU4DTw! z-qSGe1vc~ci=mQpd^(UdX{`%xj!^;RGeLu@0q_rLzHqv!J zJq=A3Hpm85(d?5WULjKuXY`1DLO-mva|T~ty`As?8U<>NpTohfejO=4+IlQ6qpK(| z^98c6FS|BlXkr=49d$E+$e84Yu~GH|gQ&N!G2|Z9E>nE3aJI(&>-^vo78c`Y~OAJol)t88>qB2zyC6yEPMSrjVf2p-;W$^w-fBlfrAemTTMyTw(u>=x?mpGR6q z0X+lEBHtY0IMOS80~@_U)yBabKW|Q#r(yPO`A;5^nvcLt$McDg9JD|iv}xiDp;u09 zhZjV#5ufV+*NBA@>V{P~Y>2;il^xGiDFK_4bCE;Fo%^*McO3B#$wuC%_RIpOKP(ZbUUP_=P<`!bsiY^*a_2I7htXKL)E;NKBO{-jZIw2vFKB`yVsS++h^Mb<#kNO13(%;OM zP^=R4RvY$HGWL^4eXh>9xBUNTd&{V}x@`+EKnMW>1a}GU?i$?P-QC?GA-Fq*ySqCC zhv4p76z)pHo}bUT3W{=blS$d#;!!>Sk-b3vq9s;}W#& z!OV0sbj$Or)_&>BTL}^Upc<9e+o;jPIOR}G)jHqjSD&tJ&ad97gl2xiE$7w^p1pk9 zjj*t!q=T^lK-dOl)Iom$C*N16L-IlNKfqUo7fD(E@hW@p*8I1ttQ*l*>6Y71ripyc&Wxm zAPkhIFeI#e#f~Pwzt$Z1TFwz}@YdUE)1CHH(akovYIP#{{QB{CFoF+%0lgafBa8P7 z9(M}O3e5Zz1J=SR6vm6+;?!otc3@z!5lEwHxGziOGAps~Pz*Vvt033u7o2E#TClGn z@ruo`lr%FfAwx_O>W;q?VOIHj z{5i(8!-a3EAh7r%e#D|k&L20#6W23e8oq*@E=T{4#izv|rBXH#3U-qZ8iFkP4Yt2P zvKSZmiw=8qqNViVrX#Z^IasCYnjBTY@^YBfYoHN6A-Raz|Ink&Hz9Jc1kZ0=jM3GD zjnyk?fiQafY2hw#2cY;-yjxE?TpDS`Qyb-R7Tc&UPvd&Sx`L`9qjhz)Vgrnk^f!Uf zqrOg=D1Kw7Gox3vWOo&egW(D*9zB?u%#f2LckTJ!Mn~zfPFl|`?5`uqP+a6w<3b5< zHfDaoPFGa5!TM&ve-5Jo`L*2{k565P)&p3cJP@SIbc>=`0B15$5)eECQEK71KMZh&Uq#ZVN{;*1crKv6>lKJaa5Dt z`$OiLX)ac)!YOEb?70S_RlEuL^jazIFq^D0eJFi46~LIS>Nx~4xw8J?BA7ya=jKZ^ zwEJb$%YPVR{THF%_yxVbaYcD=j;{Vxb;c*v>3Bd%UtI-|BuJ2Kk2ZZlC&#&6Zmy20 zZ*V!XH^t0G23@8Sk80lro?1RKy%mmHGw^Knx7cy`jTPwUU>8(EiiKv433~qHP+Zl@ zwqhq2Ap^EW3sSM^<*EX2VMIRJjsANa%+ih+r)2tZz5HiXy>lGDft_tQ`C_=!L=!+@ z4Tc8EWCtdUR);lK4ye>Klk~@Rb8GezLMf2GaL4I>+}iSBh7m-qwzcOux1B3KC2_TAAK_+^X(7m+2#l5`o$U#GDy61 z10$VS%+eHc2lvg!fIC7DRKy>)L8Y8`wxkf~c@pFM_5_OWx+WJli4-pU_V2bwL1bWbek)aR^Bpu#yH8g8onDQY4_4k6AiEsYKunGSSu!Win&X*5I^}jda zz2sE<11k9Ok4VW*DkJlB&g7q@@9&tN$FpPaT#3r4B*Z^3Uq8!qM*ysN^}Ynre_xEF z^luT4OPP*CYaur=>X2VXBNKB!^~8+fzV*Y)8wZB9V%u6Jfn@cm{tYnRr*tZh{&ctE z`q!PFZE;}#m&l1H8T9{x|D2`a{ulhWpNz%`Z9+cler75*bVPKVX8P}t z5F<{sB;n{J>|30+`nEXy=(2vfe#YKD@mw ztJXb$ba=Z9_mdy_q>N}^n#p2!jz_8t{Z0;#emF73LzvH81vqEdY8$m^4J1I3yl`EG z<1hP*)j!)+xv>A2IMUxKZ8rJ12TzhROJUFQibS_vN*$iW<%*lk64^DXEtCQn4t=S8 zuQV!pZZZ&Rw-M>UH0B!JPB0LvkILz~nki#9oHWHn+DbgV0I)@Hy^W!&G9UpxSt ze`;Op!&Mm@IiGRlwF0BiH|u5>aHjdXs42#S4a8MK)K{NOPz3 zY?f@9kJ_2W1M&@yt@^z2J3NNwv6xX0+jr^b+qjVZxPLm|)la8sVuXt`)vA1BVG5H+ z9CRR}hR?K$)Fy)|a&ahb1m&yFKW#HbasqyjJhdK-%arQX)6!^jD#=018ljLRiyI#T zf5XU*&aBTj413S7J$=!3h`5k@J*l?q?ewOu^0Sk%J+#1bwA>#ofeX87%@h>#rD;&# z;j-B8jS~a6l>cacFOktIZT0V^-DxE!Vbo7Vz(?;pFxGTA(9Nv5Y`o8JC*dk4NhL4_ zXjf2kJzt4aGq{%zK1{3L4^#hofI*c|`a<&bcq&J{Pew@Zr;@(aRkfrWmb1UidP1$C ze5r@iefY&W);^>-?$Bl|YHF@yGkqei9Vbnt&ch^ghyGoW6#?tQ%vES;(Lfs?LA+V~ z(l=RZJh^MEw4|p)PSmdB?tZ6GiXF2`?0RCchjpDm-UM2`sq=U;F4#>Yb~KldBh_&- z&GdYp$Rm`Z@qBRV<-(U4t#6W?LW|kk0-=}x3Rv%?NxOp9V>cg*Q6@GtC{TyBG8rQv zc3-7!=nBH z0eoh<#8k%}F^b{xlNy2VVA~)vs1>@iOv}hgAb}CuPN!vvjRIzGzM&VA(cNhe)!&); zSA6TXmFy2$a`QZ=pE?tP`wG=re5S^;Efo;5GSuqMw1-=l2L_wWY}Afcp&Y_^kq{(S z56i&xeBjE9(poIjHUD|_m%Uq{n(n znIep^5#x*Hu>={(@80-a*%L>^?xC!TyzJw_Y*;)R;MlWOvZtf#Zh{tu2=qDyt9^aR zLh_B(_V_fiUqzW|{~^h@j}?gf0A$KCE?u0I{PtsD*5?9>XQPi1sI;l`{VNJ%GV=)- z8U1Yj41Tz}37xjIVZ0s@({Wt!g#T7y@~|60yTTN)8(@ySf-}B48udA}lO@sj-l*7< z;G+@lkynf=HPBc1T+hhdk(GNODK{NwxF6}A8i1!KQ^{(OKBiH zWmHq?EH|s7HDv@}#==(pBK2gHdkRJYMY-h{tgH?k`fH@@WsM@2mjEuWpkkCmBNgSD z)^4SDu#yk$`TFEuR%&T(Y#N7^ioLxYwoWqktm`RG zof&Nk7gvmEq0B^Z5Ts%RbZRj%9>J2PsmQEb!pQukEyAsrv`%oHc&r0H=^Pnj6=z2T z=av>Cn~={KTgn{r$F`#CBQb41DLcyO;*L?6o?tIGeDUoVtM#zpdrPl9xn1-_Jw)dV zJ2;IT$lo5x-v^I-U}Qw^D?|pjHD)~66rjpxU$m;!arqD>#vz-Hd6X0L*+gBv#vAsR zRJJPSb{8rPYJDS9CfNDQw)xk{LibZ-zWE#O#%IJubSBVs;wG7|uc~61rcC_7JtT9B z3vVjYSeqm*Q?Q4PnT<2Gl_Nv98C~7NOc< z;Y?*~yHX}VLXN5p+l?!gLjGClZ&}((zkLJ)vtbK2+8mn5aL?RJq$T#1 z(uAOSW>GeewO8X-!bW_2m@>a?>FMyYRGTW6c0uubDd1OF+GX}UI%D6f)dCA3pyNDa+sOs;zew;t^cI=tzTPp5JX2v`g*Wq=X> z)E}?ooYO2!@xyJ)LT&0C7izLmv~oFz=#bX!yQANA&Axylv# zQ>b@1*M-W>B4-wnkk8j-b=_vbO33VFyf3uSvP#J^($|LxKL+tWO> zfi%;wM%L^ZSlE@JX;{h@Z%j5#N#U!*_9PAD_ExE@lz}Z#{CV3*_nl3q zwCWtOoFR3lyd(Ed5reg`sx5Iyhb_Wgw@+wZ+HAcmHu$Vtc&mf*M921FcRrl;jVv(6 zePinvM;UW#Fdx5w$n4QI>t5tdk#yK>PgKc{dUTTC4>;RZZN$~baLWlTFHoRGM^l5 z-5jubd8>CF#f=V$+s)oPAaXmSK9_se&`f<24a&l&LF*>w;`ZZpOL4%(Y-`7HWYUmM z+S*a=FYrMOV1U>9BztHDp@%M)J{=Pb;Dy2%VZ>Dhni$2%- z{~~_=N5k^J{_sDc`QI(^cQ^b`RYBh7|KtdN7umz^KS{w~pK`gTq@t?&Hi6sM+e_;8 zZ!`ER6k1%RR4DcE#_p%pEx}9pp17F&&vYf$_xiP*o+P{M%Of z1NuFe+BflWqgeS7QJMeQIC@B`N9pr7k;|blL4Gm@ivMIKlY#u|f+X^Ke+ORx4D51g z#)yL~`kw6=h@__%zPhp=SxLT0pH-?)K5G?^%n~h|S1Tg?%yIR!!7tXtC)1vdF}wVj zz3wa}&xxz&pEAYd4Op0x2zbvAr2~rH&4ZU&*?mtKdcN*J=ao=fRrL#o!nlxcTVO`^ z7l7zbtcLjvgD7n=cyA%i=zDw?EE&^LZ$$N){VYF;V>ZXpkw~3`t8Ln~qX;0XP$HD1 zI9mgjNADcTS&{L0OBPRb2W2dxC~_!*f!F&rC&kWdl1jC1m-z!ZijjwVA+3RDsUwkm zW8ta#b>~7K552sp7f*YJC4X6L87;|LGuWGfD_rqn<^no7e)d+}2Bu zE%}6`sGZO#ajMMH*IPakFJ!}h(=-pV&*b-EY!_&>g+qkk5;Xr@J`)u|bO_^gH|T7$a*>1SbT-ns!1RJoI=x&Nq2Kd+Dz9oCH~Op<{@* zZ4hLCI>mmJ?3o*7!}x~_U`|DU-ao6X^X6-^Jac1ysgjgP;nn+at@J5rNG0i%q(7qR zE!kTy9J92WhI@wIU+${sm&xGa)RNjj?Ksv6B=wa+iGtgaXI*}o!4bl8+Q%6rTGGQ4 zO&4r~y|pikc7%qSdJ|KD9!tuEcE$1GYUc8zlB-MY?)HAGyle`2bg%Ka?dtUp3Um5i zG4y-#V7u<$FK_2C{3u+_$1njqEYfdTp>&b>oiMr)mlf;HDUHfnc|5F1RGT6nzCMh5 zkQ1`)pQ49wdOlL7^Irm2)@sM#|HMv!BM+SE>yJ(kGZflc+}8=fK75GX{kJk{9WhT> zVgmBGVCa$C!CJR{#05W;?u-0xD zeNCw@3f<<>7bg!$ZuZW@XM7eA(k7FVdx+T772-OE$vAabys?w=1&#xlP4 ze7W+W^QWhTQ~oFN01qCct0_-8(49IaSJugTcTgBZt3s6@AFtGKzOVCw6Uk7)u(dZu zlyOKCO9UbwaL!&kb(%R2;VpGwdrn-w<_23F0N!xgDT_21*yDSyvE+3+RP5I_q@U z9&@<*EjAbd7iFmm33UO3`sPq}TKvrPPdYyA7YnIlyZmLZ!kn5U2PbHfc6TC1#@OHB zbx;3oCgPuSe1ubN$Li3cVOxbRwo{2QeiD;!?IB{#peWCt7`|^Wrag>aO9}%97b*SBhC5OX~JgF`^hm@ ztC=i8pt}T)5j}6y$WSimqcrxfZAVBmu_x7?bF_0EX+jv506bCo2$hQ^vX%_JLwb{`)VD< zkGMA|$nU#IhgtFpNPoUrKbub+vz8nLNGI(@dgK#mrc63QOKfzfJ1=B<+RM~G+p-rm zqMLKP4f=j}))#;18P#CvgaQ$yc@<9bj>pyGKnnO}{|={bX`9c#E9hy--uN@AXr&h& zo^m3LbR8@#<=oPBvC<_3!XS#YJ zr7LYjzedc`jpg-=nayfTtG5m0ly&NTHf1p7(WZL)d!X_Y{t#mC-GUuhXIg+=tyGMG z$@F^vxwz>iUx3w>-ifz1>OKph$JOKRWhFSStVi0jvgiAP82WhqoW20?ICRj((nQgs z8+8=4nwoe%6K%x~2Pda;K7|vr_PO`Gti)pZNe_M{=4IX?R>r+x**lxr&&Rr~wSDX8 zmRY!cQM41kH&bah(?hCNMd;19Kp-hk05jP?(t0WOH%2wH#j~8<5$Rc>s!|rO4+QCI z2?ig9oVn2AN()c>mah4SPFzhCZ+@u&s{Xefex2%n0SmWlu#x!QhMP6qW@^z(4;>-8 zr!vg`U5VW?XGscUd@q8ivJ2U{OY^wVwx%3obe)X$9C*_E$|l*}6qku$p+^7WAF1&| zW#kc4QH0ZX^N0;ro0;wuHrr)tMTpPk`glSdjLxFWrx>T8o5R@?jT>sq$A;Isu4#gkLL z^n8OS?=lGz-#2}BvDh&_$1KeQy%Ty;FVw`_VwR;`8k_H}?(Fz+m0(yJGSb+C8G7}N zR+9BO8+@7hx1|BB=e*dS0#CdJpR(<^25UE)m`ep&%Y(;v7Ht$%ly8bS9$bBKUXpE1 zj(_UPRG_30#_SlCPPUHyQDpf{&6C(D9^-5?Y3`!ZZGI)3zw?{1(e>)7z1HW+Sy*SW zD`Qi9rj=GadXI90A4d3d88D zRs4MDP8~)nYQD2`7R-lyKpatH@ zd~8QYmwT2qhi|FkWatS^kI3?q;&SJfd#cK99|>RL@g=nyL}f`HPhVAjoQV?-kL!U$8i=mC4SLFc*m|yxu0iLK3S$zsXuBYHj#&*3|Hj8+bt!1o+i$CEL7}F@ zXA3J%4`S|i;X{$i8E-5^Wha8YLz(ox9^GGJ;S5*TvN!Bg3Jy>`yaQEUT=M14knDD! z82WBaxgw(l6X-XJkk8gEpDTAGW#&hbZy{Y7$jhYj0d}JMgOP4MzArcW8a2u56{1_l ztU{x#YFOddw_heEaSfQIhY$lxXtEn`=$%wJxHNSr!ow=6%#p&w9l3oCTBbK9xqCm1 zY~tViWUKgaR`Hzd1|@h4M?Q~*@ZNeGP`BC0=%6%OOmHwUFbXs0-q{NXKA0}$b?RE{+GXN}AV$7Bnbg-kOaVuU>WX=>f$dQF zd>Y1&vy(ZZ(pKh5d|&Ns3Fmx3-*Qy+yqNF)Q_>&4UUIbN*uQ26>x_&}s`8>+P1ky#*1f`ah_+I%1m zpVyW-oPlsBd6MGwj-k2?Emw1PHi^~6-lLia0K%-RH#%X|OB(0lFI&M9uOlTA^H zhb|dQQ>mrm_Q>&*ctAMp0GJWJ+e-Fs}8f zjPJegsvOt`Q`C=>o-YVQafQn^a9|b#j1mG6EyxWtFZ2ev67ir z*)m?AEA-Z^*G~1XX||F7fx!S|Abu2@#2%+>aRs$bSE(n-0kPV7ssdN8Ce5dbE5@%bB`zJf7ycnD(C|98 z6l;clJ|P)_ zTGM2AMfop1kyWyPPP5>uROS1A||Jv;|( zH046V%|tTw`*!2<0cRaqVF+|&l3A`0d(ToEgij=hZ<;;*A9z$=m!DM-K!^5;C!Of72ud(q1jp>``WcH=}GbW zZif5Btp%_qQzgEUX*0hNt*xdRbFeGh8;VPtD?eUUdkJN=^y_X5*@$}ayFm1pK73r5 zmms>F@l&IJi;c1AHdwa4!g`PG7fZatetI|plKcrnFcLyHXQDV|n3`-uRU6B8Utte^ z*3=(l;u@IQ3?s~@3>o8dB$NcE;2;w#F>VWSjIR)ZF1SyYoXjk5;_iWfJFeVh9IjdvK_YOEQF z?Tgs`KgiotJAt&UKAQ2n*zXczVT$lVmW-Fu{BUEq)1lb(3u*;LSfMR!B$GTsrs#&s3eb! zTrs9GYJ4*~AH_3JSDkvwa3ld9?>(9KI}*$g^Lbq|D)2hzY>%d9)m>#dhm2f7kYs6wt4{SETPUQ~os zll7KpO;9uA#3;P7u>jtaO+UXlycez$t!}U((@idR47&qH& z>(T8b0qKGY&h(!)eqh*2k+Y*8O?ORV?b+)LXcMF?J5tv5-?gc_3b&%m`D_P_%^&>b zNwV7ny7yc9aXCiQZXS6U{<5btr_bi@AF(!5u|K#35DS0oS!5}lspo0DdK8_Q9}V0~ zeq~8*ozEWv6#Dod;EJ)0n7*rbc~mZPcJcecbV_q|i~(D)wJusdt=|g!v|mGxRM{o6 z#gNd}ZJA4nePjw7Y-wA4&DYTKZr_gmI$9|t&!Uzm+?uY9LC64#YdIpJf2T>-v;XwS z%ysa5+H&GQ+K79F_wQ{0eeB|E$qdSG-;)jVfA-4E7D&@nKVmdCo+b_tN3_MDY|As% zC1b&j;&OeU>^2;m&*G96mSn z8nZ!|otmA%qZsZ_7vqOZYq{15t(5rHfhPkj0~xbm6M5;1awD z9|J#c`ZFH(FwmJ^LO5v!UAnouwmPzVR5u_1{*UaRPMW|C*J3f<_HO*q0v7YoD=Uc? zBlF(|nI)iciO1yxg;g zT8+Rk@_%?PZxY{0{)aFFH~T}nYcp1jTRK7H)m*g^ADrx7KWt#R7Ysus$&ACBalap_UX&A2`Fo%%Il=8$WJJmou4aBMqp-P4 zdd0CL;8Bfa@gKq@h#@=GBRmpU(IT>98cO zzc(q+W5*uPX!_aL0#d&?DYEN%vEL3)c;AnO3&)gGcR0MW=a$v@4`+%_&GpgJ&nu>O z&02v-nJvFvxvn*L%xt^y%a1DF0%fhb#>Wdh9-THV-Kq~-Nh4g>eCF}x)5GV}c12sE z=?THB-ETd*L}WW(;n{TtW_Y8#FN0lK)xXhkhuOQV zhj$p}crz#B_L;VAr#`IOmO{UxACQ~kZlBhFa4v3eOP!ES{8uT~_J!3>k>*bScs zR;(n_cQ9N2R8haqcnFUF1l`~rg^43}_|olnivZL6#TmYCqT1*Y2~KwqJiy2D^ZwpL zSb9W6*i+AMB&a4+f3MnI=mLX1lcn&8rjb-I3)y#&DOu(X{|=dOYJ-c-Kb^F{I2M+( z;Yaz1v_WoAkD1R9*p^Iw0CrmWvrZzXv&R5;K|Pb96Y|$~vVvgpoOVo^y{77(GMGVC^swRv%(A3G zmR6X(h5@QE&6d#IOq8CwFKjqbM&LR2rIE`gedVHamBAqybJsU^8Hv%}xh|Bb6`4KY z1UOc4hW&J4?Ok9&n=ymEUb$x`A(+pc=@fzCL*zxFBSMnwruS5SRZy;jawWa$*$Gh# zSmB7b$>~gN$kE=}XbrE8BQwhQz3k$H%O}|=aYjcce>_b4Li^@q4S9fcwYogZ*@eqW zvcowaG%htO;c&rT@EZqCdo1$oK4ZDMcD$p~&F8&#U*1iRr#`Al9<Y;? zccr9|@vhu$iZ~0<*Pr$KYHc=}J|0N`iN23)xHSW}N*(F$Fm_%&@q?3uc^B|8jayHrPh}Wx1K%KCLPFeM_m?O~kA@35Id?1v5-c z7Xr)Yn%>!y13|s0^R{DXGL@MQ?zPZ1hm9`mmn`ax6+mJork=rk3_gtUf|T|EDU5I4 z3w0B1#J(#Yd9~;Lk?2}x3>t&poj{4jRiI`$8EP_89=AhlV8?YFs|WYrmxA-CqldNv z@k*|&d@teWlSp@j^ZT|%(b5KI&-@J!@A7%0TiV@(%H(W`6eB5aPiP4gY``vaQ599M zdrX~<8b41+c!A8Jc!t3#Z#U}jAeh&2aRIld_lFS-9Do@?t>gQc4Y%PeTjEoQc!Gz* zz+5h}oUfRnA^hRu>xUt702bzMQ0be4%^59`pEu#na+5}jdyk_5lFSWtJeLr!%SQ3* zIEj>o238WaYW!z@^Zx}+dgy_XJKZUy|BFfKj2VIwLo-x>0mg1+tq6*KkruSGUBu4A2;E0#4&{0~AVOGOt=-%-2N-<$nE(MAC@hD2YB(5{FWTqSL4rNGd|a zk@F=6emLxGUYB*-f30Q8{RM4*2J+4}_c3udW4X@ca(3(D+mHN@(#l_KmMxi-5gMxw zn-TaqUwIrQV)ax7FZbGLnKMNn5vni;iV=8Mq*9|2?JJF9mE>4?; zE8e!ZqCVFS)jxUTSQINK&-`WKdW>I7CXJv%_oTJm_v<;Evj{Qmf8FODM^2OSG6=^x zzCWiL-5DY4AkSsFQ?fF-8vYJ^=)@-*+PFGJQT@VPaQdbFq7fNO%s8x zX-J2WgOQu{=*HiSfy?{$eMU5`a=5?RpB0zb<_BEY+w*7CmtL;f0Z_Q^OugNxr`eE( z*GK9s#G6Cp-rlb~(Y8DjFHfV4$kGxSJcQj?VnfFS>8UpbHXF-HgGVMTey6{%KN?W) zB{X6J+?x`G=F_Gg{bYg&0CAwl1#!vn}Y)B0c)ee;z0(GR6DNRH}iMkwZhAK0NGL6(Gw&JqT@nP2|VApU*goAIn zSPIGZgBJ!LOn804DO-3uA`*H`?W3w6P#7@s?`BYNwQU}Br0$u7g`e6G)lZf7;@~?x z#I`Ss(_%R*sgBbvm%y@+`nak%?omd5C7+(mwllizuIe&Kn8J8pw~@8{(&db52#*I@ zw^Q16YV!jst&LLEw)$t7WW&7q6N|7VZmRxffyR=0kOst^{gLJAbZ!H;fA@DI#QPMnQpxt6&8S{5G~ z8eq$#hY*ZY0po*0-dTr-k|pRl&bx=vXC4`ohQ6lpcSyB=J;i7`G%|PzqA4p&#@9*-L(M4&c9kI*bW9|82;%c z@%g$ntb5Dn$vpGREIN5NDbEVlt&$)Aq#=26lrbU2{`?RG&$(u6Woc5h!q2ufjf^+A|&$_{kkv<<`u#cIXUWs))gpeMo%10 zXC?uqrl~M28-Cn3g426YA`4E3<6NU>+Sie{ZD=g?+by<%y(ZJP0p4V0li_2D*B94j z;ujreVf9Q9)nYJh!nfUjoN!=CE~&L+h{6<;%Xik!`s-yE_7sF6`kC|MFM0&XJ+p`$ ztB*&~ZKmIY8X+x0tBxg>r-f!_|8Sih{Fdg1!RtWqNl}T`7ovjL{;7IxqgHP4(xy?A z;!$&S_}JiF11XHrxlg{i0F`rAA`%_c4h34vu-LsN3Z<4uEBp* z%rE(izW}i`#&dIvYRim1Yt#U|!TPQhZLu*IQV8s@a9;;fn;#<;V|h+TbBG19{9cYP zP^AK;zlbhvqN}pO6v+bwfC0={bpnKte1-AaK@rvSZ{yEHN5gL-dGkt{xL^q~2* zaUJ^j@rwR!DYfZ|6bCZF>EF#C++akXHmA)`oLLSZa~HXF&_cOUR$OCu3fn8d*>`L% zt+;*yn-jp2Ixm08PHuL!kPGRwc6zy}_nKuig{ofgZEAGSeqd&K9wUXBk0}KabCUL( zZX@~d=~c_82P?;6hVA}B2230YHV6QhGjp8aIeyV7Qmck2biJ_zgDvnjGu840N7LB;~>oYFn znu)PIAwRZ-$?k%(NEMs0OGB`=4RKwjEEiA>xU8%-tE^x8SkPmz4J*Gk{{ zRT>%}s_m)H??qH=T&911F{#w|c*b&{YS7O;9y6GA=XUTcuu8JvC69Omhw#1N&t9?f zOUZZs$hzlp?P`ajSBIZW5s1nqT0r$~cK!AJaQvyupAKeKoxcy<5#vj(9By4rA9D=a zsQej{;n-zxv4jo>Nx8?kLIfMh8@iA-@2 z20(r}K=%T^*VbEZV9ESQLHM{>x3d%`6hH2TLC-VTE3H=VbZ2&5VxR>s8QP)SPknhk zmznyNR36Rl9#YFGlqE#}&(wvcNh_+VzR=(f4h^B=;7}u17#8RR#(Gf%&XIiskp#wC zzqbs1D|QnN{$EOmR{tmM#Q*=l|0Rf4+h7qAGEDzFI(_fkGf+raxcqhV@!Ut@`}ZFr z^}&)-Qt?j6LPwcjl~Mu|W4;PiA-_A)*W%SA>5;Ea|BkWVNsdwfTXXUM`rVZ7+dq#h z$My;t#(Em*U)a!5c@p2>>rx|{tEsYuq)uP-NgtiB-|M??877oYrD&Tz) zjvG<;wA25kx#+x%NbaT^FnM*3rLW(myC(Oi3fskhJls;CQ^vZ!U-%jRmdBsP*!#5L zc(3)w<=NX=F0~Tn!D@?gN4(n5iiQTZ>g$d?7n?bebA$4OjZWZ4;ZUB#r|Po2G*U#N z=bjwd9E(?rhRnt78HK$-bfMPWZVo4HR6v>0*1<_`r2hG2MEjm&w;FZvJt~S@J$XL= zi`IiuSi!`7xxqrg_3^VO?ZX2ll_zy{*6ksu4I0CZ z-48h-_tivGoBdy0g!Lc~l)^H2!)YYuD^M=$U4sr1ym6Q5>sr!tn$8#co(6o*=oG># zdenKq;Mtv?@q{~7gI>*9U4O0?N?)_{hAF7Wc^CJzbk&0>U!PFKxcJE0cHmN2r(`ug zSq{$X%rAM&Go=zHfpXdldD)NCnB0Y;UrkW;3#pw(|5_3cXM&E;`~%eYGet!5;8`{$ z=UHDC>zU>21zg&79N~CHX#*2^F*)z7rmRJK7&zlDP?WmBZjIeXnr<(w>ObAyXQ$#G zKDJk?$HzX|28U%su$?_I5LSC$Vn}`w0d~^wbNq3y4qxWr6$fVQ2tBGOxxL8FSWdlM zV7+L);>$p{a!S9y`t*qFn08A%)IJNPs_ti7FNQt})kV*GRbp|uL}d21jY?hncR+;0 zWJ20eJo7yKXM6%-Y_luP;}842NjV{L(&tzoMvS2Abmw zW4``o@MgbWF8Gw;dyKxpuSZU?dP+SMj+M;E7qU6WKlW^aV4W%F8`q+t#eA+KMeXM`*UY+0NIQy{F z+>LpGo{o+cK}ba7#B5kG9YcYwAtkrNc>I4Sn&oV$#u$TzKO)5d?(z*VrKqZD(rTO> z-+(myYJ}NTX^zSXvR-?FQEV`T8>lr^J&yJGkYEmPbE@(O+$9ChVqPPB4&-K_>tkb% zs6g6NR3ffT1Wjn!4cJN>q*VYl_^eU8e0flbgg?DL>Ek65%lTb1_YoWjJ%LXTo!%6R z{2V3o|DVm(X45lZ)VGd>r;dQ+bOxAe3CDd{}fFftd zZ;q}?J99>l&8K6#gi__NXJFVnmfcF_MG5+Xdg~sGPp;=eA>hnICW9{#cP3i#DG?#p zV;q#WIVeCGA%87GiIC1 z`2ec1z_!73BaI0-E~$gHqM5Af4$1T?fX{>~r-Yp{OyKG<|HbneB){awf3~^&q)wH;`8gN zJsM>W6)Os-4TN;DYu{b?afy~wY-R}hN*8B*lVFy|18d>iXbjNZf=!@Uqt45$Tm(Fj zh`d}yAhjdMj&}!&3qzV2UPPU2ySwOh#vfU2&3X|_+JV1B<6_(S>Gpc?p@vbQ+|c~~()tsv}2%#8}P@1~IXq^*TNX|JfJ zgjq$*l4XO=bbD%J+>H1OQev~^$rH1l)-CC>2CGRXm4`5ZDqbxF~-2-WNV+l zcJboi`b2Ce{E*l>dv+)(1DH#F^vm*qZZh}CCNBwdB&AV*`(V^yHT)}~nCbH8;Jv2@ z3LYOIM0+K{kyiCD$r8xcTk(f@6u-VCxwp?PqGo72g6kUk^aU-Pzt%kow3O*& z*X61M);%Jw`-uTC{u}36+QgWAf`i3$d`^Ep9z#Qh!{Ly;F}_AbM0a+>nuykr^G#sz zu-k3bO5yXoGoW=o8A5vTVl?ZP_N3VKcocqFu>ghrh>yIvY3RJGdAZKV4X*+&QJluT>4_*Vp|t_m-C0yE?;Gq;h-5%#x8vC}~9s7mmp+4dqBrbC3c> zcy|bGjh#=hjsCAt)AT6_qjzkham+puv^shX1Wl~8hNh@rhQbbw&(8IMejEPrv{|n9 zClmHSEz~!IF3!o&mb{mT_$HmIwfAH#ez^6J6CriGc5YjBZ=dK@hqAm;%_6;@*Kf80FT)gW0Lz7K;d3iWH%Z?G?8{lQ&s^5fa|q4Q)G zGEtjZI7B7U32U`zX5=GNk5U;lSgJ7Cc`? zK_Ha4(JX*20f-9vL;v3G!IvfXgv|L4{-Hod=40D&voJ`PU_93&HSj3exlABOsS+%N zRDk&soM#@x=sIZxk7G^5ned`E%x#fQ%(~m1tqvS4+Zyj9tg=-4z#%n_tW@?BxMVF? zVUf;I#G6SCjsNUn@Jxgx$7ReBCd@v0ny<&6q%7amL`z?QLqwdO0dp4amE6GXJ5mHc zQz-iM%rZ{b50_y~a?-@}mP^Rb1TNO+J|<7g+r1ft$S?ZAo~%pYA?EUlh4NlnseKmC z2t7?X;Zx+%$fu*2>&Cub%*r<-fvV|Dc3)}>)HAJ4ZbsIJK<48D{cxY1Kb2{le;u#LZkU1SSdngBx{06WTamRh{)Y2>dPAiYcTCPb zfo7@2pn|uzSvOx2!#8#pf=pP85CL(IFSuWu_xL?Iq#`B@Xm8`JQf3_J<~9UE8Qk49>I z*i&|k+fLfV7c0#Y_j$TS7$pL*ZreXTo$}@o4lr@TD0!^Id^o7jONbD6a3Br!M;J}g zE6rH8SNhsN3{Y@!Erv3|IA^8AA~%9GK53t3HUPXecM>~GAK4XM)&XK6V9 zo{??Jy9s=MqCj{a8up@wD(mwc@64Cse%_&#MA@J^=646eNZER3mil zl2M=;6I7*3>jTyv@L`*8*Xra|6WY~-HZzN5=k6iO@wx%MgCx@CN&uRJ z%k4Z2!2#7ltnn@B*VkXTBxbTnAEY-xHA# zut%?4*<%vJ5ywuh=|30wWzl(PET=Cqu>TeS%;yEY50su{BLK4%)8qEAC9@yq1o=g< z!PpiFz)1^dGtPx037MJ;=l{upI7&oztK!KR=xvbs{)p0y>l(cyQ+Yh|f`_wKip!s< zKpGw)Gjh9twsPGHw^gjACmuJCZ#}mZ>!+M`9f>kwER|f@+TgGfvc27}-cR;_I&0ld zcPu{*v-B1$E_kl9vK#S47&Efd9-eU1*BZ8wSe{hm)hpWcIHFBawzV|O{f)bqrxnv5 z87Qmi%IJqpQ52uc)1-8x)`m#up0_uTHJ*PSd@p0R{0JVSRv zL*Tk_WxrajEA^Hc?5M{RCdKWh`XI(3E2tw>NfJfWQ^U)Zq2fv=`{`%RT!@wcCa4sa zl18^J?zN>jz3rddT|m0m&E>YXH8{*Ibm}%?;J&rq&Xqo_`$ky9K3*BcRX@!zz6gYmlp%R~QjdPXR?Tf2mb&$vqC%0$ z#1Taufa-gA!C4^cG2#;w_a2Y!2ogB2(S$wQ&cpO;i&d=xeA1xmwtU5_N58$wNbY1u z_1-MAU}2a=(;-R3X3ZZ_oNSTwVgxTUgm8h1bh7LM!`di0htfFUJ*5AIJB7*B*?Iq{sS3?f~v4nufED zhq&aUx&SzDlkG^%7gr>?+HTQhKAXeVZ`y^$Cf-v?s5VtDc7YQ9_sKqej=uQ5n6qVE)oi2H>{%lexb9r*DzUFXr z__TH5^6D#|$OS>zp|k1uI{EU{WN8UaJQP0I7>vSGJr~v&VYINLm>34WbW&qEaJ-rd zefi=;4?lThw|T+Rwq3}}K)53EBPH!5KW6`aAkiNrXENF$3VHqmY@VxMW=t)|UTETn zKYi987wU%toLRGnOKZRUZoLZ5HWetQY*1djo+D`g2r*-6ADgdF5F!L>3uX}0G7m~% z0QW=6SVe^qbd-4Z;TaBfa~$rVsJneap=y zvXVhh8g(nd*-xHPx^f8`ZP%kF+(WlMTTrUab-{kC^jewC z-k_^;{28dVK_Pjp)Y-y;^q7WS-E+dq8 z|I9%gRshWBD_?G?TRbl3ogg{{`f~ae)5AxWuSt9@=nTWKDS719RtPDNx%fr8zFsuu zUS3!#`P2^mB}SLiC>o}kQQ%ZM6<^_h51pYgl=~6@KW=`oohve;PZ0RoTgL-OQ!TLI27RPF(d>PYCDVYeGDL|Os8)Oq* zQuT{e{7o+163{F%xWC?^uWR(nW)y?lOYSLO!R*#*Fm=u2@$Pgz(5_LxZ&%S|bl;-D z_x#cH0;wd_?^AB55vQI{Z81-d7-8=co%szh_3&8so`5U?VC|&l=51=Aa@!FlG<8%m zm1)Q6WSq}N;-oy4=R|65=~AMl@y~1^H+iDp#^>N`M{%OgWK6WwY2&0Tw@L0Fzgc`~ z*apl}Z#DH8_*~*~S};2wxo8u>u!K0{KnS+f>yy8+mr6$J#G?MPB!MF4`kCYPDdLJ( z?W9(2r&9}0Mg5^>_T@`-N5F_aSc~1&n-NN{{F+5+XE?RsG`UJ5e&fninUEB3Fddmc z*YS{iQBK8YR4+X8yvFc@_`_AhV%>yqE^3d6dQycEZO#|xZk7@T*xcsxv#qogw--n5 zOhR#g3kIZJkur-LVkzbc_UkkUPBwkbsPJwbC~SXBL#Bz?Nv;=fEf~1D)|*%ogZ`=) zHxOanRnVOftDmloak$p(40c5#sdQb+xttvf4 zP@~_;?_({P-&p2XI*7znzm=wGR|RNm$vwpWT=VJpps$e_wR=fk)JjJoe3$H?t3$c@ zLPel2zu7^vx?Knux(GlkuI^6M*gAH&2+$S1v3pnW@_TY*Zvqx>;83%gFDzP**()$`b4KEJ$8~LdT=emauAYXyq111@tpI^whP^X<>ErYBV|0ruJ&@12#sq9X+QKUsOr(O`T_wF$(0f(#Y~`XNuaESh!tDikD$&)M z`5bE|7g!uFYD|A7CLp;(BAafBrYFB)W!tx_z|tqSrZ){nmR3$8t_a;#yT|$iv!z8; zY+ld}X3k>|bs_sa{)!t_9#lV=^&TPKex5?IC4-bSEto>zLPcbfi&}bW`wDZc;tZeVAEDkj3vo(7Du5W*MO>3Kh z6B?yIxrL+zD{x^D<}^F}7`kP-@b`{Y>4!7?>EKy2#QoKGD>0QCF03d*i5Jj|pKOiw z2cjl}Y>|a?End${MW6ZLHpUkh%Vo-n=JK|=xg{P$e)2}Yzl5>^137TK!=+a@qKyh zsZ|U%Yai}fyzuP3R_v;a!aM&;4qF45A_Jgu(yTK5T6u+hriJX=ah-Oyje5!PI2ag= zroS};=ni>TEO=il>=H~Qd;bx3p?%QH==-ezc0f7pkBch=CtZKmOM||%1FN5ZIaT15 z%NScnz+m;60-Q7r+~pQ2Oz*QMCkeoqS@5}_kBz#A@gS@z3NuUAc|z3nBYPEq7HoI& zkPM;BIkJ{u!CbtCv&e8A!5}qm@ z_`fz+(uz^SQX1^py+uYPjCZ6ZgOyvayNt@0P+w@LQ)2pZN_}N!4h4 zjC>7AdoF?}U45<&wY+f@L7JT~QLlxzZ#gf_M~!bwKH0QNGJ_EOae5z9-nYB9 zdUWxSZkYcr@3LvjoQX`_(lcEjx=4Y}y8<-suvT_VSCerfnBXIiBf7RKuz*4kPBStO zxiqoS^t;ycKM3(!+OwH#f^Px#R$M`Rad@>B*O(r=M9O8Y7UNKNvj~u%eGzPb}8{{&yD>uiL zVASiWEc(ye+xS&C4(;@8MR_FKrBRWSrP_*w0t=hlmP7d>BfJZ=^Sq5)dzw#(6jTBk zg0)Mg7=$&fj&@rg%A8QmL_fS}TTI#M1Z}RB$PSv?iX|oKQ@M<)P+z-;Jvv$wKN3zH zC(vEq#04j=pcBrK%6S(4xI1kfOYUQTH9gLvn$to~QS%MY2@?xua`h#hk4OaMwnsuh zwH7^;q-pBmM(SzsB+_iwUMMn;D6d-h?|40vO?0Ef>Gy@gd$jY{pZK(aHp^ODY_G+s zN0mHwF&i5`TkGnw*nGCP@^H1J%rB=X!-vQIg*MgIT`R6cV;T0meK?e=XZ~If{S0 z%fFuIG~i!X@jurZ|Bvgev8@Rn#dD)(O(Tai&5Wim{(*Z7R#esh#{_+Sf84+DqA}fq zJ0nZY)__6AyDhx@Ohv=XYn#ZOWVXY)0IqXWWp#|s;VgO&ZW~#nlMw95D}$YqT!}Q8 z)d-j03gR@ct@%%{{UH1=d~pMHH-qc3*!J=+(90_)T4%@iVIDYmcW001`BlE5H6k`{ z)VQ>U4i&>-r5};|(m{d`(j5&;1=D>jZ{SoGR2cl(!OoGwXWvZmC~D80tyo;)-+;+! zjaQT|($MGy^dZHo=tqDNZjYGsl$sBt?eTZd9m1e=Vkz6N=Gs9>wZX8t_d%?KL+rDB zTQJuG`hg(TmxuttqkU)4M-ak3Z`%c`w=YXqH(ckIu7)swGWy)ePKu1; zfawH;ake1!tcBs5AH9{8ZSklGbdIw@nS?J!0TxmK+~L&?24SB^Rfn5PV=Snbr3qqe-H6KSsai} zaEyc~>=fg&)oEvZR*B{*g+!QmGJH5D613=g}SOn}is2(Uz*%m^{;7_#K5TX~rJ zAPb5-ls@9`PVWiaA4(->OB1cKtx(GZO}6(tu!sq9JB(N)fP!}TL!m~ix70D2I$rav z^+?cMae|UD)9#y;Je`2)<%vh3|hrc>uNJ692Bn-ML&=7NB8~_LV*kt z*n_29rt$ZcqDK>6qi&6C4gX;ngK1PhIa87hQOqvIBr15I=Ox3uN5H1Hm>tH=yu?

(f{`llNJL4A@zq)#Xpi*%0Rb3Ho;?1Jd*%qlM15RDt>5&AH zH`ZGEgE8+<@wh}`&be>j6KBa?{QDVe4QtzSkv>m|)L$i)6$kj6Q`f9T2>LiM+0VYrPNHE6h3O{DN;}XR)N;YAL7lO0~%Q z@&=bfDlTZT!ATO=eIV6KXuzVWFaJJS$9oz79W7rVWo~}?994*j zHPK@Rzy&8>0*1B(cU56#K(}Skn?==*cHn#hI^SslMHE}(8<^cQ70f@sjIop#qAmj0YSrJG61G>H+%6JEHpR(ldo<9JD`erCvGpn{FYSWE73xfP zw>kv3ZDb}(z7mw8;~B4&=*X}%aW>0o|&qp{=K0Y zgVyYXxVz_((kAtA_4mQ|J*p_^VXNo&i(0{bOQ|i4B{<(eY6j7L`vJqix0$OM5Fqcf7JWwe%s-*PpK%l7;`@T1Wn4GTzi`hhtg82y0miF~AJ3;F4^ z%7~8DP5bp56DN{;+iXxbI#W>}DZJWId?ecwsbm5GkMxNPXs7CAoSyUTlcbvjBakgM#~Qnut~naqorlU5Zv2V`Lb1^9SJwB|N3Vj%aglPe33tMT<)T zxW6NX=IvjeQN9D>1ZFZ`m=#oE5T#=_JArtyiwpfR^>l9-ID~MUyOm-NUU*?=#EKB3(YlU`XB%Fq!lB9xKD7=R2@1BIM(eV zF0M6Cy`v+sNa_uvW!6sbf|!y}tLC*5Uv5L+go*#xCVw<0B6z5K9Z69l`OJuQ*W`O{ z)FR`7niTK!BK`QTDDu(7JC@4VQSW|DHH?g#YRbnBnR#_o&i!L@i*Kay=AuTguu7&S znl93VZ~`+~Jgd1A;x7W6OjcZZ_*h|LklQ}qTOtSdFocFW=lk2 z>OGLhRG*Sipul|qnHOARZ0n7zh2`g1rJO5$%-5c3G#7gdKa;wCrd77T;c&(3p$faB0BvN=*QH;E@OYbDF$MQ?8AvM;;y;fnqq7- z;fhE@Ad-Tg++0<{vz6T(iBe#d!5~!E*7}KIWC*)-=sf;!isNmiDggK-TO9Vfh4k&) zx0K(xlgID76uV;P>|h2);fmP zgg#q*O*OlXL`b*GI5d$M-}^q$sreRLiij(8Y@BnCjtOz<(iMx+@Z4Bv~ZZV|@;8Rak>V4(Yatt6LOVCqFnl-QYTf`N9olIgPMrkZt|-3#x;hwlf?YEA9V$ z;6a6AZht3{UF4Oeks&DWurX0$n(cP~P7%9EYq?0;+uEAixxh&}nF3r)O{m;0T`Z_% z<(1Vm{V@sQ;Hco_rNqC2{+z72dw<`3dA(c|*f(gYzYmUu59}yL`GE3qv#Q+cn8LQv z>eQj(Te+?Nd@G^ds@Jb!2X@o3V7kZ}iD8wq9g2M@Ozv~HI*s?sw`X5bip$)Vsg|na z$Ys$W}}|6NRTTI8Vp&sr&|KgwHm{~8Hx$oPM6 z`2W8K6^l2;Y9s^GNlI*MIpF#nwxVJf|5!b@7e@anVh60iBkRr7KKM|q4r0|b$#G#s zO$7GxHbLNwz)-1YACO-xBDDD(o|R=@r)VG$E;#7ZNB@TpYm@)(Z^&4#CM$A=g^f)? zZtm8xSX2~u(+f9Zx+Q9Gt)Go~i5=H&b(WHq!aF7=rW>&{hV`no{p%zwXUO45K|w)m ze0p!CGj9#gl_WE>gygak!|N$-^x-nEpTfK4=5eEuy=lJ<4;QkNWcwCX5)uoGcev5~ zA(nOeiIv{5J*AcVbh}IC5BsZqu2j@-{+jk(arEwU|5Ykb!5H=r*v5Or1d1*5ou>>`kYe zL=3EBz1&!R=tV(?Q$tyVMsmS9iM=M!XPK+-?;Q_mN6rlpKfUy~Dn(M}S%57?u?+n9 zs@XQ=>8z;&u7MXpi$Kqw2HICwzH~hF8pwe2Fl@l>#duJEO~H=cK=pA%Q$B42vH4LMTP1n@X-81GhSf_R;=AyaKX z$EHpsM*$*1x#vox3L~xq%>Gj^! z8HFu-iM9 z^o0hC7tnt`on%Q&u`iqqr~R`}tJj!X8nY_#_@^9v%0?>}+^5mA3Xa5B8P&q*4r$vObscX^O0;rky#S3>ii$snge;p#8b4 ziWe{or-`t#c>aqO`w7QKADt?DHB+KhYTL45y!tkYLy2%JS*Far%3Su_aU;7@BQN3q zv)yqy*?lRV#Jv6k-_W0quc3FLDmw#GC@V&((^@V51Lvz#8>}8_YY1q$0NPKYKCwa@ ztUFDp#6%&vKGN5}@e@<30Jg&Y83e400Q4Jlyz4HeY990xwUWkyhfR!Nz3bWWVKP zNt>Hs_qVMD2T+w|E=;!N%X)RwU`F0F4aO@0d(bux=x;JqH&S~?-Z&V7UL_P?8 z3#dckOXS!uP0?b9Euvp&J+sh#X-nI^7TrofJYM>~BDVxTd&f#H{pNj2qr#PHU}(YezeX& zkwtfC3sV{XwI9dG>HC?&@!2h@gw6bTU)8CPv@P_$HB{kfTzye*a0whB@`*Qh)YTg{ z^L5X*q@YRkv*OYqw>}7nCqF^l&ruwi$3JtMgRtPhmFt1!uvV^%Vq-A4;7OV@rfQeN z(|(b1oOXt~%_mqo| z6ne=c>+3`tZmL1$T71W}MJzfKccVz@y}YTc!s@jJ1+Im!;7jy#vXJ|gaox!g9euWiVh%bo~QcB1)!XbAkn&~@~ z)w5(GV)2hdcZ&BfUSFT0uQ5Dw{qYl|?V7`&kM`A0M`#!>7o-h|zWpI%ta++k7J9bw zE)(f#N?!Xc>kr=ah1Mw?ey3GU9PRHhNhae9%e9-&=q3f-v9&*gjRoo{K#`UlA_p1x zQA4{yg9Git9(dxL;p()+h(>%G%^nP@tM{DO-)(=ntS}JxwiLQr7_w$XIBPu&a1!J= zA*RHfD~G&yKQ?!ZRlDWP_B>&!^CoBFf8u^3KY6}7(fPROJ-8EMVAk_=TfwF~R_8kJ zB^LehMEZP{o>{*sRfDZ9a3%{9_E`04dx`%W+QVW8o8*w_jq0@#F+){}4uQxWN(S7d zQ1Wav#}BH;41%j3hz7`S*HeMlw!r&VhZk8VMB#ijqx}ZcyVgbYT7^Hykm8$MdeD+i-|mj`s>ooTQn}{3w6w zdOf_cVgnpjdHh}d`{I+OlQ`BDmtc+2q4%B&6mg$iUrA-vpK+>(;{}wd^Cl_hYWOt4 z<9Mw(t+kZ^*LhemjGo$&x?Hgge3KrAKCZf_*7|KlXt8cD-@N5Ms*)&-mT1A9t~h#1 zn*3i#vr~M4Y>RXpu9|mznQ-9w0^!CC?{Uzqu{rymX%J+TRs+P$>AYvg65);dc0498 z4AFr-?q5b8?c_R5FbX^0MvYgk3?^U6$c=6{<2i*Ads6s;EQuryJu?KxMoZA=fo3@a z8)wHV;%+odS#5EM-1EuokRGpxV-YmDv)^$`2Q#ZLqr^AL==GOb`ygkS)pur(@X~Vc z_xJ{5s7#2qY-HT*N|e(wlTj;9BHq$TGUkkwu+--Tf?6dus0Mg(hNY%!0!%&W~yfSJ; z@uZorJb5I-g4Zl!^G*(!%K&yCmp|l z;H*M#*h9zaYd3O5j4E|0qF|OAX%}T&P_-pmeRf7KArC!s7H^2hahc=5YPUe&^~lw^ zAH7m@M)De|_HBp0WD8cQwQEP$DhVww1 zR@WYvfzK6>o2Qbe5|K&{B&MkR56geA-W(RGk2b4bOOOkFH(|d~8WR3-6J!bTGn%!z zUvK2SS=|R*I=A7Rhqbg~4uBG0ya+?jB(aGJ(!;!CHTK06WX)XEXqAI-x>C|F*wp;X z>i}T|)EIwv3s(oXjnin@BKS^;&e-#Ew!12PUa1R6LX7Z~5Suu$$yL2IUW-WQiSgv- zT>BhdymVO<&~gSSVtp2PL3<}BFOnEwTU2L;cMao72+U(Oxh3iuj_@05dGSdN4j{M6 zJ>Nv6s`8m|X>Rlh2V4NAx8^q-;vY%OFDCPx+z=(^Y~%}8E@+9@I3lFR zWZ&y*wLta%qC9%u`y(aKJqMX=2M$e@QO3dZ!ANedqf`T7CkU-~>!4!a?~s<`M|JdL z5Hk}wM28lLr^Kw(tgQbivw)-@339@;M^{B6s{osLpS|Wg2()LNTCeYqWMl&+&(BPX zT#7dW4X;Bq!b2l2jbwh3@l;_Y*!%s&+6R>yilnvWP>yUyqnfy={Hgx3#GYH&>YvVH z!1{V38a)+0U@uMN^_9};+~cQvQT5=_CH<6A+RzlzZd^`!(6bkFi`!btxZf+!+3Rmv zw@Fa+nOG#`ss9j__8&|vrY+{6Ux-?zg*!{@wNbmEHH$)Ilu`%Ul}Tpf9geJ`+AX0y zTx-U%v`s2emzTaF67+|(t_?)cVgI?}$Vt-s?5Ig8`YGBX2e}zltBZukr+LOhF#B`xC7IW znt*uyy_BS~?PBl5u$k9nmiO7N%)GIl=2{Q_s$~?Eh{u>{&1x*o5yvX4 zPCz$4-}QN8d!rrcBv%wK&>hk(8@N7v$nM5x#K?8-eSJYg2v~8ieVjG-pl@P)Nf$BM zxi=m^>^n%bJgbN03jiL-ni7*QN}JKXsrn5cMIfuM3WVMGd@4Jo!ZBoYdd`^~>A3z= zHbc!}`5x%$H`r=AmNq{fm8zYip(7NmP&aII{*yLCtIdM<9Og`*eTmbmM52-thcMi9 z^W#5&Tt&&R=k|kr%g0L5HU*&w9-scCY;9mKmB=_h;ew~aeCCbV1SxGG4N|b_td_2d zq%r3AgecqZSG(alCwWp1%S+J#vjiX0`Inkr<{N?u4UJ`6>5-PSzw}0@>y)|K0DOj# zhPoZjO}=R(<~CU9-D0D^b?BwuRO>Yrlb^A?5yd+5?V%cI3!S2T70RuY<@#mKYFW$i z0Cg-daWBv?uz6+;gf!d=At4|jAg@H%?1z?rv|1L1C=MTs|H?xkpS?babK9iixH-s{ ze`2Pb_=br%zBxwTwLeTCx4!S=sshH*l%YU;3Hg1EjFJq0RA#-*Z_BiEPqUP&B4b7E z!-;jGM<(s4&ak{ZGk3;5Vvh95-Jow6u0YnOF^Sx+CnZisrP|jkVD~#k>yCMB2A=F~ zo!M_Kc@JeYSMgz+eN%g_j=;j}bx2YH*>BP(j#fh~B!cH#m;W}n;qL}V8}WXxb1o*W zr`_c5cayWjNU<(QK|Hd!XpvK4E%F?#ZrSNPr+hrc_JgGzmcL-bDnNB|3>!}Viv$cyNyvx)2_ta+~By-^(>yeTr}xa>wt zmn`vY-rk|6oT_oDqoVeko09K&^(^h@Ji}O(Gzzk_gsA)M@$QB;V-y6bi0pR5Vi-ox zhfon6>Jvu8QCKiO&uMD*hVg@-zT0)UT}^&p^C71ANz__vU~qkie=- zkRCx%TR5V<-|$a*)48RPa~@Y27m}>t%t7fY?(mr9{=s-$1%*ZRI!829#f4f6Gp^m7 z_F*3khd){8&MyMAhjDxR(^sY;DBkmJ%iP`2WFY5w)9AiPo1h^n+R$K~uolI}y@4wK z+)vwwixaQrWg~uDXCgzyt4$*a_FHB$c69bAd+k@>@*&4eALKuNLdUwk_C)+Y#A8Ee zVO>o8x&*a$ectQVAE+96yKyw#8JZQtv;QSet`hD8g2!0qg;HO8>l4k*XL~pteoB+g z@Db0xF$Ow5=3O?Q(0~CfFhk0r{#oQ{q0*OYG9oFYkM8w%1LY4jC6ow|Jh@@f_}Qth zSdHsDx@g^BCiG*)Y>Yd3Jk!ETsP*_s3b0U(5wX1N%JN0^Ru8-tw|TV{wSh@jwA8ke z%+WnhgOcp{h#spLdXN;ZJ(E~}f6I+(eJ zRz>&QX)X%hJ8m^yed+l#D=SiO6Zhk(&`U<=CxLzEOH)nf-Reo865!p8INw^rXlh46 z*i)Hg=Nq~KhA-26QGO?RVwC z>py&t-wYVT!#%l7mTZNg+qicf>09I(C6-pdmTk9A0#6$F_0|`@iuB}sR)0lLKSE!1 zLu|8B(4GfvlJU})e_GNukd3Sk%F2|x&-@UNB{z+c60-N_qg#cp?8Oh9(NVWrA1{o{ zb#_-5`cLz+kML$B#jcenHhHB}$| zAA3beB>zKIl@(hZzuRlSwfFU1tz`dz(~Frs2e0(7EA@`7+!g0L^}w=#%O{&6jwU?P z?5xSB-}7IkXaXex61g9n?4hwUU`u8gIWhw~%t-E^X zh3gq~Z612`lN7u^BJef1H&6o5yT!ic@YZjgs@1$Oe%mUy^qsNr#{2PRLl-+vkNn93 zLv0=vD@1F}ik{LcO|f$=t7_s_zH#4k@?4Z>>E0fGy@N@|Z@#~TF&+KyW@=+T!{gOat; zqM-9jOV=Nt6^{ih>~uzN@aer1Jh_;@i2k9=m90Zr4`<140fI@@2U|}dHw&K+UT@J);VB$AUtn|^GliD0)|lu zoo9?*MO|wzZ^-|Iz?XYn9Fp$7-*4~hXhvkx6-ibP_<~|@I1+-jK4IPVLN`%bVCKL# z8OHbLV`6^T<@1H~3jbF2jn?IxgyKIHY%F=2i&cNw&)F6~>Xi-G;jqdgr!`XO^DPbQ z;lA)Jo4%;C<1en)WiH^_Draew?_%S)5C+Ah>h#ISaKLGHr9RH^!K4>XAaT z@cr~xyB%^dd6sJH24e0<1yQdkO;JN-%i->=x6vnfvd}GyQtZVvOdpl4aIPY!M#KW$v z7!7RYZzIfS{Jw%y?&p!scEsh!5IKi5aJscz>dhw^cLf@ssfe4r>*14YE?=KXWTHS@y_tR#QUhS@)3UO?tx<;?;l<3gvmFLeHf~`?U=v7_*hl9kY`KS9PL*B~=N$z>? zUgdpWhcgdZb?mkbM0=&J{6gK*hzJ@zGl$+=+TpQrtBp1&uWga-()_CRxfb)@E!U|_8l*V`ZkJzVJ& zbqPf=QdHA_vL?pLdZs&H<4htQye+72yEV$R_t?siTp8PO_yZ%Sis-GvYdM=Kgzh!r zg9CP%y{_E|+Q}G;w_Bud(){LO@g3nA02>eSyQD7wx^;@X`$H|)=*MFh^_C}73iGnPhB+OB8b1}vI ztkrZ{Dec-zrl%HWO*r5toEPHz8TYwvsGptuXD0-FmMrgb9^0>3S%iTrtla3Te zkUoV>aXvO-p#);+QSh;j1!a7s39#Q;#1qs=obS|RLUKb!nd6<=4x5&stixOFmpab< z@4Av=N`h_vhPdhXD8^u{iVIhEeFI}$&co?aEJ@iY)74AKL$X&}`wFfR3J=~C_fzSd z7z0SsqFKGCF8-DgQ68S$r<1ABI6Q&fNLnUAsx%b}4}#(a39#Ke`>OJROPRPCYjkNQ z$9~3#uyaR-*Ys^jrR^eS#4!0Fd(c$73Elk$z*uh{o3l%t6gxHb}!k^Y^92X zBmfDNCoeaa>wFm(DfDa}d77<|dB6FofP964%&MrgCGH(fW$vA?Zh!3h3#bazikvU% znKWYW(|g3SQd;iTjW7MhjU($hdb^=s)!1mrB2VwxkPkTo+m)Ml0mo;PgYta>xI$E^ z_1HALsH1zQSrjl|R5|xG$kI_ZA$u)t8z+JPzTNRgDPr%qGWvxMKYD9>WFQjyz{Aib z5RsHITDngDHYFx7lJ)%^#Ym}n&UoxkP~Lq<;R(BnqDWh5VQCpj@dMF2B>mATB$Lbj z7d9^2Es`caq)U3&yoV+0mTf5TAdN!RYa@SY?mgACnzFNV=qcT17>8rajoLqtXWT1B zFoZwk?xzI`gx_g@X0{e`>tKf+MazP2Yzl9fkZVuWXR4p`5MTv}^6=j_jh) zFviZcgu2N}d-0^{{FhZ-PL|C9q#afRN(LQI9A#5&(Hohwl&@Hx zRy1>V+M|pb6x%cRni_cTaB6+%+%v7MjQ-IAoJ`fAcI{l{+|$g!HFfk)XbOk>OuVlm ziIB2g;f_T3dWCYO$H1@xy3g*qUdpQ0NKa6% zI4=Li`=txFZHf|nvi+$A^kMwP7E2`~Ny@Flyer<75gZm&wZY{+%0~3%>sTHZ1td@Y zm5%!tB?m$3*@rLjZ8KhTbiQs2AkQL_N_gjMW(sKEi!g@=hFOZ1uvc-Dr9rKK`!Q@n zS~Ih6CL3{e*^6kZ|CVvY+?b!$yEcrD;AX$zdU8JyMlKin1*hXb4~$R;j$z{PUi`WB zGrh`lO1}zWx6NK2{Y z&haF1jeNd=v)rBaPTQJO9O1dDQFmoE$9s)oX3KN@&1Z`~yl^-KLF*ShDaD66nFBbJD_psb5> zAJc2Rz?5DeA|;{Yy0+i;Hnj)cpClWDOV${%N5(2W7jmq1FX0vggjvfTu`NSkCriI@ zr}oOEIorkI253LsAP(l@k8(T_S-tT!zGx~;1fcyuZ2)jEAw3;V1>cQd31&~Zxlfqc zU($ebd)u}Q+x1@eIQm}oY=fEd@@&RnMyy4I?L_3TL8~WNfhXXLvSQ0K9vvnm7!6Kj z&;5w3dFL1@@WKo8^hA89Gg2C zV1Zlx1^Q`{4?-7|&#puSNB|OdzwR}xlz7)I5n&gDEzL^K*1KQsF>eG#n^J$55j+VB zeK33`c!i$G*|l%I@%8wMgtOA~3oonnF3&GU7WOx^&zmSUj)oPz^-OBEKyXJ-dPJ}bBp1o_BJi`}0?j~D#7 z@)r&HsYO$yd>H>@yWV&DiPylZxV)F??m0K?8OljfTQdfb*wsbzC!w8#lDo1wVyR!g z$WyS#*;P4>FGZvCZxhL2PEnH?3%-clq&%mca~!cP-`@)n@nHOF3wndbLdpma-8AeJg{BaT0ZVOJDzgrJ*d(LB` zaHSc#lixDZMIi}=dJ1+&0%(1n2K&k`n4bky$UZ^2uCAO(mfvSTH+D99AV zBzP(JMAjCiy2q=gUCCDPU#J?OZWv(afcEjf{ybxTT*F)SO35MTLK~4+q!;Kw5J4jE z`BCMYH!C<@dhQq*-4VBYD8eLa_6FifOyvLZY6u$tN%eR#HQy_KJvI6z#)-8H2yaSw z4|D;j29vAzpHtdy{RZUoKOGo=%H8Uh1Np0u%Tq=HMp~@iQUD!-?-RUbV}w_3@;IvK zg4qc}_x&rNcSBh<{tSi`wLezoV?H$}I;hu&>`0w2mL||g z9A)c-+Y4PV@kWr6Ma0fm%z+1Kbyk(uTvHeS6!bRlUXTvn#D| z!Q!|_<;$+Wl@qg^$2CWy^Nop#HYfQoUdvM@0G<{5h=4IV#7{K)EZAQhUzb2^;Jo@J zwOU6~NQ&fGj?9m?i)_t*73{Moa50U}SF0Z(Z!VbOzPyxXG%#4Z4a)!@u7Qjzn|`gM z&X7$SUx0xEhv4ju7s5!eN`c9antolx-r!_x$pX=TfeR@Fo0T-x3|C+Eg_I3bG=+^| ze0TPc%j4dV@=E&YGeac#hkldXOEHuBLLI z6%%l)OyO5N(ooL@DD2AP2g-WM2j~w=L0kX*=0x-r9GVgGQ2A3CLyEn%X57;C_Sk}@ zx156bGxdg;GjPE=JwQ`uq9cHnZR{$;91t#4?onz|i&F%KnW zxa>Z&5{p`7{b4YEke?W(Nk?(rXaS+6m;!{T$!tFVr_~#d@jq5Wb{Hy7z{%-N%`|?B z5y<)kBQSZ+!vbB0m4lzQTv}bm>6s>MFpRRQKk6Q*`1C(>nQ=NRk}w+Z$Gs7=(d~K{ zy1Cg9oa;sM&~&=bUnz!!Gk%e6G1tF(-7Cx}u{3P9=}36JDfH-lF*pl-312BGNDWKL zoak>gV<6-EF&-euiO|fJyn|UwXTBG^R!EtjK~hq=K6;}2+K<;5=JI+?cg2jAyfwYa z>3}JhNJhtoLz>J4bqr@(P9T{k0VP$uGs4M*q$mQyFw6K=q8u;RuEh!e|NCpMFe4VF@BD4 zUOHIfXJP^w2iUCQc88vbDb(vv47#fKgN@}}+T_TDvCX$|MKfM7CW3T%>!Ar)skKG% zPdhH0A%$0ujHFtIBu`IGJY$a!mG|0eI1G{gVNv9}1CVR~zlgo<9;USbOIc!PX@Bk@=PUc|AmnHoF&=^#_*~^N?Ro$!x{!Yf zSPJUKQ-y6)#cVz{>Y6AsfQz|8WJegc2X~ufdZN{y$z!VPZ_;E_nkU%DK(>p(TVn~) z>0F(O6HbxV@#Q$)KP_^ZVNNE%H?mKk{C(P1G8kpq!tP!TxXer&w;#?|&B=amT7+E| z#W;$HBP7)gH?%$&Z#lM%e>V-dK?#SRA*cjODlZ$a`x=p#5@z*{NM6ZB+ybRZO8>NG zC+WJ+%$W)=!=2foAN}!HUC$WZZ-Edka7BPVNofpt?)gC}u7z$k6X73!VEOfgK|Qw- zp^wp;hY(`v6sQtzzO7D~dyVCm;;IkC4QXQ~$yXn3u1pFK2D5>6OO;%tkcs092pcWW{+QdEUZ6r{8*n8>- zfDZ;p@XOI1{2*soBKzg0DRBIKYI6fiK*{q-ORz-=MUcNJLrw$Fe-ImRSE`4E-WRp9 z!t2~WXo-o78RCzUJ4oVYquo~#l4EMnj0SE|B(ePXpMLjo<^g8Qp%n3uUF>i?D{kO< zCORp2XEv^~c5qr@JR&np_Wa52^_0jHzX+SMiqZXqgB!NB0d3?6{rf)X37-i^`#s03 zy7$=%t7*vDLg9jMD8|!{v-hg8$d#v7#rum~_l?iC&W_7Uu0aClbC3ha)p@K`fgu(_lwSkAeL!hJr``lqx(ebb6jaBKoYG*ByCk{z5q zG(ON*Sy2xsBv^S&o95S<$hJ<8^ z&VTu45=&{Xalfgy%#Y(y9sVN$tC6eeJ6U~~H+>N=a3JG0j=SCr3-4P2yXP^hXtIq6 z6(+zD-#BpLNGjOiUXkH$k`A8Ft>?q!!>L_SI~1L7wKb6=|4QCfv)1QPShW0XXeX_h zlYc%c0)@q&fp_X5w3OVvj$uubuC6#sE)eC?WbfWoZHkCyp3B2e->;~1KGJ%_VYwDH z1jt0>Oy|R;RHK{vy64imhtd0Gb zbS2I_n>!WT5Oydz)5&rI>j*DKPUpyKF9gKMiWrIQzox5owKa5d=1G#0cL_v$+k0--@e&&6+5peVBA$06S=G-7?hXF*TH*^-I_z0Sm`I(%v zbdwz^^f9}U(tENs9nL>CWtDx* zL)Y3Wm3@|RlUWUAElm+vn!;9cu0f)V-0MG4&ROZ5CeZ(_Yo~Dd0mBGZ7wzF27C2}KxRH*KJ zw;^oF(Zro%_<&YO?Xu_7cq!^E8Gj8{on*nf;?`8OxkHuwL`uYi%^AT@vs9!0Zgzdp z3u?hW!K}R;hox6EBRF_U)L^9Obe?;~&6+TwLnEZ-CF{c(Un)S_)%$9c(WbeN8)#h_2if)DKR6H?3Z3F`efN(LQc9|zl<2{WtQ&dGpG^6OrN@%qQNSZPS)2wE%M<%{3j8@rni1Y zOgXNtx?F|S{s4*v8jmhSP1n&JBxG2yA zEm@D7dj)fw;20uBEQEg<+&g}o zt*c}Y%FCzI=Bo$J&4Zy-SHE}5Bl54NNiBBX>MsKOJ$}>M(F{G@SY>Q-zX}uGRl4Gm z{6OLI37z|M!N@UVk~9c4c&kJkF2)}Uw=omU-oyj*IDJnKxLg_=e%{-{agY~vTx3+*Q7BqPb)^<$T}Yj z$SC6EVh65yoriSzI}JJWeK3zTzg>-^lR{FJqE+s3%J=>M&920Sl(w!I_Sjg=>(BLA zMG7-9&Xpl!`m_hXdBP$^G6DnU>=m4Km6>guIYqF8z`x()#!vKt|^R2`xI|WMC9-Ey^^GKI?-ntzx%X%F70~b{GvdeV(+!Y8 zckPxN!v^ey$FcLJ}^w=0Ka<4(C8em2@R~Qw# z9l`pj+WpdAC@GrL#`JOhyl(s)bgra5zbY2sS_@4?(MCU(atLxcS-ff7CDpFad*c^l z;BuC!c;qV!!H#|la#Z2TKyu%&=&UUuVW%1Z^wkT2JY{i3G<^EY5L*7%p%C73dKCU{ zV=z*dz38ZXzw3?D8q}-_4ad=Mi;a6XEwMIf>V5IB1_fFP3`n+Zu66tsmYVJE+h6bGv(>JqNDl=v@Qy#e_nKybp79lPb*g@pIn@#Md0{&1dZBD5c7Inq zFtL}Ut?HN3JRb1zMPyD20f$MgeuZgTalXXF+uOr6(5!Z&-#%Go%U0? zJhH6QD9$7qzEz~(evDalLQpNr9srF8+7c8W^1j+QRnMCILV-9`MR-a>hnj#1RR%y#?Bi`V^giVFe6=smuqyzM!aL2-mst-9A>ry**oxyFXtw7H zOti@<)<^PRX0mZ75IDvww9jU?l1~5CfWu!M^0UEpYO^?|A{Q{qK0Fohm8)_kduROB zU`ksI96838sccY|4B2Bg*09A_H%E72-kMLL+}$gdb((VC=XBK~AlKEbF;N#8NhUCk z&Z#Q&gJfP)0_G6?wg;|@G9>*3-2;i=is zY;YP^B~>`f+1G^zZF0T)$9d-PV+xr{&aj(73-qs=U{Z^5Rhbe}AKCebv)YsbSJ4Co{oaCPb3O<6F$EX0{Mhn@T<+$L`xe z-GdXIsp}n1%W*Xs)pc@aguo}V!AJmBoyD9rv)1%5K7diDtEQe ztHyc(=ncB2{aP@@VC9zt}(-L7ikYRoLC(^T3}i2A+t*t zNwU{~A5iKuOghmgYn!m+Nm^G#zIPUkl21fB?9M%B)*R1TYPbp4!VT_#SXHvlgA#&U< zxX!=Xe^C}%ihkh}{JD{(Dz>@n7E$#b8S;5dgbsJfmWA1z-&ulM6s9)_M?aAEmP>iP z+G?;CN3gPKPZ8M!*HN zQkD-5%LnK+i4=;5hjcsKaWT%) z<+DqrwFT^!XINZ+cC}8w08s*_ipx7#{)L=v7bUtV6ut(>a`xU^Nn6=$u9Wyw9Mx$E`eLppdB;B;c>E|{$=ya@&k?4 z=Z=xgF5=+v*2L>BSCXMU!mXw+bJhMm+SGN5?&r!F1baA`N#c^UYZ%za7H^+oj~%xV zF3i-~&p;}~$g#ibXVrPSu_}L8ErgO_1 z|JZmti#X|T@U`!6suCy%T*h1w#gs2*$!t(&??Vm_3qVM1Q}pG>{{FCZHg3Q3>o8X$5POqPf5aTRLwmBB~< zDTwMHCtg{Se+8e!7VXV9ms%>SaqQW$lHYnY!4LGwe&NUbF47i^HW(AwYwYAC>u`my zIc>kiZAPIl`7){Vz5UKQx^0V`qP#5%_^zJFZ!T=-+l~3$4XuE!Ecwc!U$@~Df6S4P z?-Vg(BqJ?lo-M%$hz#WpohFVdD_W92loWwa^jAiImD-$2!qHhC*KY4I^6XNUC`pn0 zm}0ZjexLi*V(;trZq7JR4`#L}=EkQgoO6Pz?Zwb4``Oy)Apk@!Cj=4LProL{xq`?K zg)z<&#l0Jq_*SumKCL|3BzI~>OHgP#1?R@b)Y@j;4_N|TLx2}I2 z0=qB8cFMH1K^?K-1M(#B;R;tZq#6Y_-Q1o{Lg%gWKB7qEJTb7>Ybt9)?+odxZ*!Gd z{iXE^mw{r^G$%`1`sAx?2Y>})5A?%jDpheyiiA;8fqI4>)0ZDgFB1QDx<58x=8^me z04~;S&SP-%yd`oUx733T>9*W1tbq4^*Yi7F2_tpgkXY-Mmkux(H2XIm>wHM{6x;wm zctJ1r#kNo61Na+2Tw~?r6;(1#Y`Ws*ekdW?-qa~RuU4#G2f~vFZE3d?YurswhkVYg zLQRVr^;Q^{Qt>7v)t1gyw6^@HnItYEJp+Z+K^o0}v;b8}RBN>e4syl63&=@_xE&Ly;=+Feisj#9 z0_B#vm$YS>zkUW-5+nM*pSg20wnXNvyD&|0lvs%_-VoJcPSmy zSdq`X0&j;eC7UBf{&V8*E;wd1cAJW?KP_lf+}+*J|B4Wq#s9z#KvyFpWIgJ#efexv zJ8QCwGf66ON(}OYza@aVu7g! zn+zs!O9ichmi9JzTnimBSqn(P!1!~wgEN(c2ZvUs)&xw0>M%uDj(=GT%whA5zumh0 zjc6brUBYiO0??~bdoF>Ub6fV2r$vgv&KErcEkD$Rrv)yf*>fWI--0UO1F1=2Z)}6| zWIkS*Tzh)plb6G{u3oWMC~}pSV6tHrR~C+s@QL0tYXoUAN8X13|GwTtq^^-dcj(W; zDHz{IV`s06GjWVqed5u9ww~Lc&rogc$O@vdb`8ocWWEiAa)Q|NKRshz%iYw3ITEh` zQ~tXzncyJXdmCBSGeaenN3F6PkI?;V!rAZN2L2Q>I1k#HT=w!rl5ZB%G>58Jn4Y$& zI^82M$dbDf4N-_h?oRo1Q&vMoW*`wucaw_3&-)l;&nNlb~7=w3LfbXRi9*%@o!-vrUnZo7!ZJkLcYtWyv?#0v6)oTy*e2DvkXMmt=bOB=kB@5Ew33A|UVwO4x$ zdvn zbJe}Q)_L^Cpdr@PDUB}3KCNb{SiH%}Iq2ik+3B5wDYlK61Gc2dwQH_ev$x{fz*IwF zm9dejOrq8&c$m#t*-foUmkd>PaUsw+toOyThiSkm;_W{+weq&Mxan-hQW6mqZl!*D z!Es~d!qHr}u-XUj!*O1N7XY-W55fZxko-UI&t^z0;Q2CW@q%|F85GCQWfo$Qr9dH%ceMpOK3w)!j=RTH$cJvyq z&f28)CC-1Ni=x*cBiYI<$Vwi6Jr5ktbkeT0=NL~-Qfk@~;gobB2#BqxpO>*8rJTJm z7k}}c^%NhUUqIlj*|B_Unl3rtFo-yi^hHaSe)#w`2vpYPwx#FSA{8l?7;8rqeO*FuP?sB*H4*gMSjeN8h;A6 zS2t`5Cp7SgbE>v~Ms+K0-L&^Qlnoy^{?PlMiS6=AD58QT+|+h>uf>Bb*GE!tU@3rP z_>V|RiGdR{Bec;O;H~m|qS$;j3&AFST@Xt9QDxr1so*=}guBcua%|1j+PyB>kTZYv z?2AuE#nsXBJEA@@e^1JPG8=(ZXmAMCdPUmKl&0!0eI-54l6L$+&E#$?1E(}~-9#NbrdsYj33Zz(Jtr$LrVy&eAWJOvm^7GQr`Y*| z4c&)QA-i~%D?JZzoaP|3T60nG^!eJf7(!~<;k~T0y1n#DPq&C}hE#Bd_1y8z&4yu_ zXY2d$%j47=+pjo7202U;BdRvG^H8_lw31Ob!<}Kbp~ldNSw32W68(X^4TCG)!TJsP zZ`hvc-TO&$@~y|HFUaaJAjVqSwmNq;U)PF6y@pb? z47<1BoJ*w{n1GTL#8|HexxvZhN#xcy<7tNgyDlivRcPT#DbK1e7WI&f96r-C`}^(O z_`O`WV^OiIk!KU^?#Yu07m1z=k{7G7RFACnUH&Frxq_GK?s4jrM6GCwf@C!fJ+TEb zfF(5Ym#k;S;_gqyP)f3Q8wQ@5_tec^3$#i1f(QBry_e9dd&|7|J!bXO$A%3z5v`_! zwWlJUYDs>EPR0MMa1W`1Xu>^^DagT|XjjEpl)zcqx1FeceyAeix&DAv$4JvlsafNr z@*lKZ4+MnrQSCATiiQ!gTs$YbnAt$B1l_+|5#IkO?+_&ZdifU$5k+C-{8yHTAo0J< zQemV23jC~i*WIi3kCX>GbxciJFB4*t#qQtMIHW3MfBm+V+385!ANGeUAExrpecw0V zNF>-!tK@zS&Hcb+MTJC%qt(ESrXF}p9mg_5yh%STR!L+fX9sHG*yShLjjDXFGgEOX zh@TDuBG1WLhQ}v*H6jXEGQTpDrU?5tU{PF)6^q9IoY4DwJg5e!oJ<-WJ_B=Ue4m9W9ESo~asU$^CwN?uSd6#xJzfdzJ>f$L2y*tE9b`&Y_NeO=XVPEXwQd>fhlm;b0EMRY$#GAEaog0Zo= zm59?9!Y0u$O08h;M;I)GYiBt{hy~7A)}{&eo2ud@*>@I`W|cTa`K8hWrNJN;+CuB) zmU;Vwi<=wMpVEpBlwMWhp$;RJ4!Jn8Vi&d_nAoE&D3Ry@jD$(Q1*@e!e9NLr8Geh? z;<}(o{NC&^V13|)ej?UXyo{Jd%vO_tKhkeG7r&?L7y}dd$Ep`_0^3l4qsDOoqzC^=&Uz0NV zD=7#w;lMTma%sST#XrG+Fo`L8(YD3(+#ITs6wziKb35rCSn>PKK(MADC`EJ7$sfdx zMn?j)lA%_~3TSf-jX$?!y9HtUy^BfRw_#ZK0jFJ~-V&%f9(HVx>s4fi;Hy(Uk{C== zAB-bDBBK>3eTF+x{M&d8if>eh6K4iNY6n2?rvX0KqwZhOh0&*)U0CibO22Q{mfkw?he&hXl7Lc0zCZI0WlVDDK{k5ao7XQdqyHzZW=N7QuIJb~HPgQKoFBAu zFqTMH=T@o7QyYl5B8C34cm@&X=jVqW-@-1cR)Kr9CgGfwF8fWg23{6?IcXXETIP|< zP!yq?kuKjo4v4RYAGQy(nysz={(4(PSVXc_d9>W+pvaNJumly(qjSSO$|SnToL84h zYu&dT-n%;cdA2Vk1a@&K#kGoh3Ob!j-JxjmDba0?!!rZ}U34VQX^1jh6vb7Wn*Cgi z1qliIQGbGX*7|K$#y>ExE3>06=p~j?i`>`0D~lNiT+Z;kNHMKEo}7dUZ=-XC405eM zRTVAaDqhA$aF~a2E≧G*`xxMD8qd%q~c|vJy0q#iy>JWCo~E(3oMCx1Mx8fBNpk zuokD@Z?oU=i?rpyMowt;TJ^Ikl2*!GPIIhoPH(MeIT9c2K3H%2>c(5$9^(@ z3nV6$(3BHm?C38Orb0zP_9^va-IRVeVR0tNgLUx<8}$&Uhd7h8`U>CYIw@@?AoK2& zlR>vaotrEE0o6yoL~E9E36N01-WwT*#)AzK>jBy>C&Z1f9&_7IN&uMTn9 zP^noGGUOy`Ve!pBSV77F^FJFRGd2;YsC{ML2a&g)-YEa?PD4ZZq$qvDwh$*`{ID<9d;VnA*l5@AjR5aDnAW<%SX7OdZ&%a#71k8;9N6i3&ZKwCUwH) z?&pWsWp$7B>wKLYhZ{?Syc|iEzino^$^o!QjP&OKtSLpf)Z38OP%0_T{NwpIPM!E%3hsgDAp$iO zrG0BtF?hn}U;^L{9b=)8m&Y50?RtV8V>V5PL;&kCv?I`6g)pgqJ7FL>U*0%80cY_y z>(Xomo)CiP&{a}jf4kc6{P?1xO6g{wXC)Tv)av;^DCzQOGlY^Ra?1zPTRb)wv!aWsdy!gA^^6C)9V_m55vopFFBS975qG7FIjZA9F}oWatZ zO9UMwCu2m8R5+MqyPh!>#Td97hV7!yWao`0aJUp-84=4W)1LBQzC1`y*-u zbt0UbLb+_;-4K(omUzEb8Cu%-O|WKTu|ED=g{Hu2eYa>=tiITfmTD#ffxUaF_8tR{ zcJOga{Uqf|`u=DqoMo|QiJ+v5&>dK>S-Gn1GP4Am%Zn;n{^Zdyj{5|DH z3Fy8g_m0*NYOcE=nCG#Gj%2gnsDHxVxpBZhUW?(VWfVl1fba7moG`-pt%e{aegKc# zo5alAmojXz$?M!jlHS`)gsqLWoqkw8XZ5<40u+EfsbyR$GG3Z2+ zN#ryP5*QJqZs8V-iQe6Gk_Ze7j24vHPNZHqMKmH?j4O%UQ{jf$4_%PH1#d=!EGUge zLnYWj`Rq=szRIsftR?)RfY7&HWT8|N@^ezg+#u1@L;7jWSJmvVv&*r*hz>KfQ++U4 zWh6DOH^&E>s7sZt+G;8@hK2FSfHbOR*zR7u@WOK^uhc6smYfkD18vlnXw z{$agzSvg^oT2AjQ2wo)7D^LZ@%rjC00Pt>HMyZ=x0R;rIs?h|5vigc@`kdOxSL&(l<*7n7%^|NQgnu9DA`0?(T8#J`k>w4 z{7z>jvu;`j5ie^Ukwk{N1&AZ$3VHhUX)DX;1{#d-TAaT-q%=YB1tYah!qy69uWlc97T)%CIdEIWGjfc~@E~p@VPO5Cd z9MliLgjSGKapRh;a+`rek0~i?%{=jIV$JLvt0Ru_L@tyqgX?4A^KRwnj6LLXXKm>^ z-GcE|Mo%qz ze=gBCSk zog3oz#U`iuI9t*Yc`}L7`*0Y*PF6C8k#u)u;y1Iba49e9t(mY505@{~yQy5L8G^F5 z31K?%Q;*qWMWrm-qFT%ab5e4&HOePo^o=BbxX*HSU~T|pt5c4zT8uEgIp(iFFnsdR z*gN>HCR(XH=NBwon*y=*>o8 z*1mh{RfaNn6n?<@8pREGj zI9W}7yIbeXjNLp|R(k(5@RSzlgh0Ncru!b|N2K3OLx3WNant%02)W2~FomK=)-&2*TJUyZ>@3}1iAC;C>$pOTh9Q>4@Xw@SSR zK{YDJ*49@IN-Bzc))DIO9v2g)+DG&zX3^3 zc8N-3jo7&pOryuV=T}jN3Ko^|hXalLZ=bJVu@NRO3NdA=PY5u5U;%@%0VR~Gj)^E? z8Zj(weV*SGz_DqGtLMI~K|A``9V&Si)3woCLQ|Ve7H;RWRDcGSED# zFK(C?7@lx6(4Kt9p3H0obLPW%4(MGDqT<%GMh-kY zXAyh|Y%|38=zSM*VA_9^aCf)OKW(Sm%Ij+{2S49v!9wWiV}7DH@S2MznRiC=8i*JA zPRrbHo$zDs_?KH#wH5ap4 z?g`bTd&7#9T;oVuM`Am2GV@5N3QZR-x}?Fb=s(K5L{VIU*C9J%&uwD_P3TmWai1Mg z2sLe%FALSq(eTc47FLE?k=5#BHJovaRx9^C8U-$4u_L<5Q|E1GTJ6beU7hHW)ygwi zaDH2MO6-S>->BJ=09A^!XhhTK`%?Dyz{r3=bXvz1+&^AQl8yx1K4v2J)m6}2pU-1T zG<}9$Zaip6d-WY2w;6u7N9&cVMi~C+)w?I5z5KT1;_0WMZ5^uuQ!i4z>xjA4swdf5^py%j0XcLg5X(kU=>5`o|M;qQc3oY2P zIB!^Hsu9mR4}E?BJiB!&?O+xE(1T&fsQ zN;X3&32SmTo1vJK%C!A6Y;}Nv4eAi%#Zy<-qeoNrGuBeM-?;>_^!Xt7$RzL~d)h;9;yfjYLNOIaL;dWe6!)SZhtn}$0x&9lae zT7S`m%A-j~${UHnq>ueM#79ZbXPY>4QHxXhH)j*$vmb?{f3wk7H`ax}-jeP|d?>G~ z3u$o5{@UNawl|%VA|dnv2NpizxaUav+Ww6XM0Ppd{+lVLUMLv8D9 ziI>~fa0peYI+A}`2yjmKEhZFvP1WnON;t{jpXk4=u||Qmk6nx_BL-}&RwD|G&dQ`O z;jMJPicV+mrNqytv4jIwW6q=Qlt$?)&Ozyu*|t!|mel!#bwbJ>uLcq4)v9md+T&94 zGLgcEDhBS>q7hg}&YQs|U2Y@no(_Bo61AzG3HXbwEqE=>*neGxb&u(LkTiR?yuFq=gdFG*MQy7yBop*}L!vuR-?Jj@!owG6FNa3A+mupi>Z5cZT?u z*&drm?N?LZlwi6$!OX(q>Lh*RNF|3u`#5+$V?}S3Xq}6GbWaq9aORZi(+$4NNF7lV zcL%r7x?7CrOK;@^qoj|?pDdp7T6w1r%J2waxG*O+^l4IcNbQApQ}4uvFk<i7GuBJdDJ$lzpEr%o2qX%U>5R0~!x8dn_jj=P__U zP?PNKmWJ~pYd=_UT)k*4lhGYTOIT5_8IWcutDXVc{v(QA5>vW|JcFdNT5?I9X?N#aaS zv<``qoSVFdbxKN8=ZEiT$<5GrcB#b-h%*bB2Nr+ntUU4+d*9Cw6Z7mlcHeP5OU10E zYAr2Ii3sRvxN8A9SwDz*&=y8?wslF&LtAm{4;wfnY_C}dAF%ekM>Ftz$7JvBK4-CO z4;rfPfg}j;?!ZYl63WK#Jc2r*CdDud2b=%4{75nhfN26U$3aPa>c=xz8MKT>?>!kMq5;?alo#TY!1`PB*FE-{)+}y_-wt>!d9_n zSFj|kVhE`9*?z}s9p^a`qEFJ8@le5hSvtyqM@b)1EG#=9gs?Y*M%f$>#-Kn(guF?3#s@MeKWsem!i|&l(*UDP;Ai%D^#YEnNl8j1@ z%gb-ac%bw8Tn!S0iGaREjM_V_yurS_Zzz>wx8hW|o`9uem(HieA*#tX@eZ}WZ{%*D zm#Q1m92joJ($+W5QQ2X&WU&s9CO{J+z)eV}_uwP@xCpNteXlFxVyXKh^anWi)5RG$ zC^=a`K+t}ayoRUIpuCTdJZhZC(?SL!Bu&$zN6vhyU>;deSchtLgmOhr^RC z^p->*HC4Z{<&FH>Pq8k{RtUR5p^a&PWXR^KXCr68)s>^bZs;&cg{>G3Lnk?aWJk|v1Bd~<_xRRwcXF$C3=FD(EHnla@Ij#noy|Iyo*1}-DfKT)YJ61rO z0?Tx1UEJ^L=WNJU>NAZoE?Akd#sb>EcCOT1YJ0h0T+(`lw{>jPeIS$0^>?=|=ei&M zFD}2a%;)^^7bFG#e}wO^TV8Oohy*nqQ8wS+Ht1jpT?=Jj7O~Xcx_K>C<%r%f=2Ddu zEQJ*`52)H0hUQ%B|6OtkmOUTg>1wNqVtqvo4dP9E*Ou-}cZt5fz71r}zGry;a#f31 zd;4dY#fPb>iM^>wc?V-s+ODYXly><5Q>s}oHDz8wa*dXdyMttab{WBz)|k4Z!j(u1Be~{>67k?;*4RYEr z%Wx5szcguR|PiqW(ONo2l6ZynVtwV_x4R#f80|F?xOAuS`ImI zQQr7%aysJV0=jgZdA5kB#k~|MuSB&F5SrFix&%qqlo$}VvCe#hqG$Aa96%!&_fDr- ze9QL)Yqgitu~@fN(WWS`j8ZH^>WVP69mRJ|bTP%hrCn`=ZZoj(-sz7IVi&%Cyg0;~ zx8E~OCBcj{c6c{hQfjY@kt$DYs7an>P7Pm?c-P43Eu@UpEWJ2=2?p*NBo!5kbkDXOzKRwmoB;p*!wD0ejBTgd)q@m1Rc* zUbfp@3mL-P!EAi8(Ux4RZP*5=RAqTvn754Owd=fVd2nHdMv_?n;w|?6$~=qUa1>fX zgOfaf^l6^)@wV7Sjh4HZY)giW~O zVY3(Rg1fu$KpUDiHQ4nGeXjwEe&%hVBno~$U68(#(U4|jVM)qHSMNY>GwVfoT;P6^ zE)Wxh(KIGuD;UvuA>oi1S-8}TSF*krphZmeChFH{BiMa+e-^sFpgl&BI}~d1sn+RW z$D^dwPFm3XmFzg9*W$2@gU|!zvIH{lqULZB%JDob-}I{g8O6M3t{htBRTFL@k!lw+L5RAy%G z8CL;LIBvVI#cn$U6w1Z5t1*P8*nB>uzXQO8LX}24bk*g9@@VC#9C2RI&>iEX-E|W; zBJHu@cH-|9@*p!@$K)c8XfmE;F{D3F`>wScSGd>CxUkquV=i8+E(Rv8z7M5$uHjry zb|X=D=zse}iIn+4zwFOvcrIp1DW~&uEw5ditA_8x9tTBNB$HxdK5#FdR6XdByM&bd z0#WP2fpAD86LkkyzGegY@F%e|SY1p!BHDzb*x-RD90Ib2v zslzr)u^MAmIj{Za;gkt}7IBfgGku?HBnSRTciWo$kz4asGP$fsQWs8Se=;rnjYn%? zZ*@(hn-{7rV6`VuEq=?7tlnYzya9*lD>$~|lPwM!_0Z9_kwX*3e)-a6m9cOwk#gBE ziGa|;-dJj1G!7(&s4k7w`5ZXKJp(5XbyAj-Z+{#&|Id53B|NEMXGdIh7`c!l%5A}J z_t<32=KA6eqK~H3&jV29FPzj9KPynQLb!jJJ*Wa9iPR#xU+y?%5mu%EE;fe*|O2-eMzJ?j6%!Wyu;cA_L>BB zn&x=<*3Y|NbYaC9ar^hAPabceTbK`~3PbneZtB)O31vblgkw?cW)C=_*o#tzCFS%H`J_*TD+)`GHm`!9?r3n_A=E}wx+bf0WUR%3#jIyZvCZBS83e07chHFSD9TSDP;Jef;=ut>wLIQ<3d7RePq>y{%yj_IjdX z-9|LfVuQ48TmKKCr7hez^WvdOxAROz_FR;+^)W&5_On6$GRHQ<+TA;4(*jDeFIsQS z*EJoV<=QO(fC09Zk?m6j+xwl%>JI~3>}Svjeb7U{E0N4N)%;1Fg+B~C(88V}lOk^E zopXv;+YrJwY~gAO%c-DUit2>dtt4-TzQ%N?%`2joOv`5Sy05xlTJa+mO11d9&#-aC z#%3S1)9@a9n}rPkIuy~Uqs8RO{ssgYUsCMF=#hHo#s`#_F+pf?^h z3xenq{@mH}*mKUEt3tzCDy^L%2^w*vtp8>pqdcVc%92!0xs#AQLZJ{l^_r zg}feDwv->-9Y?lV1ew?YXb;pYtOem+T3-1=nbMyrN+wE=;A%)#>Xnvb?{%1tPZd@ z;bU~BC(C2XTgI1HtBp4bQ?N~={p1|>=^2kdp`r~goKTunc&>#OWUrDac2~3wC)XK= zQ3|G0W`Zr=q~lU;fv+oAqb|uLUz;f@^D@r4pt{t}Nuyy`b`f+%hYECar!7#QuW!a=>g%WKsEPhZ*_RWB9tk5Ya}h0*z@9jk8{O^Bx< zq06_VlcV;>TrTFH8Qq*pK=aDyW&M&#qs$~~i<%>9l3^BB=jFD8dYZdm%@HG={mq9i zuZB_v$LNiJ(9t2I&F^!FtRi3TVqV{S)4@9T@iVOiWG43Kti@et8ZnqfkgY5+eM{Gu zC)k35sb4pFgYCe>n(OOSSFunGM|Jy4QZ;rQg1$LY-N8kJm#m}iLHr> zw}xa~FYOe(80m>P^%_v)M#1$UYb{6nDA47+mt<%9L(!sV?46!VpEhS=S&(i?lMja# zy#khdAeVe2Ei4~9nf1OV>z|77j46P1<#<@=g5=v2Xh{D0UBY-)l#wI^HaSR5=kldEN8flp|m2xB=F0 z)>)D?)s>zHsw8&l7#=wv)YRAwzy*zfZ$c$jzZDDJ+haw)TxQBo6;3HHb;)Vxb$M9@ zAXxI9olo}#u1rou%lks860~nWY$-rPZj%}nx>6rQfM=(W+MhqZGlt;kM5w{PG8^8K zSKlrc61cIk zVT7!<`lO}BXWahy^vW#>zsHnah%<`&EumYDWQUi5KmfhuEx92@7U#c(n;IVIG=xev zf@a`o6`@mZirc*3f@+TX;G3LGfrU?zP2&}zSdoX?1`|mAuWJ9R}DyBT8c8ODNC#EcK(~V_ZL}t<&>L$7*)xlm^cO zui1KG=};UD)mrZm$vfOZ>L|7Izt_Pz5P+NY_;uS{wS|5=Yd+U`Bw6z_gI2qcM<_gm zGW@CInCRd5Bl@@+{-2Sd!dG88fB(HuAQkWb6nlF9r`SXPKgFJa|1I{E|Iroy^98Xt z*Kp`z*u=W;(nUtqJ?!j(N7YS_^T^>nt(PYm&^|rl$MsF$zRf1GWP0Mu9O%}X^lC-awW}gPfJBt-dI$s}nsN{Sk0|v@w)em`GNH{q-5QWJ?^tGr* z0vsG1Hl3if>(>&S_Q~%VGA4H?FUjs@ zh?%K}wjts}=)*kL>M)DZ+$K>J?^~v4#CRNyR-kmH^E{lFpZ_6sIN*(-V9W=mhiBLV+#+@C(-$;EL`4@IDQz0o%5!fJk_g(3pAqy; zaqBe-5A%PT_qQdsB|;2pR?~f};^}Zs1j7s}WX`|b_3j;Kgt@S6Ar98165EsME#XBF1a|0u*9ZRov<)KZwcZs`zI;qsak z{b_Yt0S*rXeQ*fX{(vF%H_Trda1L1dl>Vuk^P#T4mB>MJ*YiyAz4m;re>;ecIG{HJ+dRsh? zPVpOpYR}^vCEF4}2!dngqW+sDrL&GDBAd4|C?W0YbOr1Wa`SYblIRh|hT&}2v0+{Mx{5>;VHe5KL8>MBK@}oV zpe)>k&--yi8EYziDgnwL$dpAtt2Ok5Smk31&T^#OYHQo4PPjcG8N7B_Udl4v48;ih zC3gHuB~94>74~n2_%0bI?)_cT#bBCr`bMh22;QByLa^ z=Oi-mVSZ0|hw_By6arsK>WBKY=VFBo(M+{)ZL1xzf_j4QRZUmyr@yj1O z5GU~-gH>n~S`jlINC0P)M+8-#=U4ng-R1${fZ+=C7EiwZEi;BVd3hAt(1TSW;Wb!5 zS-Va`mZ(Di{_5wRLMpTPXGK+{gIu}K(MTmr>trs@@SFw~weL6 z*8_$=&&z}D6{6xcX7;UfI9||`AKNMy1LXcR3@6_8_d%^??WZ0l_%w@aCnH@VtbN`H<0D#gkA zog|>^_<+}fuJyQ4tHnKCuk(rGdA0DYrxPKgN=xEvE;i6K3$1F=4hkp|e~P<=1GB6`3o{%e|IeNjN#-Yhxh zFY}3_ukDcjW@zQ(>{0R?G==R~`F#t6TjFjOTXR`WE~Etlb)QZO@hM^(Mn zD$nVS&Zp>@S~j=qBG2P?T^C5Mc|-eD>hIK~z>^pIfBIMTJj}eVT?GnVlg!?wgw`0c zTk6d^zkypexNLdd_sst?T;uck#zyH*;WlymqeUF~*5cG2Yh1;|^>#n;&9(72h52qe z5H^CgKInm6u&Tl6ZwDSY>=&~_ce+Oz8w4t4aaN>s-=6_QSpe%D?RF+P4xg%QjOIp8 zL!G25ty8+8MFip~C7TW1w-)lgU7tgA-k4Iwe3Rq0?t8W8YK}hiqC+URGVTRe|Jr5; z(_oTNV0Jpgf$K)^o*=Yt*=+4*k0c(y=|@CASy!tmrKM&GPz!FWlFufy`Y#hQo69wz z-7G;{-3x27*lgg~sp80@!I14q$Qz!eahBuh(V7Z=M}~Dnv9uqot*p5z&RELlTp(mF zmVYSuz@e7GyS2e<=SBbZ55l;={nPqB8ww@#x}be-+%HG;&R?@0-(M)w92{*3qCQGE z_Hko%9pQWnUr*7fd5C^m7Tr|Ofktz5gS9L6{ULRo7gSq`LO^ESbhkjH^qme|0Fz$byOiXCi8OLYq2Ov}jUAG&NXF&;tyMCM>eE`h!jV}F(Jj?r> zTddd*5G+;H>7#~5*5C%*)z$R@ME}gplzIRxRh5;C^_-ZPm|LL|;K^eB*x1;oLGQ=N zJz}fksYM;t^!1BcEUrrJMU)j4@jie4{BT(D14L3{mbS3?C6YyG2?jIKL_A=ghf}7) zsYUY(3p*V2A7Ig{3ys^^PDXuw{gB@1&E+w$|>`s@!x+>By9RWy>OBLGk9M5 ze>?1w9-8P}mPv828Hzv4{eyjbwtdGi?f8$qy9Kcu|9piMY3g}#SaPIp@*dS-$*rD} zb}p6^5nz7V(fU6h@`}TAPK5gI;Q%@zztrrTl>oXwOmi4yfJ5!r&?>oU2OmQAJ#?GT z?k;iJ|8N0XL9#-=C&GmP=Wj=0uLd7U-c1sBAVmG9xIEQ)a6L0 zDOUf7mUZ;yNy2LW8ro%sUg-|z7UAYNERUsK?WZ8sKl=)S0_{nBRx0L2pw9@O6|3(FBE(v$MK_P zAwWQtIwI($|2vsvP1gO#K0H{}*4T?e-X5fTzy8mydL+vjNjOWaD^gm-7BS=l>x>pA7>wiB zMNx#J?B`1MYzVgi)p=Lb2ISW*oWO(xjYS&M^gZ*lYB%2}Aw3Z?whQg>Y$VKL+u!*e zdT(u?af=%uk$qZ8G&L;su|VsWiiM2|-q69Yyq@i!v0#R_*&yZJRAZeX5~x6=TQOD&`{tg&2=!A= zP8O0U^Rur*ArGW$5mDFDD5+>a7wz#6|CpYOM)-#x(~16d1M#EF+;#nnwEljh?2#6g zVr7$d237Q~`SgalKJkNq@5FhvbcEfPqL9=D+2dfzgSvgkhmNvIJM*K{hQ|eaD6Pqo zY*aFr?Zg!NN{gT6jKsO>%}xZty4SLa3YFpS;mcy4MaMrQ7a(g16x39uA^7g{uDO)A z$27lXQ8t&#@V!{|f|3rm)WGIIVKE72*1$zyEkOFU`OF!>ET~Ls=`lk( zKT!N#i#yvZFmlu_&AbKFu^paOlYe6AGoeMh5Y)bh>1EO;{Q_{=5kSs|CDw9|yJjvA zN;p`$mr$8bC;{3_Vh7xsW5VVgUE$|G2E=tq!`FkWKlw@)E~Y#tqqkK=NcA2@ex-Hj zOr}fASkZr9K!OXy#0#vKrzzdL+^-R#uP_&ybMMdHz`zu}b3eK!2>R>015#IJ_n;(F zf|sy|L3#fNc^sKjMcE23>rzxi;`or-YD*Z{e1DJQc&h!#E$~a(TNk@6_BM=7>68Ks@7PAD@BcC6)E-%N;J_amKh zaSM0JngLuHT$%R8oP!Dy;z7peSlggp$2@u}_?76?K>}+}Edj47 z3>PJv>C50x2kY8CYelCdo7GPFgHCVhuk=cKQCEpMl+-sczROlxA}xV_?sTgR)wrp2 zyY`j!=&)X-EJFxY8iVK4^gJmv0#^>Ae=y2cZh%HcB^XLM0DmJfU99qG*~Ej8SmhZ3 zOiS7jSZaX91?xKAZ3apYi&2dtwe;C;T7OWs@nLqnI( zSlhg%H5KVKH`t^l6{KKUdBohRc)n;Rj`NUo6I1eJi^>l%`6pu#CAc3rZm_{&;i|3E z$`jBmVD!}fcnQv>;V>AmdA|W(s~K47H;wJ^tayvDFy!ae^>PJt_=q48(*lEVXlTHf zKLh7+3CD?X+4mNk^jUyXfLf1-!;jp3)ajrrz6`^~yGWP<+de+CQ-nL=&Cd5X0m?lY zP}GAWwigY&#Nl*?qF&uhg`SCs4^NtM@CjSoh1LqRy-NL^Fzee(k-bOyce4ZPwO zIA&1v^k7sZ0sn0BAdCaurzDXo(^qbKlHC4E`u$pRpp2M(S9G%MqQEd`wH-;maBS1B zJ1TXSODHv&o+Jz;rQE+3524YOa?1~iynYvcKV}x1sQ!NYcgtZ%ui%oq+0$u<2UU(4 zFOX^D`SPQ#?{iS+Gp82g&FI~g$rR;Z`X8##Mumb%9Bt~-brR-$>r7ca-Kg=?VN%P7 zEkErChXUQbcy;qQ*D%&KpA89KSlX;_8a94t8#grm;D86)uk-J|pHRrIjr?7<&VxEq z<^EY2wra=$9FDwi6}!iIK@02bWqJM-QBgNNX#Mv5YMT8TaVmCO`0^WdU#?*rB*7!p z|2O-ua2fWi8B*=B&r;<^PaW`ezamt+as576d8Cl;T^Oc$vHn~PRqtrx_q&YY!Md@> z69wIh65e=nctTPFwT8+JZzZw__$>mAML z)R3<_{|{k*85CEyw2#BVT>=CN7Tn!+upq(R-Q67qcXzkogy8P3!QI`0yTgA@{T}Z7 zoGw9XDLh za-Z%A3$7b-s$3?LGT$R+DDO-!SED-j_t?HGWk9+=;&UQDk63VCl1_=FhSzos?yA2F z4erG-I$EUi`LtyxO%P5z11~ma*`2B%^KjMO&b_XlvZTJVeyWU)kFUJ!$cZG1cdw@n zMsluxtJtXJdmv4DaUd_O#&SQ39#pHM<*;1jRQI7|B>SN3@^> z(^eiJ^FHSFL?O)iJQ9pORIaxpd0*SU{gvPAxiqnnUVrKKJ*E&PviNR9zmCrdpMfW5 zFCL&J6V@6ZwDAg_MlQTbZ@N|+Q$zNCto4$jcz8f(^7z}71;CP}59H8ZdFwLd*kgKx ziDA;WOKlJXjCsGG%PMVnv4y3)jg+Yq4ZI&R1;@bEI^LbTZBJ({ntwg}h$Ta|OEP|$ z*7Bb^n${K>akIe8+V%X&E7%{`H!O>w{luSv#n&6)v)b1*vD&45zdn>QkrXTXLIOWN5Ok9timjp~~(D=y~57D%9p`DB{#;=uEK_kKZp zBzDi$uD3F~7&l_+LDiQeiZ7#6mUvk{or)gt=l~xU7K*4M-dW*!pWe9EbAXN;lxCw1 zk;Ayt6#`n|3;c*Ah1^qAg!}4?JX^o;lZ7CU@<%DZR?Rv@}RGM zen}~`VytYpdGXeX+E8$0cd7!sEN4=BNkA`H{r9SIO3|!q!|~1=(lT_^VaaJ*1^%Nm zlj#C8t!a^^%VIf$ZV)`Y6QZbex^1N^xqz87GOSEG+Wnil_+ocLKJyZ`MbQYeGnD5I zA`*`P%uqbri@xw|#%jc=#<5rCWK)qU;>+h-NlGgG!PpprY>kVLjUbkgZ?*3ru)NXT zCdQ$_wxe6Z5js{A@kc7U>ndrdoV zZ?B^>w9SW<>9b_^<4TLzU6IrNhB%q85q-kmekQ|AqZ!h1zckbC2cP?=`h2c~d=j_q z;)yGR2_uk4lX@D5qs3B4_!x&e9$Fen*qCUq$(v~3t{NOc=CjO^>h#)(qn*5FyI&K4 z9<}MAXCeF3XKb|GHeKp=6L|9r+Lok%=iGRGW4_u{o(PhEe&zd03w*q%<}mwy9HoQ3 zRu^|*0+ZD2AYOsZpYC;2*i7VUKKg1QEDl6=UpZo-Ceomsvw=;tAEIFxU6v8Bn#EaC zu8C@S0qEf9ePJz+BE*6=rk1dKVxgoc3IjzBUvF=HOa?Vd&b{IqtoM|%6~wb^lbjZ8 z-8aP;s!qm9*~R0mmwiiLP$BNruJBUC1-At02mI7H=#KZR{*Lpg)lvjhTNk)u%DeyY zm4zw2qm1+a=^ZjaV*ir}X4?8eJtP!;q&+lW7-Po-G9OR$qY(@0UYqlqOSF-MLM5_O zef0}DWEAvh)bR+^upktz$q9`}i0-1tljS<77_7wx1B4``-jW!#o;Vc7*B)8B{XufQ z8GcUv)!6P)-C@T^?E()xt4~c}X${EWJKN3B9EMfdVCZ`IIp3aJSb7#Ix8#ze2|~Ii zlH`#UFgxeSPJ*k|7-8mNH(cwYwnV2P`Jlb3@@$x$;)b~^8*JUAQ8QcZ76?VwE|aV@ z+TzzNv-5uKvIdh%iWf~|-&2pjoa{+)yQ2z{cX1M}Rt20FFPaV0eR^1SwC~mh&;y6?!oCb@?vL1*FuYy4Yp1nh&CJ>b23{)Fr@; z*hP||;df7>1G@Wl)1b3s?1`I*0?<)v$+lycR**VEnDqT=jd-R@2tMkfhVr<|dW-={ zCcWi@Y7O~ucZ9$UtX|B)+ojtVh2(mS>X))hCDQIo@YSZ_F4P0lJX!wPg}U( zt4G*7)CLIaJ9d%k;bOmBxXl0Y_ARQBWxgK2Hq&_En5X@f_t2+ckWYQ+W`#aBjtH$H zFxVib_GaWSm=c-i6E)Ejf|)?T5@7U1rn2FRl|=#&ZPxW)sV4v=5TB^|&o`H^VdM>b zf=qQv?}8RN6)&R+Ut~Te^B7Rpr&eTb?D*s2%GmqsFOgc-Xn^iLc~JkQ1r=38xmF#4 ztq@d^AwJ!TX*CI3{MJ5U!7o3_;es7Io8}xcls`n5a}%gEu%Aa5$hfN#=*H-r4!+1G zo#>x0Q>fL}5F!<${Po@qu=}>MEIv5E*su8YQ1we*t2sl4N#&rF{kM48y-63$jvG9D zLavA&J-*|G^1{MP41XX}AkBWQy*UD3Z!YDW_ySAg4Biu(4X=>fAIcgOdP5Xb<~nc> zmkE<4l3=idEO7~Koc&Vwx~KdSL{l?s?Sa4Q$$96BEgzwh17-A2J1$|_35{&GuLC;v zNGVa#o9bOnfwK7BI~;sNGYs{NZ+)=t-#@q9X-uyPAYRZl6vkne*+mj14~5*hEZ3Om zVy~{J|IB9#s7I9u4Ev(~q6T*P6K&)^OIQFT$Or+S?;fbJeB?m1?g_L$vbl4?FaX(1 z7&Tfxt2A%AR-f(KqAnk{Z@0C5Vp`oLLs{h>29&-09HA}MBoa5}d;Kft`r2EAIQ)0i z(dyUM1g~$RW#P0(^HHROTB9H?f6K3qz6x9${kwPm=p8;u&lC6}>0d?c0AZ7D(d@Td zHdY2)-nyx4Zid(&ZcXXucihuV?PDrao_o5hFb2D=Iy?wc~|A z@yK!KEGJv@-rCBX$YUcKvxm(>tG8PQ_uavJMI|MF?dn7|367<2!l+JD^)yjfzTE~A zGwn7cOP{JRi{C`3SqQh@=JK1Z1o*=s)p(+pcgFKiw=&!|J+CRpEJ28WzQwX>7&QO- z*>@+lU5|+>-N_T-0vk@UgzSU<4PXc#12Vb#w%!Dv-#+2`SWlcX-g};{;&bK3PB&qP zh(70c+IIha=igmUU&*aa7qkI8TgL=HYv_d*f}!JwRi^X2&>$*z`|f=bD14ldXu^Aq z?&yh`Us@3_@vn+z=kTp&_+X5Vo6w(&xPk#V$kGzc@{-;5pA?I?tTh26Q8W zF}=VZ(Ky?FbmvL4vp^coMxxcK4lc|fj%?(|n3dyn!p8RjW*3r(GMPy+6(7zFc2k|qW-!!Ct znoyH4;EPyGV(EOcsSETU`GJNlw0B@Pg2MN+zpbP1$7Z}0XT(uXj)<%3*HxjbabDb> zpxq=)GmL`(3iaH0#4unMaT(1~@+A%F>URByoRMFv{uYhg)CU{yUN`;Rx@qsVs9 zYvZ>$IG)gyP<&ESAbyPBhL)&u9f(f>lZ**RE8{?kRP5t79cJw&7-}XMZ`Lsy%9Isb z@=Zg1Z1x>T&@=`{h9GQrFS3a>#|X%jx0@!_>;bNwMX*Z-o8`epgif6PiGwvZ7)47= z-w>+<%8;H|hd)*gJ97v-Sm%LIiOo5-UfnsKRFc5u~k`=aXtRI60pf zP2tyG1bb!ejPz7CUl0#S;|SjN-e2DAUuN>VF&9U*C&9cuSl-e9BC&C)&3hN`xz?0$ zhb{)SoY3LQ(Oq{zXgn#WII2f;X&nmmo=#&`FT>`2GvaSL0ur?Ek!w((d|R%Ix}|9; z>D*QNiES|G6^k# zbeG+FufE^xU=_dbg_e0+<1obZJZ)j%ROy`G>M_NE!&`_cs5R<(3JRc#P-3%&3t3)z zKeqrINX3;9c`wo-7z>*fZ@noA6hwxBuQrH~I=+RO@qT-WX%JY1bwB|<{GWTmhq}%0 zg$Lu^9`^Cd0YjC*Tc)LR{*RpNP=;e)&tBJZ?^~QB#KRrZ-)4fY?fozEK9@dMbK!A` z1V%B3u{c^czXhFCFcju1GpVq&CAwR*olLaRzL2Dspm%NCN~YA;K;J!r0yZY?l3H1a2Z`x zozt@jG)j@ph&Xw{RY^&hrBTN*?YzdzNd{pgJFG**P5ZRJ?Ip?+zo5V-{>#Dq0aA2z zr6t1!*!#DG%b4*M=!bP&-TSu{+HT9`8pv!uvkaKdB929$_YTfQ=aYpP!Mk(3WBjV) zjnhh>htvCVYxQsr{0^nmbswY~_4~ zN$NMQ<0k*}7#fl?;!bL~K8#hxFQ2`4T2wN5ra*}=!t0x0NI?(-%;vt(c^``#`f|-h zyyc+Z$H-5(e^Nhtv)i-ZNVC&BjwnUT9+-8n(P*LkU%25!_!FtG;4=%mK!^&w?In4~HKxQkAnYIs4&_$q=f;Kybyk=i!gdPdk zGy8q^-&n)uP|zs8_C=$6l^3p4(FU+d3oir?$?h(!CE}mICl*_bgG*sicQZ%wI)~%5 zrNkKg^gO>(E1(Kr;R6{w*Sq<)S<*9j44GADtvfopqV<j*j71 z+0jY;_7A4M2x#0Wf^wF>0QtkNThOS(As!5I^nZgzcxiS^_qpvbKlLxbVUfj}tcVyO>9SDU z(k@0R=Gy(9D^f>Z2Z1ROF*J}}*g0o+ziD+p>;Dzw@8P0A%-3a}6+5#@wIj{BipL&F zLaF?;xM*=CmEf7*d(&>o>Px7WuCbkXpX9p)jy#Kn15fgG@MCFg2S5LiN8%*Mf+du} zw9DeaKfw{&m2Ha0Ine4Z9?R_UjRO4;I zb)8yV7iRyoBN7VIPOk)A?v5383Kn*y5RNvrO7=+AqWN_xAzW^?GhS%A?rib9qbu~r z*GxHr?_XXi!&o-ZI zOVNJ&7+&^@U4{}NJq~;wYMJHGdBzq;FbnI zV;O3C%JreFPfrwpL4U|qHZPPjC|@g?u6zs7;rUhgr>6VWKJMd`dW_xH%*JL}r4Br9 z&YtYT`q;KDY02^?wD-`IjcvS;Vz zixA7E-!HH9j;kHOm}vkaH+mg&-yy$Oi9T%sw;v`M7_d}Kyl)2q3Guwx`duZ7bsw z?>9Eq|3M3mYdkniZ?-~ovekQ})7{E<-Q_>Dt)*jSgGof@$@tajLEmeER#gfr=MX)6 zn~PDmzh`h@Lx}{tDmersSGXYb`w~)UVCdn0!W!fhDf_t?ir~8=M7gy+BTap=IFpuJ zQs&l)9{~YVj^Anfkei@NjO!nm%znVySwF>e(%zI zFeHqVFR`$d@%NY!4+Q)%(v-;gJ}*{rOJspEBK3zUA`V%QS9MGi$L}3~;MV1C%#3v& zDsK(QS*yz2%4yA{3J4$Gs{zw7g-dx~IcVN4-zfoV^MDD$Me2dxAQr$)a!5yos9)jgiZ&07y2<{VTpHj)v ztBnOu%2|XULLw$6DlFBe44m@8;B8Dz1tIwwflc`{TJv~&w0Lls@-j!Qm@`F-KE)X! z4mjH(-|eNk9(W-y^nQieX>*_hG!YVg)B+&y+p(^$?007{`bgiOu^WJ@$)X7xtO?q;7kJL!V{U2aD@>`|HNezE`FcC(q<~ z$+rL>zM{Yyvx=zr?k_i*O(~sg+Wp^E7g(AaPsIhH&^v&1ST7fI2;n3Uwe=OZfSw2zB}730-ri;nJR|8b1b7l;Znyb-+H(GB9;tHHdz%k z1ths|*L8JtgngHD_j!X;ha1l?&5rI^rtfMS{+7sQz0jaA3L9q7yM8j_7UJb~#QC{f zNatZD$NZ}g9pQ@-pvKXscY6aMe#L^tXlBG^bhpynNN3u4uMf`f4c1|0RqpIE3lP1zX7|xJ39rk%4I#nIKm=5i2d zw=wK&Y1*wax(D@RhOw`AyJ1=nG%WrEd@C}w|b$gOU zss2Jp;fzxZbeeuQ1$9z8EjfOP>Fi)B&RNm4k+$)$B~2ua1&wb2qiCq`WaT*;#(dmDN#Y}x~1$%zzG?y&v{P=G(A0}t3iEl4-lazT;QT^8kr zMW@2)ZE`0pZkZ*TZcnbC0Q3j5@z`Az#m~HPAN@3MitaA@y41PNpUmGL<)lL}HPsK< zFCPiE!&%VCH#`ExP;2Q5BVvK=a*2e9uPx1w#9HnhS13#Wb5toVql8#VE&Nx_KinI_ zRCILukY2e&2xlZzhi;#ohLx<~8t~{7jz{PPX*soI&X=`nUU)D0_^g>^Ox_Rrhy z-}_M3I{f$k14apL5-=Ya#@06#nkvq}eP`$OK!&jSlW%_O_4nj4@|W~v^Re>$dM6&w zlc-6~(R9n1YWah8;I1k_%KVU$RPAD+`2ioJWN)>=PEDssA zV3?iWPk06v#a=X0^SBxE3&n+lshHgTmVxW)~Q;V#1(6{j9F@ixNLs~nXcxR>n zFKD?MfiWyUR+gNfH>zIpsuS0l{q|D`^-q8D25Uc?|G0Lk*<0E>Sz=4r+Ok7d z{Z-BK=8r>?ce&?dEFU6_1HDq4nS`>@pAD8{c;y@bXZ?VLyAP1zJ?LFLRB7OCgSFk9 zlgdyR7%X~LuS2CSK2rKH6_2@@sdmkrgeH-Gm9&AGup%UbjeM4QEKs%$J&hDXq~+O( z2_fhM8<&i~vWJRI+ldZo5)8x@fK!9mpbQxihflgTz}-(7#kw%h87-e7RZv=o=XO-r zz{d+*@gS=|*MO4p)Zn=tU~rslNonQdFKzp0e_C@;K^?Q`Cf`TcxGd7B0RLFT9tClR z4SGie4r_O7e?dVjlT@BF@=CD3K!v}z$ap8seZ~!XZhw3BkI6&>Psxh9n47Pp)9wK_ z=QeLJN+oF`UlXlpODS#wWmh#ctr}V&F;m<4Ck80eyn9nf`RyQ<)XSKeVSKj47g&rx zr3Q}^?333^crN8y;eex@QUaFb_@D3|f5%N-vx3e{6n>8`?Z_A{ufiz})bx?mhQ1mR zVPCNG4HqpGF}`}v;INYhvl!7!JKtB=np0&Bo7}OXf#W|;(KeNTH(i3XgAs~r(C9!m=IB)x~m6YCaTD^irU7e z0Fcs|^|cK{ZV8>*Nx6Da1&Uw!Y~*^WeXg#^a-+*wR8Ap&4}wGz40kfYdTy*BHFBw60g_BbZu>P?j~_K8uVa5Zf>HQ zG1JmMtG0ZACV#9r5GSrH9toMNJT&0kh!KWZ(-15ouW~!;v{hgY)BHw1qg++ zVbEB&xwX~YYJSd63iD+%@ROntGgqKL9>N;mV)Sbh8OiV{1OxsiGMrYjDv*0=6rHW=h8ok68f)^@!h;WI9Gw9TB*=g6i(c-rTq!9w$E|g^f_B8it8% z&@%PmU#`2S(S}MFM_{zSF&CFQO0f|*1cg@D!*$p7y1S|`AvP?i#LBzJU5|(7l0{FL zei3^s6FY7ddy78B&I#P~Z9+Ee07uQR$_+AjT>)1x*~v9kZU82spSvBbwt92XH(=Yu zagzbdup3Z?G}CzI*T1^1eu>Pn&7q!7sx~b?P~C^bsY9_;VY4cfBIn4z3R%-?vA+p<;430(gLkWu7x)hPBE*qXOJmkkbT8yO4|2B2BI|ZJ4UQ zV86q7;o;D6hkFDeSL|y+xDT63@qhf}>F@N1-QYT`H{Ar^?8o)$7?<7g!3V!S=p$Rc|+gbMZ=jMR`eW|DIFm zriR2dH|KV&QnM2!w9KtiYdoi8B>|5oT+S@BGHpC-9V*A}Mo&`wj+K$p5xpWGw$v1x zyw8Ho>GmyXo%*h>knx;(s|d%9X@~w1(=yv9NL&@JG>BnZ-HQO7`qvN`_!G5pAW4%l zOIXFm@Ee<@naxH|Lo)bDdi|^|cMO;zMU`duwbie&$)wtVc3WO?F&B7Xd8pe_v(f`f z&E0)v9Clis@fNPrLSrma7rqM@$;1;sjjns#&6{5!4`!9KdSqDyeO|c{EyT#V@o|K# zA}Ypoy$u-M8a~Ej$GF7*<@4fDLZ8QMf!7&>H9!AfExD#<1k=-FbLOi5H@$=LEFZfL zeqGx%IU`DGyfvTwf68e+=VL7UJDmnwca;; z3F>h6Y7U;HxD;t}Kxgc#8u&~L0)zah?^Z7*rNq?50D_fzW7vGM5cR%R-{i7ZrXAAv zJ$vqX2L0!7XCs6O)1J8*DttBJKYe=VU;nCD=Z?1YQH=d;D@lx(4;UVfnwn{jTjMcQ z%Hj$DgMdF=Y74}tNvY|~_brR4Fc=E8_^JK7K-k{FQcO;3+u`t&iQx~b-}F5z&56H9 zRVE1tMnf&>3u=mDs|atBwAeZ0P`7@fz3eMnc{@T_)4^FVwEOmyk*D9OvUm)@8dUS0 zXkAi2e;uKugpz6yZ~h7rH#4hQsfONFt3L>zW+wV}iR#${qhX+u+I3bo>1&#hdGM3J+W+}IH6>_cQ?VX!Z`g63@S|{O zcv#->#xYF_dgES#6wM+MSl-&2VaF*&`g23&?tbAQGZPPQ5Oj&6AnN4Y+;`(NuT?pb}O z;VEy<=g|}|iwLBCCRk)7eS8=SNMyVQm+JF?W1(0OQuk@F9|;T;m_RLDj7xPvZ+f=Z z9^))<9d>lpLU!~A;^8$`Mqly}%uYaGL_5Fy*=i(o1G_op0Cu$^KRDy9PHK{6R`Wx` z`0q7pe%eW0H{`C$gdv}8we65S@3^zx74Y8cLV>k9w{N>mG&sUXI)i%XCL{nfG+Lkd z-LAW9VMf-wt#^B3Ic%PtCcoqCFg8dLj)46im7=^{Drzd`|4}L2-Q7X!WOcsYj`8Ko zQLiPyiKr;o)3a(9Pe+hpnLB!3_bReRIxd1R_(oIl++R9?w^HqQ8!ID+u@nrY}d?l zZ66yS?(pvkYpekez0XqTr{`Z`30uZnt=}hKH?dQ?4Im<14;|71T#nWVe7lb6s_pv* zef0PU!Iir|=x$v4th@h;oQg!J{||s*@jqOR5)fB|3)=ho&oI>x_HEWjDE};o{uG7> zK>>No7mMW=x`N9-N;PRKV2CoSelwT;ag{v?<#0fJGx^!;(}8pG+gffhCZwBNVAc8X zDgfvxHR=}uwXJ&Sx&@PTMLio<6zwlk?|H|ZDg+zPq*6dK`Bq!t?w|v5Ow3AEk-cys=qSwM}d~eLy;cOX@O*j^QEk9O& zk_8aDjU~JdV+ZVT`dnm-K5bamRv6#bm*jju@7>)+gAaOIGcCK{;oGwh8Rc<3>$Arz zujd}!EohhX+JMQes6aMEoY&~7(6?gGHO$eMjp;^+KEZRrZ<^Hj`WGn;8?xcOK@%0S zuCZ~)JzJ&#=;dw1jH6JBQL6M~V(;}K%Er``B6|3;d)UVrMm=>(3O)ZX0OfQajJgRgCsiXerDgDKFVowTKJln1yvIe~aT{~=!& zJ=+5s8yezeib3S-*Hzi)#d~jVZte@-O5K}kQ1~`;w1b=)!=nZItC=A&Vt8n{5M*jCJV@+Nua9KC8j{pW({QsBj~%k?8FMh{z&Fvuu8d$O z?%&WygPUAB3vjbB#>_aNsZi3>s~FxGo}QjIRY)?YDS>`J-=7yZHK_rHXDw?pxBnl#8$<&K z*~I^&cjx8`fR6Zo=-pD}VWRmKi0%9lPmcx9nQm>mHU#%onBE!m#Ur>*n<(_DCAVQ@Q$b8z?2kgLWutWs-}&BetES5qLDT7k zDeOR0Ysvy*{blH`uLoY&cW(A5m;tT%W6s;+CGsWLi^Upzf^w!_o!yLnT|0Z|# zL?CSFnQCI9(-`h@v81sZ($49ByEt0C!b9O}57!>U`MEdqFq!D0qrJ~`^tU;ZYK&r9 zeInhMp+;}ma2jy$OQ!s1iXttI z4|qmoH|ELrubfC2r@WAJhMAd7Vez>N`VR#qzU(wroj(QBs6nP$LyJAI&Gn8=X9L$b zir9I;%bAxupPYi~UD}F=!+I@7`ktZ<{t<-A*)2l}=Zd}IRzf}bS8)>`BZ68Qj*Fgr zl7Q$2q+V*@L zIzIfp_xxjPjSK}9)taV(|7SfzLx(;;_=?gG;xM~^JveWRf$_P8_{t$|>}CuWM7F1d zf+uCI4m>wVBEeC@AQPp_A4DV>6ew{ZPVsi9?iUVb{05iUT=fX^VlpO5pQ$W*)}ZQU z;0}lN`T#zTm-tO7LGpT;qm(uj6+Y;&RJS`^^-huH)nk7QM+OaBA+>A69&4hLH;n}* zsp@h{RU+qEo?I4N0t5TzQ9+b7kWxC(N%WtAhT{35&`L)oM{SU zK9POIz(6q_xnW6VH0D*FqSeu>esI&lgiX*?7X@1I;Oe-iFVun+d7|*#8{5Manb-4A zWoaIkZ zow_c%y11TTelLAZ#KKhMb0f_k{O-*DU*pR|u`97jY^x!XqQ~uN8b*>@UK2MQC2%=c z>wbBE;kwm*r-1-?=X$wB9iXjpB@KzDdfi#ulR|Simj+5hT693D7QJB zOqNrvaU5L<;Od+@(Lbl~O3aIK%K=m$;!Stchd@O)PL@toHw=9k>2ENak}?hcUY}Rmp7ehr=iABmCsYBHKw&=LHY^+_@h1tCE}#> z?`pLbqnHQ*Y#A(bo{>7;+&=%hcaFaVufhbZNGp2tbogbCK=KQ0?i4CMV2V3WCXFa? z)`a&6VGY&ztdJ+SDTvL2Nu`Q7vz@JTH?SF1eJbTE*J+JdqvM|!l9z`2LdK)s;11&q zJU=s9xvO%XvzR|Q8}N{}(y0y~L&AXv6~TL7XyR?`uFV(wAMEen< zJo7m4gB>|sxIFt7H>K8kl5(}>)M{l$bojzCjqf3Z*LmX?VygwVm0LUq54X81O;xto zc-K^9Mtyi8Lh^->PJCJ~Qmd|RSLH#Y<35FriP985=GvpS4$cc)huxm|545#eh9yyp zlO@Cmve4tq4`1u^=f`~h-1O1!h&%_({Bf!y*Rphbg<>i*Nq%;;G4adlT8_hyTAbWq z0;hi>(q>vRA3@W~*3U8NxrwOelf6D^A9bU4MZ?l@hvJYuH=44Jx!B(sEq$Bq_u5Us ziu1n00lKd!_QB}(&6%E=?%Xj#@~cy)Ih}<()=t*Gyfx=s>Bb1j9I5RCuSha|Y{>F* zGjThfa56XALERDD;}fxn)VU)O`0crNeq1S|3IA&>VZD6OpY{~_P3SqI(w?&=`$Kfc zb6#^sH#+imX{q_0F|qJ#R;z5YmYz{E@=RyHcscY$kv@leu=3q8lPif-p!Yxx;SLo< z@b?!)AYc1N3wv$S^4lNN-hKdSAp6GTAoF`|Y;vwTb*18!Q@7Qc%9Fe1=ENfm2D7}_ z@(RRw2t#3}`$rM~a~yxM8YV7L8fOOJ{lN0ETl>8?#nz@bVpoynjl}DyB@SbcM&kp! z{rs8hvrIcHJ4T`1G}6o3edQoQ(-=thrn?02kp&PDkqK3FEUC1&({j%KXa5i7gy)n0 z`~2ea|EJ66|6gzU-!hl{|6w*qYaEIIHqQvepYNI)^nFDZSCli>jkx(XeQUQwJuMZ?EarIdKJ^V4GXnDHX=<#ud$DED0v**~9`#-Z=(AN#^1h9iQMDgIr9=pi_` z6x`{ms(uFv@Sdk#DS3dFv}Y6Q2C8Ln5IhX^L|Pr(bJxCVtZt@iiwo$ypjd8kLX7KX?GmBH>}o%AaR)H`+3_L zuNBoMgCBJL188mp*&7@!OQ4-YhJk#W99>25AF3yp8lVdrGcp3#p$i(aLskzd`gl87 zxM_adl*O&7>EWTFkuMUR7|jJmGhli7)5hrbbMm^jot=<=Per$fdINpt1o}$lW+YS4 zE~$g(O85N*FX4tp?)jxK(Q3!3CO~o?0k*t9@FyxHWDD}YhmtZ#CP(=8@MLvFal<3| z!B}@6SC=csAHp?BnaRlMi}ClEntHbH)KIB><(@}OXD!DP4NhDlKfgp~yK^mg|3w=Z zfbhQ&>hOGp*!4Ci*RCu#SXr*0ZN<6_(xgDOSnH9z2aGMZ(+gKVU~>R(3C-bX~rwVXC9UlcpWAJcvUx|aH$PWPJX4jj_pa6 zI#mJFgBQ;B0X{FhD=ptt*aBkym>K1ab4yn$0jXDdxRk#DP~C>MwGs z5uGnFFNX^o`;N7LAD8Gl^(+tZIxZ4CAGds7aMlHun?7m*zA!NC)?>Nu1PY71x-2&` z=)cxgISl4V);B+613g>Jf^to ztGr&otac(<@IO_0?U#coWxleVrM)B3S5g;U7Y@1|rqGD%+{&)rwuUqxg!ptvRtz@X zzYQ+>lV$1r$$GE<#B+My^xC28_`E<~^FF3+rIFLaNge*LA0c*j1D8?@a%TvZuST$$ zPc9;`4LwE`>#RUH7Il*WwCPwGnM7#jmAsa_A{~-t@}+2ttNSq&VyhQ@K>laMxx|A# zMJ{9H1ZX8W=v~qu*eVG{^}mxapoNjY8z(fIc`!vA!X}MxuEs%~xI05Eo!Lb0fFoCA zbZk`B^)fWC$%qwJ&>xfqAL1dFQTDLQDY`>SdWY0 z=N)wT2G{9oKyDp*%r?X=anENV<$4S!Ym)_j1Y=OC4bwN!xI|!ih_)dd;rfX`H4%lV zRS;R$8hy&-yxgWPTmMR$Sy06a!_EzyULOp_Mei5XC0lL# zYBLSwo1gy_++px%lVdvJ$K^iX#@2=|9qAVIrkfL+t)A&2_z%~H$sP7sy7EHe3GZXg zpqg+Y_$y7hh7ZCAXs7Og9*Qs#C5}LyH`c9l*U-HhsCu`YXqVG=ftX!@QflbQsMebU z>hm8W!_Kih7DR$n@ydG{A#%|deAm`_f%(B{9&=rWoUUUcmaDH79giLrID|x}OO51T zSY;3gv8{B8tvpq`kFQEz9}b$$&x}O(jE1414H0cP!$CF^(}amAWs>J^0!bZDb@(I2 z5*mt^HAA80f*uH+Ih;6d>gL_;Q1O*(FScd$cR6rhLgmSn#Whz_uux2rTNq0Yep6G( z`q|7L9U}=(?SRoae1oPKNui(x*9>w5?BeK9W{V6V3`CQH_*^TE>UvFiv?M$$5v%$T z&3D_jo{Lv`{~mpWAA~1?Chb2)N;O22n|pDJ4-X!I?lheyzTH?q^S6EN`ghctlgmRp z8T*H!+buBhn@3!&CHj_h^bUiIjjautg6I-j@+`|;EIVf0LwXhLaD+usdz;PkxcBnC ze62d>E?Pyy#nxze=iXuPo}ifRD=I!yWE)YyQls7%Ty~fH?9nXewx?${IcIXk>lm^y zHtZEa$yazCt%WI1#cRv9Y8#R~77I8rLsUos0aQ{-s#3i_v21M*%gpR9U=L1Ko;uCq#zp!$YZap_a(RRlKZ@=(5EX13sU|t+s50s_VkAWv<7%5 zgkPq+))wAs*>o6le^H}VU!Fhhxm>3=-6~u*w~*d>^xmTe&yn(k_|2zt{>_v_g0d3h z`Nj6(b_B8%f8wgf2Pr32Spqm~s%c6SWo^!k6TxFfns6|df94uve{kBGs`YnDtuf6y z&hb&CBcQv`;^mtGAZ@GK($ir(Ecy zujAtlh0dOs>uVy9DZStkC=5 zuSgQ)+#?@eyw(-621d#8;q}M8b#Wp?c2C97vm*z8`lUZyT!*gW$OVATA>64d@1I^Z zeAhENkE`Xx9!X&2Kc5XVUlSqdrGLlnbsGOrT@?4x1Fyl?Mvd9w9+yWdLG)O@5aJ3H z7KLDv)z3qJ)I+vY$@A;k;WBn=`M4z%FPW(gf8kDr_6#523K{3byxO}rIO^#4_RBn_ z!Axt=`ZG%T!t=Rng=MexETBI2!Wc_2Ssaz$3G$loWYo|4sgA)j*1Mk2P)#{C)^Nia zvX-f+(yKg}Gbd8W;$s_pjstcH#7b|O@_{1^z-G#3O)c1*XXYGXFO|CV&A!sKP@6fX30l4n&KRH-hRP6jYf|a8KP>6bbLQI#Zihg zo^h?vvtGj2vYTTJr{Q(s_k^L*Ei+y`Fp-M|Mq=4|!`>HF7htwa$chPUw^iB=(*-Qu zGHmG2oJAGbJmD>!zXnnnLdT-R!P?o?crMl6xbogl01AkTea+V0f^{ChkGC2?j$L;S z^*Bo#9anxcYt2DmC?zJ~u#M4az>E-UamKx6c=0!F+_fL3(^bUE3H-18Yvy(&!cx+G z^?vnlCGVrp^xPzU*;uxwm)DZ8JO^@hXRYPK@0K<%cl8)h&r46QxM~mi)mrx`icC{N zLPQWnl@CF<15qvQWOS>nrP4{WyWZJp0NT9bH~#$%FTe`m1!4gB$YP~?$GHc1N8Ukz z4}AplgQ7spvRtUydE>yjS%`7iDI{~M=*SX#KUFQyxuYXry`UTS*m898I)U5I#Gya~ zwmQLV(PphAS6^PXs=eKa{xEsP>7{Rb`7J-=;P9_J53xg;BSiYVb#d{o!brGeD3^TM z_!I8FG)uu2%F`4fA8&NC6%MHi)al%(?(-F?rX5x&4A^SVi+GehELo4k5& zm`cv6Hobs^b`pb@=D@VI;ZtNq{|i@=F^854KFcO(TZNS2gam7fs6Gq`H)gtD4 zY`-jLl8cpX@+MRVM~=2e$-V(etrHWi9{RN+4fsnC*3EW<97d~WojZD;*uofw{54L< zQg5T}9c<6Zo{(~!Pf%a_e$kY^cQ>m2f4seAa2#9AHfV~O8DsX?j+tU+W@ct)#u#FV znb~8CIc8>NW@ct)SikS?y}4iQ->t2hnd<63XG*QsQ6EWKG#q|O@50}&>A{VDoZegy8WjU(Uwkus1w!r+0u|5 zGzkQ+b;0QFZBA{t8=ob6u}gG$vBm`S8}LV3X-7{Z6|rV|QuXNturtlgdT$e>%D)di z!l)PFLuf3+WtQGIDEZ?_*EP$5bV1kaPxBMmZV}8sdmUj}hjyu>Nz`sGJ!3sHz>+A=S*rZHcR27Zr={ArbsYz}5+3fjjoyl!SER{mw!!n`JKtgr;S(^r2mOx){xRM; z&NK)O&AfvYuvUlCAH3sfCmQqJLMVX_Xsq#2hQqeM_cuP`0Uuse9yDapE_apXGpb1n2;|=KF$6FpzL^p@%et-DmwcAshyK+L4Y2 zB#4C%S~M}N_=tczNlLn}VjB0hx{W?iT&34AFY&2Nm!&j5m-r>a^ah=c1T;^5-@{ma zgDtQSNidNNY%c80@zfbmNSgY3oo*yBTxYUiC)t%BVUTuspCL2jwWTTZ-b@endhAc}uUD&vs7?j)Gdpo;^hB;b%-;FG@$kO& zB5=GK?Cr0|aM){y?g~T)(*Yx7r5iKGyHD^oU*NWeW92irM`4^26g3{tYZy$S;Y&du zguR^%uBszjpI)t4R3{9_7x9TJ#*3HW&BM(BzVo_?m)O{x+JG2{C3wmo#j4 zG~ia9nwM|wZ|wM19uHi5a%a6kBeZ9g&deFCAeJMYNKt<2n2T$W73DvdVeLQEEAe6^ zzE?lpA=EyCE?L{lw-Ok4D`zo>Ex+P#UtDF+n;w;H2>1wG(EbVc>v97EhipD{oFcZa zA+ws!WJxw2?Mg=es`N21T&b+8mM>rPe=%b zPy9*Y0a~>ka9&sJ>(-D_k;jA$dr;3_hELBxD)x96Xofm5#_uXV&COSxCI!dg5$GmS z6{CMxFT9)J5MW7%XFvMF7qkWz<Z3=${Za_+>K>w zc2Zbu8Nh(JWeh4p4o>+?3oJdiI49PbNwAu%I|u3fu6FWsh?QEs+nVs_2y%4t)pdgL zS|z;>;CQXrW-nEkOZ+vwhxy=N^Du)`uyAfl6Ruo}l0GD3u?4+j@Z4YxF8XrN$Zk?W zK}5GX*0dzmdvBo8LX$(FIcnfzn*ww7mCQ{?uU`v1DH5)>%4cbZyJ-{QI6x5A8qTI5 zL&2$hKTShu=)uce^NsA|hEQ%h8c)Q5MR;ox+2w^fVy5!n1jZ*L>kD_O9)}5YS4oC! z;#5G@WF&j*FJpi;J&jvk6)$+l{@YqVl+0>JxWLw{o0Q>?Y=|eu&2Ikye?RL-kXe$kUp`cBgY=k^}reLalUIiKNmJ~mdeAZ ziyC(+yeztZlJilXjNtSAtwC~T^ec0}FoCLzm;ruITDaY5L|e;2WoAwZl{hyFqdAcD z6gmm)DRUEcukqYMAT}9^#j8*EoKvJIdN-bNvfFZ{FI*UVU0avqC<`*p$zVyK296tP zKf9fJyO6&3rEAQy$1#udrK5sKa@gIg>v|d>`J1-=gQ<<>>tq;xfDl{yS0-Lj!l1p40(!o0KrgN@cSO&DX@FxT;PP@xZe0%@vfwpY6KnPij=Ur3 zHCuxitf&F|HOu0qOiThYCtiO+`95<7zQ0;|8;py?u3{c9@N_r4x4iL`Slo*AEy<34 z(K`V0`O=}4t_EU#rfV(MA_nrf_c|{J6rw5ovnt-%p|BGeR$13nj0S3+Rf(mQ7q+Ml&7f^LJU@iE z2EDBL`dM4vToxKuYUs=()5q&aNxMoqp1GYA%6#(Qwrq~D4zeS8zG5*}5l3)Ipr~X$ zO(}DD7DuOK+3_|fdi(1kYD$})`JA-p23j;kmrgiLrqDf3S=;``WcI@zl~)1re*TzbFp0?GM6qjHidMuQcM4BJysYoW4T*h$Gg_@FqP!d z0EQ(Yn#mmLbaK8op!Y$8qzrF3I(ox?l2pOG8=|+hz5?Dlq3yp?9X{_2!XIO4;^%}* zpH9x?O_%J;68N^3`8qS>h7xTIElI(2GdGw@@eN-!2L$9ITnylrw_*UVvFIJ{%(G&aX>3`@3?R6BiwLd`TbWQX(S& zhoiz^=9_*Rf0r;a)ZSd;fD69Q)}KP`0IUZ;!tH5uP(=BsSiha|(w^P@0g#a4uw?t4 z%2e&lD;S2xE$l1fIEWv3q;8`a-gjbHF;}$M$?CQVC+97=fLJNT-;V!0dEs4j2gB|? zd+c9MIpv^&bdMJ^t}V>+svzUBLGzFUk@0${eiwZilwqr%A0bJJct;Rqy0?dGi`X? z`samofVesj)j!)s5=XGi6#FH{&vz@$D;do?NF>348c7c$g9XTIUr2VYaN9pvZ4wg4 zfXv3D9YYj$J0|#&ALYt+ZM#M$@Vn#~G=FO}zlH_LK9VBRsFnUbm$CgHLl1=`)dYNT z&IAUhXo%4=CO+st5vkaC4Kqkv9X$IL)6#zSu>ZE|iPuC{-~ZlukNI;1fc77xh~u>r zQ5nmP@yXw3j1rrd_t%aS{(qss{>RXzs^;#}be;Yc&QV~a-r2_xph#z&;Q6=z`PdkGz7x)50E|BZft{@==8#!@3oI~!0ey<#r{rHMbHq?oywvmbZ*UvVKR%tcdn zTH4fK4>81ss!`9#Lv#W6_Ja%8H8&Xr@XBq;U2OiIBlv`O3Xpl~r_!8xvDG}+%~{*Y z?es%2n2!qgN6{7REa^FYe0n!8!OrgiOlNs!+q4?}>yKoXZGTQ}#g$kLz(`BsJO>ob zmd)2|Ba%B}rg6}9;%Z#g^KZWMIYr=*K&;o}1` zx*bbOrNo=5>Uq2X9iK($>RsUQzP11l>QD3bI(zC^tM|fMJoF8~WjCwru6ADU+xu$! zsf+Q|#VcKTkv?oe@m89zoO%BZd=64=*TN<9bOg%C(Nb`^V4^u~M`ZB)+_r&p&tnt; zfDl+~-J#N#j~k4Jl<%dDlanVJM;EY~z`zA_h)2$JZEPy6S> zpy9Bw0?$_PmR1lqXO`=t=T5$*G8v)Hy|ZDyzoMj>`Nf^=j~R@QN;RD)jYyO~(!zbO zpDu9L_*d!3q7-U0W^`F;^~^}+W;iX8!Tz|MykYk`I9JG?$vwDmg!BqF|`;TRjPs~fnDGh5qGb4FK0 zntZ^S_E>m&Uat&Sx?1b_$;ig$SU)|_I(%@?Mv}|S>Zu(bO1k>-*t@&xVopN<;BcFjj>x5;0aGbqX zVIL!K9-YbeeTr3-WQM9?^2KG*WT4^3GnTr#8RHLf!KG2H$H`5~F!|0S_2Pe-u=`Qt zwsEDAd9d}tcaUMmH=Wlur4NfNb!rd+8ra3QNq{$z@35UOn^wCqE7NYX%px;=pIjj0 z`^RV??4eh`635hNbX^El$?M`y+;gAf;3WhC8osf2P)OUZ<28hwM{M3F(<%DotC;uQ zD*uoPT1Q7}U}>}w4omECL)L*DXG%dZS9jp=W72yTIwI2XUt%J3#_LQ0#?q~rSm!Cu6uh@QS%Y!u za?r#<$b$d(&()mE4q5UT9VZ@AWw|A`liw>#G@{$Bw~zp{)Kd0aluSBx6DtpjoFUkA z6esR+0J4S9MOohMC7tU_f3>kKlfM3;f%FWwTbOMy%rX%wQ-~plW6W6HRBt0xJTuc5 z=-gg-A}ufAb}O#;5fwY@WT1<(%5#{sEXA-%vdhT-L&X1(6Iv03N0D6Y)i!IhA=wiH zGn$nVI=2Sub8rt z8?(72ytv3F2OMpUg;5%f0R&28NCW=Cr#Cq9MLxusfrOwlc!nvBqk&AR>DYyr+BUbX zG&a|U{wCe*JsqiZ`EvM;@?Xvn2R;6JTveq@*E<9j&ump3ToQ8`*`@Vgj=Tyl=TAj& zRi+7H@QTW&cLmrzX*?Z&d$m>pB&9Hpwz^S0(2RGNK;d|vjOFuvfPP%Q3AVrA3k8uH zhPs#+Nj{*Onu&tpkc!4+V99w7>oa-!hU`b^KgRRzULEKe?^U!{nAUSM*kn23Z0;BD)Y|qD-c!Q#{T*Q{Aj+8CL1+4;u~+e+}N=_SoxnXurVQWacE;t z5Ag83?B+WZu+FzLO!08;w-wkz|5o|SRLX4r;XPKv@dDjqB{7^NPm&pa5XN{bykB!LgG@Q-(O16!tjt>7E<5@z$&KfH8K8bpiOL6INd3 z2LA;)`hUr$51gR=#ub4+{K!^HnH{4#%lDEEkA{#ahCOa5is~_ zJ>gGis}2%H0aBz>7=U}|re7)ziNRWg<^RfXsITlC`nibI!^@f>;B9zCAQFaG~rAnvV;+pYd*nf$MWT2Ty{ z{D%WU$P3bO{}o|q;__6o_VvPJ zW&fs|355bIyK;&Jc+U?#Sxd2;pOy}#?Iq!jOL+n1bGev(32R?c>5F8QFi z>x9q-#%x{Lieakk7ME}cFxPG@#2_-KY}Gp>K8kzqY?^4Fg8#1aiD~n@GV3^o@98|b zGi+Gdqc|QQAnMqVQ&YoAD=%!&{!iDryOTIx` z0JStV^9kpp?*{NC7_eVQ7am77DyftT2G_{jVOc@a>wSY(z<760Aaf4fep9!xp&c3;Qba~Qs7Uy1nC87tfC2Pa9%X=Us_9Mc2rNILgx&3@)^?W4vseBr^>|TC#D$XM^n#a(u*x7RI@|L#5#?c!wT&4E2oixXBvb4!7LKwV(=e zD)=y!ipQlEsG?r;H8IQE_p*hc5G8a-(V=;)e^)ZJ!%n3*N+{k>u+dO(lZ4?C$-n?r zb40;hxGY(XJBUxE8c4Uu6V)Q@1KhAJ9_+ijsE*;@@+|+_&G|p z<%Ondg}RW+xn7m?ySivEc&&m~?az%_o6 zb@*OK8|wQbd+$K*{@tvKmvFYqPe0r#mx>S;3Yqc5!Fz}q69p@*2FtpeCL;aL+};Y6 z#2X}^hdWCcBO4L_(0io!{ux_!dvMvRH& ze)}S%MU&F~DY;dVBgo6|-KO9HpNFiRudOX(=R|L-&nH&8`HlE%dOAePp(&5n`<_`D z_1z9LhH_so3+U>d5Y?c&D6MpMq*fb;g;<*dxpLp}>Bo z11qW2?}IMy9#{)nF;Ugvt#^DQlws+EIuqQUzNbB5YL9bLUhEo#ot6q+EDF|av#wkm z(cInLPx08Ub=FZAM0T(DHi#UAn_|&Xq~!CEDYZ#o4G)IhG`vc_@=d{~I)7`}aDi&N z*1h1Jl|5=7YrW|fQ^#DJ*d}Mjfki&<^}Ez)e>sIg*OD8vPNPHwB$;B(yuXL6GBPDT zt|!({37-|$y*^_R?zmtF$;&e^A_UI9_>O&KVuj2UOlmep$`INbzr<>SS%PuEogYMT z#Pd#JVJ+0u==8KFeslLT;pp+ZCcxZT>ViOL5rd%PiCyA* zL4iZHSxgK+ITF%ZxSkFdgK0!}7V)nnWaSNkd3a~tNIhrwv&+~y8|l5CdB24iiQf_!%nUy~?szG1 z`&_U%D&|wSoP(>?MDA4Z4XlW9=Db5BmV-1qoaZ}mQS}r$e8`2=$g3$}d{Kd$Uh)iZ z^Q%kSJOf_uYwHmg!p&b6`i&X&TDhTuLD$WAzN+}X3Gzy~@r~<6$_ZCIrZnjJBp#5U zJxyiygoeim_uYMMd|n=2dbqx!;j-DhN}Qg=JieQrhORzXoA_QxLt;)z`RiBTyueb` zpE7kqB7Cd|s)Y)JiE$rm_ZNbQ3%}&c-xqDrZkx0keQ+a)ihM+vK#2T`EYmbtq{_3i zhegHJ;JGAzse>l0G0GxI{P>P5KkQRxFT)B$CzdL^r4eszN25lPa!;fjpvhPZVU?L5 zOphJwQ8|UTJKl(etX8Lvo2-E}SWq`bfn^U44uJTGG}Em9YU_CHl%6W5+=^XC2P?}= zN|;P2`8^1@k&^)=Q#*wuw|kG{!!Sh&JEKOc6X0xnDswlW^~jlmhrWMSopD|Fviafx zj(Ic43?5d#*4LjnvWRX#YWbd03TW&F{{o}X4 z!KKWCAuoVb=dk_L&*bM7Q5B_HW~UVut?9%_hndq^Cj8Y;P`LY0?XMq&@UI8e;lIB8 zu>#n=FL<#VN?Xf@4Y^d*foCdPe~?a8(1=Ew?_2voc~-d5itMfTX5#FmU??|ZjV-Eh zQp;5y^vSSbTl9KMuIN?0fOx644$dF??fc6={A`;OvG%*SiRv5QY3axi`1L!A`J3Ow z$j2MC0^bWH8aJB+zP8;vq$HB}ATa4HcimK^!j^pUOE z-H0&Fg7mxu6h^+%8==L`yLTwe&ER&GzpFa%)Z;;f?T}ghHdzU$?6Y7(pX9oSP|wXR z7^IZi%d-_%&-vYnyWa;DHK#MAR}1?_geLx{8?QIo50{<+;H609LWwG*9$F+Zb2;47T#=8f!zQQ9?1vCC7 zS(-wMBD}m~xvll4pA)g3#i^h&9JGt!K<@-QR(N~V?&@MGC+iD0Y=OaK{XxU)^%x}+ zDMqc75$fgkpEAMJ_8%3rX~X(F5w>ZRn0V*xQ2K_Dveo-b3ybTv)0#1ItJ<0L*F1sK zqMvq-j29_EeKGeX0(+Z+fm>&5kz`d1>L51m1`lCj%Aq8eSQb~H-m?qpo{LFhbHFful$8{`6gj7`jZ$#lz+g!^# ziDR=UB7jK2as@pa8okU`EcffEIkjF%ULQ!!Ld@(|7=WA!aRx#}Ws*yalDj?QUO?)x zONQ}>#LfWy&Ajh-r;eHw16whvXM5khhN0KfzMwD#w6zNc!Sm%jrh6QJCiy(w@m)=cXfdOcpibKlxOE7qtEhn0 zJ!Skc2UuNm+Y%f==;4uT=^g|g6R*DfSvhQrYVug1a81+-{pyJqtJ?TT(94-g@cUG^ z9%bXzBOj<}Y~l$#$|Ws|V|b%`HQ754k z2fYu_A+vR|MZ~s&*k5Zx(w1Zj!2!9Hl+DrOH7_1{Cg@Th7Vsz~0g5$Y-;**awxh&S zm>JRrorcKl7=Mf1J7jy}OZ6|0t1Xw12iJB@9h}&!AlN&4P{{~&4C%p35iQX%Hmy6a zWv?9=tm}QG^!j{APDO=liL;V~8M~oM@c1zF>B&r`8d_FURND_R&;-FiMAck()T3p* z5q!-$Jj}8NDcb0He^YU8c1n%^w56z#v;UMP2${sfx9(ty7B>1rbHf$htb3iX&t5_p zk#v$E+})koyfi~=p{Cq#D#6=n4NNTPFg+Pbk&wcTo(o`?%gdd|>Q1-i79An{TVz+@#9Ko^NkWx@2tllz zZ#;^GL}L-a?^Ccca0X|8&J0DeW|Im_&&!Jdzu}h+?sa)lFnYLysglC- z%#Q#1^OS+j5mjcu1~ck{X>Hrd`6SSOFnZ8)pV@}F0a}8lCJiZ+cX$+d6!ZqrxPOuD ze_iu|uhXhT-XBf-zNDNwp8H};(qpRcY-x;0(0%BiO6HRb#+cKVTHZ_kx<@K7&zh`c zbp6v7H1X3xvAx_F%;*voSX3lwxu!ZwcZ0^4Tf~z|IwMTKJe+2OvU+zi)!Rb*!`~7H zrCOb#k%7n7-HK|xDIRUvI;*v(`-Q$X)6V!$pxw?9p+#?g@8)4~a~UF@rMv*^??SIP zsv=2lw9hIfXrI(E&W_zpkhaj1>1B;JU(F$^-q@`xr77o}D7y}{_J@#h`h}IRo zi9CrfVr)v2md;4(e{2m!$9LLMWDT{~2(p$E}7hs1yui-Q=cT+N!n|}GoxYSZPz^5_AQ;{0Q=2&fr zr?;ovrl`NV$es@WMsr*HmOI|Zfn0Xb3FURUe$7%deYSR1H73ZOZK!bQGAjVs1Kd~F z@CN%wdq?gDL>ISccOYh)0<`LDnU9LAjLAl3Rv z&F<{F)_{Ube%nYTda}a4JG9u|A5}2|wEz#2gcgXSF5j3if*TOuz z`4ZaY%&ZP%JE%Is&RP41FuYCi6>6CXll3P;^_jR6+b=cZk*5|dJ&>3P=--gphGYoQ6TPYZaB%f6{x=fukC>xRDoMkrkT8uT}*XR8wFlk@z>Q z=ar16r*M%47d^wSvR(S+x~PuRcQf=5AJ=<#vV9n4rDg<`nz7cH!k|u;VgG{lNB)JH ze>y44_9C}canX(YS+kE3Erpu|F+4Zhy`SL&5x_WA=&;1nD2?>t)V8q4hAgO!W<{o; zpvvDh8Cj}oV33z;RJx$Ze8OAsyVx>1yWF0T5Ni?J^ z+rlY3s2dwto!jMgO6s}~U^Y4>s?Kd`3DcB@B;8+|$eanWx2*`XWpUbldti5&*%>87 z4Vfu--4KFBhDE(*jrA)Et(Q$a+{M1_e`~#&ez9Iti=-1cLPL~8m&ITq^-k@wqA!qW zye8#cG&b7WMj>nVI#xSC?%!~wZ#XbT-iVJrwx-_dOai-2zc5?xgznB+pRfeK)8S+l zbCoX}i22M^5zvHb(esQS^B}L45`PZ*Mp56wXN--lZ4#@vDD{$RQ^ptHwH6+%n=%`g z)p-`ZHW5(^oxwxeWiP@jROrq!K}UMnD~(fM}a{dqQQ7W*5-Xq(oI-t`Ht3NE0=%qj^iK> zb~NfFhudAs?AGeDnj3Q44$|ucT*c+Il;8Gm6&LBZmkMT{Vd{7hpQ(jhD4JW8AdAJ?2>>$0$y!#OEh^_=v&T?;!2f9Ymb zgxP*o@Vwo~`N3mIAhUell9!sUuu_tomv0dH}`8l&)1GoHy?$%Ip=yw%tA^>0SE-%XhZeQd8)No|%t+GH9bg$OygW5D{O>Gs@U2yv0BnxoO=sD{c4!|=ZC*Vn)g&570w?x;&*AFq7RNh zccPa@18MNm2S?nYu9AkU+^dKd*K3^_d-VZ}e+Ky|4o!R7i>6BkLYn;eHu=HNiPK@udV@aXOg%K5u-tgJE!eK6VxZY~PbkUwJp-5hvuKEl}-oXk%^m&N#o zCuh~i&s^1LWJwG&GWdKnkfyUb5>*Hhc5*hNC={qJvO<}>(jaLAS6ABm=nJ*Nq$G$L zmw(X%GG{R=syvmbdjK0_@?jpp`tf0viy+x?Yi*&al#nOYG`N)}LRFnl!{p(o60LVR)6)h6*G zbpnN64!YTM&k+>1cHPd!LSNA+v7Iu>i6C3IvUsrwqvbMhRi8_0dbUVF&#j`KWc;*0G)o)L3sR00ri!W>Q7c*t<10*Qt{&@DAk4l4xpGEmB(>tdSTEQr4 zFG#QNYmB4#@xsX#*%tHRzQ6*?MU}v89Eu?L()OZXEASBJ?5$x0^BS-2+Or~Dd;|(G z2Us(yRZ9_V4}|Ul4>2im(uNSV2r_`{hlwgnA2y||EWz32Q@La|LFlRP77^LN<4inu z9(&9Nx5iQryKX&b-1JA~`O-{UMknS;g z@9p~jqPXD?PE5Ag@dLVheR`Q@i!Dbq<3kM$KGNK22o6w%zGXo9H0GxM8W~cMxbN4Ik_%&5AkvDhA%yLFH3H!CZoHc!eTlW61hg zQ8BM)At|ZGU#yFM1(?hCSzWYgf-8S zW)d^WOKO-_2UyK^uxC26WEqO-qBhltZnsL-=N0RJe>vVB%wi6#n$i~#nNem5zeD5G z>M#Qr%a}=FWdJKI(|kuY&(7VY0i5+eFMdQJhI94jjap4bhlgGLsCJ#ek{X&+JkHG*rS4{80x4|PJj|34}eRaj|Cj#MQ z*bpuDHXj5ifN!Q31;F47O-VqmX4O(N(2Uj>c?XlHi_|Rwuy&gxmLHSFm8gJ1kp#^_ z^#Mn~HPb&8`2sDQIaWFNt<87=!V6;R(`8EaJ;b1Riv3JBH!j3!W_HUx3O}c}Qs11d zduvm~W4sddJ^PnVmAK(k_=&~N&4QD=5#v?E>ZtxnqMvM-v|(rHl{7wUP(&j%{ylUY zlbF*C1dh2pPwz#`p#H@=47{@nP?Fk)K zsl#t`WF{ue*VN%DxDS4V^=p03&sLP&E&;D?VpNpnc~z$@UkXPki_i0`F3Ad|w7Lwt zjv|P7$VkV3NIgnu5*^b{W@!+==C21H zN};uu*}Wdw2X2rYn{&hVt9N!Yiwc}S3}pRk;{tVb+^yv7Pd9xLztR`#RP5Y#5U@664`Dd}yjAYF43FLZ1YKjG}!*1?|=W3n)Y7*4EKSXcZN z{bwiAkPy`F%hSMp&4)f{oj;&Eh<}>H`+SrZB z_Ni52%^~t0RS!1TDK(zTogt#lMqj-k;h73~2KAuP#+0{YT6z(;SA^2fM0c-k%jwaa zwLW%$G>K3RNz>lMlSaFCM*4^aWpO8?F-WQ!m*8CT#By@ILM|Q)27X>P5m2a5s@R5Z zYxl~##8OUykS34a06NeS)w6zkC5(V6jhHGDpZ-t~Y2;=##D8D=#9O0Z;~LJYQVv(x z`8BcN7Z?N{;?4Yo+g43TF^6|b-au5m+Pa|E8w}#t!Myq&F`}}(#+kuXA1JBw&ss28 zquWW^-)#~xbD7b|F@3nR6q|c|QpNEm6DOi}sC&#zwkj>vT0_4c<`*{y4EKkNi3Zns zp{kSn=ZhodJVV~vKfXS+2%gP8<%M1j#>V-4aO>sR6om7iIJnM|F`IfmtHf2o65El^ zsnoH!1bHIWGyG9i?sFWfn~2_SpXAet>igWWGkuXAHb3{PU>3X zh$}MfRTL)Y03DlJs59ZdM(75Ql=n$%F!Gi~mwAZKBt8Xjn`gRv!pMP4>po^@%pEPx z!Iw)RK~ZQuDHPqciQ=)9stFuv*J>Tp_6oPmH6Hvf250FR;qRVoL%6rUobb$KRnhR> zp{_6w`qpMDf#u_LV44uIm_>iNf!z-`OS<9nrW@JVrD;$k`=;S!e z>g;AL-~_Pm&D6gT(Tyb1dScYWB!In33<}C$c*=|pISrHm@akTy-37Lj_O(BvM!9f~ zH@6{jj(S%12P(bnpUXbMq`OJ6plSEzMr{Vj_X`SymxAl2^;(!rrjHDc{l=BP3!svw z185jH;nON2E*3bZ{Y2Yw`Sa(c?4R8w4Zvr9R8A2btlQWRo)pqm;JPhqSJSoqSQn6z zLY7jdiS^~nDsvcxW>re>$Q(AJ@So?TPRBL>wm47@ffpVOPlSp_sw}Qs{L|kpKR`us z1Spn)KpOOwjS&Wo$=P{k`sO{(wmr@Cc$g;NWI)H68zTif7#LL`8!_mf%hfENQ4@5u zE{J%>5m}t(J~A%9a9HSJ$DJA|8h0o`Qe73(Oid_Si-wUo-j8~Who9fOqLAOS2KZAG z7Lbb<#2_z&#tdIz`J+VFJmlN>wEFKjm*l{~a7-+`pgN24R_Z1g@rAZa9yaBGMog*eck$67k&{K;0j?9y3vXH*kpG{DZtE7{vx*Mpv&tXGhmC#3bIK zy!AK-)ZWN_e0!F3T8-2g0Q< z;mfMDv;v>P)ja-dj0-tyY+{X_i*S*4Bl8o@o*h?rM|J?a4Y2zA?goONeH|#1q;b z1bN|x+l>%g%;$N#d0y-dn0f3R0TsDFsM?*RARWe;w&Reh?%PKYBHkfMY+d!x+QGcs( zWOzj=Dc*xZq9^mG1wwvs?+h6-nxzsC%BdNBIq>GkukotW&NKA0pa{Org04okV{9ay zq^ZycWL{}y$**%P2oTJmJg#*Po|%wI*(H^-&x|q4dYHkT89kRStZhgwF~lfLxp@Oc zNp(nKsCxR7`Fg^FYqWV`0=eVS*A_RSXZFjIbV{iaC2WaG-rlGH+xv%|BI^V3*N$&_ zUX|u)MR(-Nvy0fklAK}Y)k2tG@|BU(0g9Z^@_OoWQc5TuUj{5x*Lsm~q^k#_5X^0_ z6;=1hR?gdZQ`j@*i$ZPumP0SbFC0FK(i=~5sDiw7AM=-3hZj1!R|p|-ZZ>&Ff!I=F zI=sw~1jY|uCrfrf;a|T~<7BLgaUJoHBBD-tuf?_11nl7IFfdqLYNypzWweB#`a*6l z7rPR<Ua&*nPVGW#Y`MgMrrn7N z=PE4Uv&6&V%~Eh{Nm}w!(Zv|SYSdzt!;M8D1>$7fv1A@;Sfk`v%Yt%ZN7$G2F3Wg& z1}*f?yB(u}1?@BDzWr@^Y+go;V<{iyY{)RQCc)?=x_T{ZB==`SY*qp+g?T~9RgIJQ zjdt_h0NEr;@4~EqaR>+iv-C~S#Ml9ifH`Hg*w=6kH_c*No3Jm4oWq?@cl_FZLaKNa znLr1J#@7*MstRS0%3y){sK|)ze7mOsX$Bk8Yu3D9G_l&t2h_#}yzvC;ftj}dCmvd9 zPfz7lY`z!ok^)S<%eIv&bP6YBeHOFZL>L952?Qx=+@bBms_+4C=Xb)IO4}7gRg6bQ zYlZwV_t3tFD3WpVu{js4^XyV=5^lvo(nB9*=F{SHt|V01PxW zZHgvKu&LzJeinpyekU$oyKe#@B0Z)TYvfU6wwvPBLSe#OD)Rg_mROu)s#pqB;6rVF zLKy?WD$gwoz>eUwR@T;QUGn=!=!~73Fq^e;tqMwEGTGfK1-Ez!Hy#3JXWaN{k}91z zb|yhi{{qujO+Zc-O6i*K)lCMI0i>ok3yN!J+)(e*Lmo#%mC1tU`a1Y{7s4?x2csSEi- zXMHDxg97bLpONGpt18B$5I*Cal_(kJE`p&SFg+4cu72CPj8{|3N2HTq`&r7&*-k{_ zZ&6@U;c@ZESm(c(3zH$3y(=Lhds?Ac0XJmiYfw^|vdH}$u$Z@LuN8yZ#W~rOogUB3 zmbSymr&GK0wc~R!DjXDDVq9M~u9REhR04m{W{nOp=5yjFXC#*9rR%VGfwOZpHx!w> zu84#U6_q(eM9PxURUfb#NaPLbyt3!G5LUmnEld(nMYLXk?vD~uxwsNqaQZ}?5taic zLWzeH$MmfTpf=q7-@K^OQ@D%Ii$Yo-zO*xZUKC{ zvKF|DiAvHy3cB-bkX~lV50e?(uO%dSh zjU;n>N2832v|g)=SO~;b0H<)=nAS=K?tt+~d#!7(uW5$!S(B4^QG;u`R_(R1Y3AxR<0w=ap5|D9CDo zJv%!alaxg933q(*=55ub4-1=KJDkwe)SP5Nh!zV*deitsK2}v#Em~L4cXTkx#2+79 zDk>@cmYa$G0tWgyPFz+tsr&HYb844ZL%zevt*_MT}KAp~!ktKr1{;>5iXjd9`|yN+rX667@% zMU2dT#K5G@2?)P}OxHlhVm@`WNG`4jd&&2pqC_58#wQZ&?P4X&U`LqD1`+o*S!Zm_ z;-2R>m4qtw-8lz!;-bhMcUo5~@!F(o1L}T@Wbo{eTP`98o04DsURE~?=%~t2eE~rw zRZ(8*`wPXMq8}VKaS%Ew;~Xzfw5xV}ql-$Mmse8~*vdjEd)*mnZK(OtJo$0XNlQMk z=vEm-y@BD|$#PUit^e@+;C{12$oO(ZJL~-DMazUZsq7FcbJ5z|iju6WJon<{4a0%~VBl1KJP{>_+N`f5J$lv0f!^*@OGx8Sl*%xNW}tc`jX z)WD4T@^*8p>FbNK_K5$RGxjmO!7f9%IYRvTX##{JpolPs6 zt-yr`wa93Hvhu{+vp8!XP~4yo!y-?yWi}bot?9@-}%1$8LPny;JuRh zHKwrHlw&$LM6Y`C$wy6fULMt=!o}-m3MN>`1^QaH_KoeUoJP{dI=_QvxlAVky?kw4 z!^xP>F6N;SPXvF|6%#jJ2+&>Qiokbma&o-!_UFFBY7=QTL_s{0KY%ZpE`#lKb)nkJ zM-0IeF3=0ovT>q0&4$(H6~Z4kMd#T*@ZgJ!&i?4)Nj-BXGp&t<`wU!KBE#u0`|XBS zZ*@ygo{%8}>83@Jc4@4s`SBB4mj zD3baTP8be`>UuQ64&g-6fVEvwkX7$)odkwp(n0i%8%mK*i{@r{tMa|&kg9Trl#-)W zwHUm!fJV6n4tV;0&n>xG$F&-4g;F1#fOkkj>^^19W9Z5x^xCQPU8pn6f=xVGb|>39 zi)Y%hE-dL(=T#*_*2G}e<^N0OIHlGQiMnUP8LSq|voiQ=vbiVHp7>0EmW3=)TO^w@ z5MM(Ta&Ezdmx)?Waoyc`15_V5PbxoV2Ax$UYjIq18}JR8Yb;Zw>pRGJbRKP899B@t=F1>Xu*yt6auA{~aQJjBMB*(lG^Lf>f3n0mt2o#QF z`I_0^#@FX`uP2&cQSaq8AbKFOal@ve(N;sF`b~YA7YnB~mX6MC_D~7c022q?Rx>%< z7KJH@g@+p5KR3MC88kZAqOyeh~bv=pg{ztsC=mk0H)Vt%)Zu3bZ|1o$} zH0`>K;roi&8ziZsgFtILg#sQdalg4O%AiMN28YtZe~)c?$|mzsidmgwAUd3NZ;7mtNEBF?e`A{=vCYB&pa!`NzCDlU&9=D(^SJs> zN1fr%Ze<4A>mgiH1><*r|4-jwSH0+8y|^u*?;#gu{_5n%!4~N!TuJX4GtHd1$Z4}q zoqf&w1mr%2fR%F{f3sjk72or!J8s84B8ue<--^{UHdPwlTOgJNafCthOqmX1b@I9q z_Esm(1kD!W!n}5eBK4^9^MD}i5I_rejx{g1Iy|h-2}o5V1&KAVaM>)(&vBX@IeiI| zN-h9h2(5MXB6xo@cZMI3nNG8f8F5I$TyUD?0!I6Ze&fGryccT=flmR7Zv*bV57XDQ z2yXJz5E&jPQCNTcwcXaxcPH+3(dPvvNk_JHj z&UV?gPP57-uVjhl%s$zC+D%Wi4$(3!(*aMpOI$#o6#1ln3l>O|pgq6&2vd;M29v!w z{)>09+vU)@%;m|@JgxG(yY0?`1w&N;YBJ;OiYC9;(nN#r(AbiaF%uL0f%UuU&BxxW zgR}A;9G)z=nsZlDJHdi^z#>2EEGpi6;CEpCaS`5X2A*xqL-3;*pOVHNm>}_N%-V=! znduUt`JUK%_-e4V3_D}EZA|pHc_Zx9H(tZ<`S|UCR(Nqp?VqBT?;_rZsUSzDUh1|q zFpBu5h1@{-h|Xh-rf05fkY7Y(W*qSccZWI>q(2^LY-X9D5ZY&gj*eCNFp^WNpF{+p zAr42X_Z6-27^CIvfkOzhesjiV(*fY9Iv!yj?G9s@@&^|hJ(fyw!H5X%?!TAEyBYzG z^tU9tWAv14|IogE<7RyU&PFM>fHt>)nl`j9B;Z#5i<^!CR*0ggYr@PQ4Yc@=_Frp@ zK`uQp>7EXI@{mg)>rJ~hmF?4`MB_Yx8Xfu`A(B@j8UuAzeWZ<_^1RB78;q2Yo*BH?_i|Nzy zE@QWpN;&9fXZsuXKrPN;Z9wrh6%|!`&vg6RV5Vda`c^=jB`+Tz(gWAe6nb`7`m1-# zoR+qB+Zjxl#s07v?4h@wyglb(MVPA*HfzB5cE)k_8sbJzmM2gdRpv@$|*YVGzSnW0ToCzm5*ZrRG`+Wv#thhug zk`c;pPNgKvJF=~5i!gg}rV4F63^Q*ElnNHOthZ1JdWeso+CCX$Ot)wen5ZBrcU!`r z`7Z3c9TDad`C`j^Gl;ob&yCkl%Zzc$^0^T%IjP?B)zajlb5C!{lREUB^; zFkx;wcwE~dM_RO)BK8ut{kNOyCUjTvpL-0fe7!4Ae@turXpue-w_3-X`K(^Lm?8HU z`<~9YfuR>8rSuSfn#7h`m#vE_zv^skAk@CQ)ZQ0BL*D(skXTOXguT5z!?H2Y`=2)s z{ny1Eor^~M$cjKno6fK#P3XI|^Kobc5`xWko^9MKk9vd`hpdbFmL~f&WLk;}MCxk> zkNx=P@P|-A{)Dy@!LD$9X7eCHza(X;M$;4slzzT2xH$QDsbkS|*~BJ{Vrji}ll)>T_{vR>-*-72^v>v#|#e4w0vSt8Vp`B=U*PZ0Bd@^G_HEWjG!QI*R{OLZ33? zSPztxPc&-C)UQA=x`GmA;nhN56e%(A7gA%|QyL(jYE+b(pVa@7XYRyDb4>N;VFy!1 z(||S>XIK>Ae=}ypT96F*Ua-Ijcz<KZ@A>A*QseQPEx;EA;ipkJGA{?nidi`JsN|}LQygKg%VOpm9B9w)g=R$jb2U%r%5X8^xP4koWLEVRQ>Q?xrt;pv(Gh(2kjDkxKF{*99nOho|VP4 zEw$qBR?8gp4wp=lQ5kM@vDKJKr5{(j z!c1}Xka<>RS}#~{iqGniT5qJMjJ1cNL;5HUssn9v%8FE<$iOLRJ6+#58nbRgL^Z5hf{xB}OuTdLjDaR^Dl1uBv-dcXbb4@KN2;nykq|6IPX10d zrBQl9#fByvrw$Y(lw!!lz0}B)^Z;)Jo&)qtIdB~5!*C_eVz&Lh!ubOvYEK2$`{SlX z;Gf4kQr0fC$vUnX4{zcKMPGC+cl5f;s^10;Q+zrfOi)@(dFBaw2k#}G?>GdH&?wbdBQR6tvC_f&{?zXS)ots*`WR{DSHvr|Sv_$42+( z2k%uN?g1-Ac9U0nkGze7_6nOxYJ^e(Rz%|VUdK(!(cb>EgBzTb%V!g}&*s#g)^1kR zvhqr*+QC>vaB$Rc^3swTUP~vdo?iGjOMMq-Cl>r|jK5y_SkZi&z)_Wu5Jh1!?OsdS zE)CI%Bj^n;)=^cv_QK*TTTsQ9#y?Czlz%1nCYVR=EOYd9>`AujN6-PhVbYVUp;4+` z!-$CR_{vpgL%4MY11&A>)(s0&aAaiUJAG+sY5XNv37S0)4vv~t%zp~=SpQ#zZ|dH| zYUY7Gxl4Ad7i|xLg|`a<6MHN|BP)HhK}WP>Svh0=Pa|e`B($_s=J({w)MXZXCyEv+ zJdov%4wQz(1+K5m5%f;-UqSqpCF}LcV$fAt=a0x9Ie1~v%G{0OJ#pjROBWAb?_CGd zGOc%8p#p(Uwhkcj^UVFsnX9GYbys~WKxlg|x2-UEf8J$qRNlbFgQySj3F1DBjkY}* z1Fm{c3cb`mSK;q@)dvX-9xy`>3=R8G(>k^s6GzOef|0*U7G8Lz^)r*#f)$ z!5034mFD__SHI_^fw_Y~VCD9u82l6Tzy!2KF2>jGb(0bIwE=oh8`#3|B?&jVTZFW@ z8_J)5xKY6>a1moK0xZe7db$TcM;vjj4R8q_wxV?X@f}I)JbQXQ!izT}Yoou|Zxgjn z`Zo8a_OUKT#F*n8=4>p+)<%j#ea=DgXPQ+JG_K4|PB#1%*c`OZ~g4I=0u3y^NJ!+K|I z#}hZ_4^MrA3ymgJrCr@kj6w+$4)7xejqxX9_VS2|Hfg|BNyWQcWK=x#w#SD2`Mkcz zxmOj3c4l^ZoQwXrax2jLcB*LkEA;IYY2l4W@0}yI{Z7xjO_J=MJu!IVU4j*;2VrU@ zwh1U`J)PI{C3b6Lg#9X>uJdO^9L-(DhjG3NebTP*#LkES!_o%3rb-Ybkmr2zS!qR; za!fF>@2q{FSKHlmnojqGY(2(P=&55b+E`JA0K2t4w7`D(qWArzn^o~%?}L7`X)t>! zdvt!mXo#+QU)J-jpP!O1?TY5g)XGYAA+ik?gfddy(YxJ zawutrUXhQitA^Ek@-RXQ(b$*_k=h|z&+x!1Rm$cD_G2V+$w*x(?}Oa8EP{d|Ud;Dl zVb0b+V@1PwXDBvjPDdAd+jO-m#48B~=b#s&oir>Nt~hS9S%F47cua))s|Fegb&=+ z{gBm76P&M(^_hpZ%uUeis!(Jg;F|*Fpy?n)s@X#Uj(;_2a3W6UP8lITq5)LHH%6%q$}#}8gSPG=u067dUv2w00~LgL@!GilKyoU8V%baI2PZ{h zOC7Kr2G@g?Hl`caDpdu|NGn@97m=!ZkNA3VmlM_2)%o^Fd#6ZyQ>3Hp%k9SYZ-%I> zO*{(%2LrA92^HqIWxKWZy?%A|ktOW-=a2X1dyL?*uLL`s&XOdkctXi*^%r)xyt##( zHP0xwVyff9RC`q-3~}^8@>wYMnYZ+MSc%y@Ev*t;T_DNI`i=(Vumg!`osHGp)#{BW zgHDIq(6GMfwMaesxMwx=nMd~JU0cR5v$%jD_uHT=!2QKz;fpm&(k^d!qzx&%Z`jpv z1ey}FCgQw>3g}2|OM@{0=E&70iL*fIsmoXSyIc_w=NoX4$s%6pL2}2E3LmG1(%S<0 zSTYP?YsRuJ=$BpKsV3o^>ITsW-Bb7Mubk_5MEBqTO^?duPJiF$rtR?i+&BQV@{9y` zPr=F%GaZwG5iGE@rY)6sD7EbLvAzXI}4nc}msF3URZQd*;)r++rcL$P!sRn%?mqbojDvL&T1)Z+f*fIj8XD+w8K3=X$XcBO@ zvsxM#5oVc8qv%Wjl8>*03M9AA>L0eXTp06-Q*GgOrFrPuvME=YY;APkXll5h-Cq?L z0~ZPfN*obUp>Owqninf1M48P`sG6^6S-p691(j$enrkiSn7wwI*7w(sNty zT+|tzl!}{!*d@Sm?bPr7&cH*B#|-dvarjg5(Xw7>gBv5H>FoJ-hHa%gD&9if@o^$g zfrTpGKJU2cy>uKmOwtq%QNuJ1*e#>#Kl0`OK-2t>&Mj{l;9hv$AYnyaX9FJ`&_C`rFYxKksJIw@dgktpQf@BA&v#_L zv%@5X4@7PS-Ke?c;Y&V|F+p(Rtgol8y+u^@UcM~gXeDw}Jcn4XH6XVuN8HOrqByLa zzp_5Ep-3=ZYH|k02X9!~?$LqR=Ou@&h$l4BBY4DjdCwt@u{6XaM{U!(23X8on<0s> zwkpq*GIBLDH%o6gEE49Gx2o5_u4$ubx}(`yw|BiT(NWP!KU|$|7D14?{`g7ZiuVob z&~GySPnq^sMBjK%afz(QOzD19%>at3tMK)TwcodWFwy140=4*@*IP>_?2WDy{U%=% zjXl!DId7D5H{{0pAIbRNN&81f{%6v#O5eCsHX3+fSsDu@MiD7*WT88396H{;_|(#H zvAYEd9CS<5s6HHB7H~aBDP(npK6{Pwi8Y^@Dj!~{h?UIww-h{ZlDXXPQ*u2h7NZ&Iq5Btg#6>%VE(n8QU0~<6yKzQvlXO|FO|f==}F^dqkXk=&Ay;U*hg_kvf=_IUddJ5pX6`cM&&IoshYH% za&w>{4b{yYa^)BAs(B>d>Nb-{T^>pmQ};SP#sB?r-x)fsP;zEY1d{rnmoLcV@Qrt%&822_lOE!tZ!`}tl6Kp9Qe3AI&`Z)5bxg6uc zhHpFc_u?(YY~<04s*s9jCfxNX8@jbpfxOngjzTtQ=or_@4x(9m+}%cWow|l{3=oD< zeL||sbNv!ie{&-d^iF@LIIIDx$^bqlbI2A6cP{sx5-#CyQz0Iams-XnUH$pso$FW-l`)eYko@Llb)ZyKWB-U)8l#XJoXBuzxI@$d+WW7 z%<2U-QPw3Jg%9Io`5h_-na2y!d@%I`uEWV4TkZ1tX!m|&VZd=}8lB^A%7y&>$1Y3l z?D5i=J}$GF4@O>I{Q2v?p$ht>JAp!B!-gC$IJlsrV8_PIV9&y2l$j#YzA$x&($s#k zJVaBR^5~AvKQ#Y*6$+(aZ|zui&u{Wq?Qqt03~TUmwAxl!uAcU`v?CBap08yy<6R$V z4&Kip?Y)MsC{`lmP)JK#k(`P8&`~Yer=1$uL?@U0ik`}PrT)=5sE-|?%c=-ginHY7UUrmbgVX@G!hk3A`m%#%f?5Ar(j~ zsa%tCOwr#NFRA?5#h*6uJ)2~zhFeNGePDO}ImXq}U-I_86zgDrR%LyV@fJCaTR801 zk*&-UR`N;c~pm9$P`} z(VY<2Gc%L5@uCb}w8rX^*WbP%il1};Sd$n1girL*EVe*N0U?>`BT?DUs?I1$4*O-9 zk5wMbSRLC#8^bBF`>`zm4C=w;u`b^F*jU5NA=|t5t1I0~S{EHkw6wrkFNgGWnO^o6 zT$W~Hw@o@f;ivEVY08t6l5Dz``f<=yo4)j>$F_m2)8$H&?6|H-J-t)djo$4+)Z7dz z9T0!9ZUTPx0cRccJLuTUcjL2`wC*98|X86$oS<97q81pT~=h2 z@^>*Z>d9OQdNQWMF!Ug>Furt&VtZ))qKX}wbqE_fVSl^ zvo7_s?r*4T`sa-6VVR6Uy%SxGvTH>JrD^(E<upyWu^YxP8E`ba-J%lAp3P9Z!a>*ju1Mmhj3|Q zL+;>$77`yTzf#|+P+G>(=4z)FL4K5%ePhHmZt^r9hTlx7(mP4fm4?plXBw?%o-K$} zhJCZq(w>PN`b!V>=_RTzzm4PH^o>eX2+| zDp^|@-gb?DQg40-c)Orxi@OmT7sw~6<`MdpYgRJPAMpd3;YM1&g=kjrA@deQ*5#v98jK4$5ZcpUJGp2vO z5PU653VeE|k%yX0ex*hd@#R!&c_IA;2CO!^E1A!+tDswIWHj=PE|0$;fd_3_w>)#k z7Z{Rv;7m&A7O7a+AsJ|H?=5PU1CPHzD;{+s7R$|er%fZBw`$PrDLNX#jUILQxl_&P z{f!u7^}a|)2ZYt>RJY?&@J!tVyQ{dI;mxjAxX}b&ZfnuQ`QgW{pROfUO6CjSj86P- zDEaJeKHa%*Ran1|&n%j(*rk!#I3=$!&XTmqK`yPti}{Sw=FlE;pOBloJrtNRPKmMg zg{Y$E`PoB@oxSU2TAZi`VRpy){ArQqAb(lTl`$IG$wHR$v!0yM1Lr{3lvn~L>ou~vb<8mt ztsg^tR|liC@A@6y((F@%SH#u^P2q&SdJzFwYiFpEk8Ai#ml5e6PDDhwe2?p|8#$eJMEbg(APSHmebl96 zYRHOi)(ST%<9f~I<^CWgZ$w2Qh1tBr3kGmzr#;3>JxP;PKj+D&dK6dQ_4M%l9`M-8 z({esEz~W&aJ4y3SrB7@Io$7}dq`;4#KRs~8<$PZ@S17c)C}enq8g%PhoZ-b=&O&b& zcwtQ%-ad4nm)GQvw%~C#`-}CoyDBm7(X>Ff`ti1K9Mt`AG-_%$M3ps0hLw9KMA8a1{M;((+53Vl*W5*7Ygy z2GORNUD><`l)7y;O5-~8x`vx!s{f8)t`amR@9IdqR7|^nh(1_wW%4BI;Sv}ITVN0j zW9VzzCDKnJ1@?|zrRxPX3$H~O(zYI^b?*x(O4X#A0T~lVee^x2xzZz$`B%hIqg=W5 z*le9P$RDDM8#O8Xq(7Au3YZDM4^z1sG<=FLXg=ZqUzOw?(-cjbmHgajMt&Qa+jTIF z>-11&F+TSJ&4!`&(_1N$_f@g?y7n1p8! zNi4H9d+>NF zC4^f)E+ASBD!^eNs-fl0vzxea5>3T#?;Zy%b>S7^^je~fm!J+(QZ zc3mcAqQi?qDJg6hNT~YK$9nz9i#}Lc-meK=o)@r4SB-*I{ zUUeKRJ1{hvA(6o-+ow3^NZK2v0*N`FiO^|Wyim`@FB?YaFlWNmtR@3mC0OkzK0}T6}1%Sj~$&vxV6`B#af|sFG6t-o%VJS$GuY=+pXSb zhWZ)Z4*=vA#X{*D*}AHPs}~-p_rtAzJo1mb>wj3d7!v-5tFm{(Jc#J`^>xx|+J`M( z&yUwz-F{x$)LPoS!569A8~uj2v`VUjh1RX!-NAILK|h?Y+lmNuU0n`&ZrDsgNSn}; z{-6DnCgWyz)0-`1i%%!4t!-W51#FS`SEq%sZ3hXy^;oOpAZyR!M5A@)!zYj^OWUDK z*Uzm;<&)=W3CIMPO*Y0PW3So}RKAAcz>jfy%FEqWu(Wd;OQ~#YG0)O+nQO7>&vCw+ zNg6N9s|K+7vGlU`W2=>HQva8twTQQPt&L79?dbd}a4~n+y^mbUi~;8yTv`$DTP@T) z?99e9K6HiO(9e7sVt4p-@mMRcU6Bl(c*C&{53p6*yRG+`$Ea99)MCa^oRWiOLgE}P zR5>tpRjSQ5uZTSu!zWY3l4af~;xD<(08j2S15dS6t>*&IFu*(t~ubq#VH_8Qk-T95LF=o(RB zl&@cDSn?=Vm4RtLiggHN@s~y=9hN-HviZ?d^KXm#qieX{3=L7U`x8hpDyb(*$K@5M zDAJ!r#D3}Rne^DOYKhAUimzOE%(+{=njToM+basx-U0ohb!7A=)8^i3@wJ+4PK|ym zK6!eNT`2JD$N{0vR!xwl`C<8decaFEODpuA5cghj5TR^rM^v*y|3Y6UzT;ejkW-}Az(bev67Y7VHMlKP zeKEBj)lK5IBPwUO@tsK-1mBD%(b(%mL;wv5>x!D#eK8Kv!U&oE~|12CT& z9D3ArktF2&^p`M*B^Bq5zVtTI2{g=dKKOY`b*lu!?4rb2%V)xOA9yTD`yqq|9(8w9 z%r`u|oUq>U@=yOc;V^UpFL@0x=Q`dS6|!=YW%Qu)UZH3D#v}jK;6ENm5Hp})995JT zJn_Y&?$i}mEjuada0NXxaKo63f-)YtboPV!Fkc-={3~1jcrUxab-N7v122uN$zR8mGH^+6s1rWq3Hez;E4It z)>mf!y>E%%jmeNpBW~(UDnxtbcA9tWFK1-GShUQ&ct&@10<8YZa%H8E%_{i`>yiEy zj^oyaSDK(JgF-4BJuL&}ov!C}wWTnD3?i7PDAEfjqPw%aacyX-KtWz!eiKUROKGd9 zPaq==%X+W*ZpHNu+Nf+*(#`PHuD zvxPmi)c!WGM?*tP6LM!_OO3J2@ea5A$$I&pw*Xq@dL?iE4D^Dpx3_o2Wmv<;#s)U1 z!Wa5~O%|7xWx-5-R8t$1!WNZIWi!;kPo(>$&@R2hK0Ptf0fe|C``(@Io&wGawJVL7 zqW?IfbuvF*Y-lW7I9uz77henUB1`PQY0ddQ*p6&h%r$R4>76{{R0 zT^E$k7 z?zs1&$OGqRfar}ad&zd@Nd4;iUk_nFXFAb;pkQD~So)&v=YYpV*1@wmiC>P=+IuxF zK7P6iXa+SbTdlW)bA94t^!4AAU7f4t$bW0IzGITe>+V^U_MOGl5YC&!D-o!ETzB*3C5yb^KA3}&Zvn&If4G(xYCC|?=y%|as zRJvTMuv4j~uHFrE)CVsw0sb|wZ8h)l9*NMNj>6T_Gx)cgNU6WI-^12wV2Wj~|M!|2 zE{l=0i=sHQn1N1fS7qtA-|okA7dP|9u0wB&4&p5Tt|2W=O-+&81vBaQ?>VifUK2>~ zyqP|A#u2p1350kvF*AE_MKPXVO;b2X^Gp%qRMveSCj8E3I5dz*C$FrmCHZ%;mgo3g z%0E9pH|=HEYGC}yjd+#5k0DG5c2PzJKXC)C2X#Ka8 z*Ktkc_wV0nBHmntFI27?99TCe^yu+~eKzsof9-iDf)tZu0+wx95MFPHsQy@RO2S{9q!N%71VJn7R08}@Pf`TIO zbKb9dd}onLMEW5Fh^=+%2R5>B?&+(kT^RpZ0j>nGv~6+j93P|sgDm+A!6$}bfdg@<11S}=qm0-OyT&- zN)sF>&G+iqGJ(9JJd>_EZ)PVC13UiKPE2&YY~F$W^gIE7tquVoNJ4IZK5CTgl4|2z z=Pg1hW>6Ib0YBkUQ3Xm@dV7@0f2ez*AD1nLgS7lx(PNG;R}^nlsID<3Q+ly}gP%Up zW8q_oVT|nZPzv+gw{IV$`fF-Sm44d&&L-c^xeFal8MO?^bhpxO=_nK}QHowg>s6lJ zw9>Eo1dob}dVRY36ltu!jc9z6N^Zu75KTd99TN$ckenWqV ztdlMsO||2(1--w)j%k@)qWN>pCCw~%S*Z9d9bBRA&=`TR6@L?$vm~@%Q(_FDQe(1f zaRgs07BoSOf?#3*27JN_gkOtP2KMOOoz?f_;XUi+baS3k_+u0 zHlCv6Q?mOtrSpb$v!ix#=&%#1H-?DYZlQ|7^+m>{{Tm#2C=L7iYlg8Ooei;B<&+%X zleOL1@+3F7rWe3cD4g3%UHyUn@t@#Q7R`S-GB#A zkXO(%3KrnH`ONk{QxL(@RO?3onhH4BkZQsMw`hD3IOVjTa5IHqe!^4zqSL>^^85i@ zCHQgiy2q}@_GBT}rmWL)BJfi13sR-UXpgpYoMTLNQAW!%xt+b^PEYC6c2UYOXRT>3 z8f>o!?tW9x7!lF+N1pnl}=*qI1+-|;pgq9=|Zn{_>POBY=9umLDkc%|c7Vthegb$4gX>B}$3I(#Me4x)il8o1W1P8ew^IMd3FFl9(yrgl z{ozUUp}e1|u%oICIN>S@8io!-m>tG7Q{a?cEEvR~2unzUEvacw9)A7Bde zHn<&q;d%5BgLFHlw}FE0lMo$4eTQak0=@Q_vj~uj4Ali)q-`M;m42aj1-sEe}8?^xFEd2;J218pR0^mjhN|c(JZ6cyWj&CJZecKKDQ>D0)x8`N z4@@X)3FNxXGrEWCy<*=5G!K4lHx#X$_zrX+`gN$VlYpG&WJ<70LN+jL)EjrVX1>}w ziQ#>XMot1iJ)F#Uj#-o|+y})YeA4POGlyof&!_8vR5OiLKG|z-`?l!eq~K!)3DY75 zDqvDF{y-9Ah>nSoS5VlFX0Ehzx!RkzM+_bh7|zK^HgOkID{|}15J}i50-D4F5O_)G zrL)(-Mf(v2*YpF3@r+4Tkw~{%!Sw90AGKA48hJYq>FW4dw{r{aG9T2f0Dd(Kwe?9M z8G|f3bdsu$6hoFf$xCyB)}v)H+xtEt+yr6T1qN)^Hkrz`JQ93dB9>jF!$ht~TK!B_vsJ+A7I04@ z)JxKIW%OxNlpg5C&Un6zw`+HcC$O0Zh=R_ZntK|_p3n~uvWi8pC5o%0E*Aw|YGV0? zlQT@f-@(W8JfF6=*O7zq^C37s9cIehN}_>HyC3Ki@=IvU7^oC;WVANN zGp}SO`JxrNQDWm5n3{QaZM9VGgO+P!uOliLMN4y9E^Ykqwl26k)my%Rbg>Ao94{aC zbEkoUf$yZGGFMu44V~*0<9Q7*T40ov)H(y&a})c<{^;t{jK!9o%vZs zpEM8i$H8GoS)I%A;N9&bIX@T9X6D9?pU#a}75tPm%>5#E@o;HcKRTm^;qqHwHR4FA zwg&fqe7`w0s{tUb@(Mv!iu~a-tz{52Jn~7_#(Dd?ooilJEvUp%bk_1Du9o8u>@a` zj9&yz+j0ZZE+SoIE&@QFavU(HXZzBO? znw91X?I}2^Bc?a%Lrle7Cb@O;e-&&AsDsO}GY z3MY2(4Jv^uF$zfAmLb;U79@??a_gD=)z1aYi?vYc%7vIy8o{ZJe&fixCOY}nJ>_U4 zzclPKw4S5i3>**_q&nUz=L5>O{`jNLt8JGEgUF4TV){l~ukWRy#nZ{>3+)(hDERK;;$fZCtVi2n%+dA}Zvx%* z*iBKQ+tN!I%-OlSvmvN3)plS3JvMZkvYPChrB-tv63FE2iQY+_>nOZ2YQW;&)i4d| zNQAzKIV&q14|D{NRy>K@N~I45$%QH`Qx;J4GxG({q9G*zxnQwdWcWR;HewKSYy@K2 z3Er{BstVpI-Ns@m7(zzwPFch2Ja~AI~?L>(rI6G z>|MTKVqzi|^J|KBCa*%mHthf+adyNMO>+2KbD9cS#!lS1&+yG@yug9i(W^ob7Z-oL zIW98&^TMr1A12P**%rnoXr$_G4gRWI7GB;I8*PMcWfA$_SAVDgzkY)qiKG~x&)s49 ze_MTLHy%50y47LPqyKBkw%iav2*W9Z&Qbr~t`=)P52IjW7TD^P{j0clFT8Z>Z3w!< zF%hI1+3@}tpI@pnL4ZM$JgJD^|7d9oX6=gCZ{9@i?3n+nw<(tpkg<9uDx4FyLmd2P zr0pSgVjG>?9<`p!GK$mYSNo18oGfWUq8yP}QvJoMquwNot?8GSE=ZW9zf7N%YEu0} zYAYU#+If_ce}dBOASX&94*Z47N_X7s>UpqK=}ZBu7lEb!Dtvso_wmaGplM{Jf~uA6 zx7hgj_)}g8|05*XTLT{?x>0RtUAmCcfCdN1&3&hj^<#BgNki2f9}^Q389aWvuX33d z#gz0e)2Pj7^Z2+F)^!q;6DM=^ETN5*aQVY~+%ON<1h*q^M8ua=*N!ID7q4@8wBFnF zxwyIx4i9gC!BbY{gvewoi^Wted`AZBOQ>>WtB5K5Z8zb(AkbwR0N_GF!6@98w=9+z zUAf#HhIcDTB|7lCOuwfxM-YC^l+lx{tQ`y+lV6B03nW+)S%i=jPj)|X$;a^&c5#;9 zAIpYak<>ULr(zTVNO@|U$Rc}G#M1MV`y~P}DygM^PS=lzOK}xLAjx4UtD(VUco z+hD)k*Aa}MiBWktTZ@A7>|IAeIUgO@-Cx21Yvhn5lN)L*A+a4z-B1ws8J$_H3~u7W z&Jhii+z)lLTOJn=8l!VVQGlVykNumLuZVHOD=XQv>u3|ec@z{BO3MxQni%X;hdbOC zH#hXu0%>vGv!yV*9=~JIbs^6@CjkNXPfy9I?IGm~Vz4jbjOdp1@ev9|#@6C&9cZAZ zN=Ut>2!KHDkq~R1AFMTqh<g@E1ZAC zmYv}9=Y)$35rW?&dbJjq9)N6^t4oNDs9|VT{N-(BGkN`=v;&OvqtX9=NxuenozIQU z}3JYsznaw$!q6xQN@ z@MlH>uk9RM%ZvhSiyG0hab`N=bA99;~3{Mn)bUjo$NcG0`rF@KgX{SOa;H zP%L@;DEmg$xV#YP1?-DUN=i(Q=gP0o){T45X<_~I0pVbED&IxH-aOPIBTk+r4Mt#` z9`rcC;+%AtjYYnlL>}mKQorFlH`7KbBoCkhgBdo;$GjcJ7G*%x2RV{=I;GN6>*-S7 zrye%L(NW4LGQ&BHR>N&1dW{YgudV1L8SlqM5^RF3v-u?iaVN{fOT9!P7wz0Oj zKgk+7o+$^mke{X`T-JXe^P~0dj}C|SyzHc;5EEjiaH1m^fbMqmCJ$;rkmWR=;(6Hs z3tAcZCnt6=P<6Q6Sov>YX$S!Oz#(By6&U@!j@!V-nXg%@^(7+WKNoaZh#Ae~pDt3N z`ZuW9!J?3t2qnch;a~Ep4-60$6BB#XWdBRE0CjadEuLp_qZaC%{}8WR(6X$gBs`2_ zYW-6(8qn%q5@#9nCyWl;FH{95B)nQ`oUfFQ`cq<%sT67nMO8(zX8`(fZT4i<$bEpUsKBkMMc{eUTz})&|WrP zUd?|h4qR|eDS!AdRjvSIS4s-Sm|2vph)VKp$Ujd;Jt3u~rGtZm7tz~`ogPlbz#U>+ zC&_QQc@qv;El6bF1nMXF>36XXa7}rZABFj*g6r&oZ_ zcwLxAWOkpGO5_8wwx$l-LTk4= z$y`Uuep+qPWG$h%Ec78{{T6mhlAq?RVw1uF1U=n7w7*X*<;h- z!apkq&d5_eMNT-2blUCd*vbc64wb)2qm3!z2ffsLhj^6u99HC@ z5U}Pyu(Xy{zd==BzZCG}9}k1QL;%L(VEm%LSW-_K83VS|0P%lU1S>0MOz*o7AEN)=4%i?z!hD60@Mv^ct)5?#)Ma zbyO4-YEDkIe|3U_ib|2?cn)!R-oG*dHi($$Xf(rI)~S(u@^L(1#fZ6VtbBpG@y@3C z_jI_C)lD@{ta%Kr$SdvxB9m8#yZl5jBgg`50k416!Owlqz!y+7aA36rGM10@lo*MY zaTtg8NXVhu2Kc7O4Np|KbprWX-j;=;2rszT4{vH`{7Xz-oG9^Z3>e0r;gSKm!~~!= z<}Wid8nEk$^=eDK_Wbp+X>vcei1g#1e`4D+N z-Nn?eRmX0w>1%0KR8RmvQE41lSz)48FaIsTOEIn*CGbA4=7C&U;Pf3}#M*S0T0f4{ znAylT#0=*0i;D|bX!Lf+EY031Jyzc#t|e2|sDuG__8C~ZXm2#rorovV;zJ&*3;AN5 zH6B~4Lf$|8y=eV!xEye%6&20y>5&cz3HcYDGQ$pKEv@e|i+=;K2N|CoT-{$m8JIE5 z7;(KUTWu%bJ03sFV!jy$-B`YE$)#fUiH#r2?b^%Qf7m%HT6{ug(|3KWuz1*=Pf~Na zA0`z!C2wdKiMn%akGyjLMm93$4j0v~6<9#QIko}!Ap4MN?`jJ_(11KF0eHk2$ z_`@I69ZkT|E8^Fbl(wkiB2GfgT8C@H!0+F`hsVcbi~3xva<-~Ub(@L-4_+GqHeoON z{n#8;1d|&bTH-VnT%qT_-XDM_`+FbYnA+QeRdN)>Pqt;%Z|3>W=d!7`?z7!zs*Q4= zCvS{h^Lz=wl`q}uY4SYrg|+RT6SSz@dMS&7IN8rZfmn)lPAuM z???357yhT>0-!@~RIW~tdPfB#W9?hpS8xyLMn!&IR$FxOlArd09sASm`MvcjPGcL0 zenGz}t{zS&$?oU&fms678tB7nD1Y|$r`p3f`n92>gLgw)Ij z3D(nr#+lOjHc5}qzDx9BuY~;g%p#~D7r>1EkM4@|j6WyD3cJFwdM+>ccSxkHl|;jA zYf;3m3|OcFx{dOQCcopNO}j*rFA#}yBR94sP+U2D29M#|peyMqXLX`V>22KFnLT9{ z-HVNfuFgmGa(yx4HVXjlW^^#EziyE*Y1jD(|EOm71d_^zpIgQC82}+uk0g|&Q3os`U+RoM|ON) z!8z7nmP>nLF?@C%2yjM9_ogfZrnTQEFkP-azFMcLB^D9bEE>ml2o%e={Q8TA7C&CC zw|$(_{AP-lMRKDewqQ8rZmK??YK-Da_jU_eBccx%KP$2v{shQfdHSfqH%?!GT-aqHFf~^wt&K#*p39*5cXwDU=j1E4T*;tUCBd zDm$+QHs;jSxWFVl3G4z!dK$Pf<{OMn#s2FvJG$okFpd3)>CG(jA=oK7Jl?f>2_@#V ze&eBlDpjF7`g&4QX?mYIKf79RB@c2f4q(x+(7=Zz z1O=7VNK3{P7dN?9>H!XKqBSesnd?6tYZa`~+RR4V--NJKe0ncACqWG_A5=An+m)1n zSfgaY;Cf0WiT|<(O(6ZD-I(2oT@-`cHDKR6=S*;1m*E6>Xz%-7g&*>*TH9_fxq00v zptmPrh7cN7;8@XX+qhjC%ID2$zb4SqPT|TjGc)62?nn@21|8_=j3U>2^wny=d)Y2` zKC>j>{LG1jr7Uv#L?tMnYhCMo>AWrPafEwW1RM*9r^sB6r2{cMf?W_zZvC`Y&q>#C zOlB$_WmBOPBb~>_-J%r;eK|s;!=5I`PDE0UXAz^CUR|7Tg)~4~dH&()6qj<5-jQH3 z#g9IY=Ii?ua!-pAe(03j63y}kel(wRJ*gjV5uGHj@>ahXo}-C3F9zGrBD7xJnjVFJ z?FVme+ZDH`bH>NN{>FeccCVt3%YnY4 zWa4Dq9VWv%{QlU6IXKW*SnvL4&Y`+a!2O+gq!R>U_sC~8R6ilF8=&q02+y1 zeQ0M*sxQY8gc$1-wNo(fxDmO0!K;jc=gcp-;7r=NbB0l_R?2M9T+HwC{ZAt@m7CJQ z{(wnw;#8v8MhJpf`*avsc#hY6PyNJwWw1#$B1l)Uvl1FMIT@Epzay?L5wYI(<Rv z|B6!~0D@Q`T9oX7d}omy2NjM&6ejoEWtwYkj2g*xzkA|quuy^z3J`z7;265adpLJ{ z><279><{aCHtKa4SAgG(`6d7^@i$vFtEPvp#p9ffZ7bz zl=(&6G%q@|639|$Bh9xA(fu&h!+(%zy&?Q@U{3SNiKIWGVfVFv;WkBMA}18RY1g_2 zpjqvIsUad|WVVuEl_7Ooy*s)5_OS^YfgQ8gay^c5)A6f!3x$I*PD%racoS&ZQ;B;q z7ueCYyk@%pCBRM0X^YdkgScKd>^7b4Cu3;b{(}zC4%uP{m&x?eH76h8rj1k=*b-7f zQd<0hSTWZNVW@*LTgtQ$L9vRy-=ws%p!1`G(uISm%Xq)Z|0?aP!=mcGwow#BX%v)v z23$D9{;v0ZukWw#`~Kb6+57B0 zd!4n{zVCajc^DKsC`htrz@uzpN^x0u@e7}k1g?ECU!H)=JFV5*PihrFvxagZ`bPxJ zC_C}PmF04DU~#xQ2S4U=XNZP|_Jzq5{0iRG_AxZ%4=ok($Fl7iX$388;rCc?C=30_ z!}<=b5WnAR*QbSFt9tHbWQ9JTeb-y+wE20@H`V-l)pqd2{MR-}RJ1}=C3MydX-!vU zK!fWJQtSCdJd;w1z{Z9d>st#|z4XSx9a+0dK(jeYSKaf#Ru6Qwz`GxKcOlw zclbIFby<7@@0>9emnX4FsMZqpq|^{}$P1vI(&W@00(&8jhF5vo%&Ux+I-I%!%{$GX z(izBgCtfo%1{Pk@4P{4)Id|pHanhdpV1sXm=d?};RaSPSG(rks=r1* zIQhCuL$)(8IxcN-+(XJ!u0hy}&%5gSV{UiKJD<~_gruk%`WEKnvuQetILQp~>gjZ! z@?z-n``cenPpX(Y5(Gn}3AP(5@>8k;P`^m=T^J};V)UVi2#PMR8BkB%2K0~8WJ)sP!$EgxGlqVt+hqB)H}cDzpMTnHI+b|_fI+)FOI z?eDHt-7fnpu}2X>>7%&lU74F>iN%_gji>oM!EkFjG^iw4Nm7F+f#ul*_()dv^jmG= z74qSx#EHt)54hIRuR0S=FtA$EiFzElGfkYu{aW3Y;<6m*pj$ql{JuJi>E&p`9D#%vBUGV8f;T~{L6@^T|Y=+(|mt4SXmR^&NU&myV--3yeIKU24%pVA#n zmT=~hxjnc&)k4&Rn6Kg@vz{_I8D5PBlQIiSnpQ(a&xE7q4u!hgL+G@2tyARmy1cNr zZ8qpu7sO8z+-Y(;yWRKe8;ue5C)l7C!9lk{^`jrw5Bq1NNG}>MQSF`_xKUPtUYjxT zEM>1?N2lGMq3XJ)-HgoU630n`CM|-wxe~GUY0C;d9yOHfP_j`WM=>elXyeltCfBgKk8;K3tu{Ux>_{ zy1l|4I9l^Og;IaT%Xq1Pn;?*U0#2Ez$ousBu#owi8&e;zkXXOE$pqhy;%Yn%!4j#= zXdhAOpS`H}$p@sh1cc=tfBLqi6*@+9$--tieB2;?-jL}XrPBSz$1mM@33Wv;rYAW1 zR-~_ACtmS{jd1$4_D(2(aw)xscKWT3)?Y*pI5A1E!sKrxkm+3;9b2|aC1Jd=%dWVt zQwrNRhXZFwrMR%LjnIme;8oQ^c+f|a9`^F5%%O`nrm0`~QHu_9ooC(=6<(uFBt;P|Y6qvT)$Wc3~O zBo81w&aorETvDE3qpe8YemdQ5%tMx@R z)RR*w;8rce>Q}p%JpF!g2W-8!yTTS*l#E0~hO1CohVm7S;wXy3tbwgluyx&e=kf$H zHqETtMF>evxJB9M@kjJL^O9*j?3{JW?GDGeOVDwzj}Z)GpfAL`_-6L@C+f7e|zXDBF8dCj!XrEMdn)~TKdl}2d-9L zyC9dnygsWdrxgb<#yyL}ELIYfz_L9@E7e(ecw7Hg9W?25^&yoFU#Lg>;1VeTA*)~r za3i2NhBu*=O1>TGj4Jp2h6V486yC_ie3|sp0Z3XqECcXFG#v86jyrAI`8&&$3>xT} zuqJeWZ_mmVJN?!kZBVqS8YClwp`f6UOnjP{CIyS@MHhLknjK96$0~S4bpy}Kd-6#H z+^@+FluKe+9C-TdnT(PW9S_gDc0g2#fFPY}_2Lr}V&men0KP$7w+(hjpuh$|FcC0t zrv|{7vx=4$ac05@R&fHLgS7RN9Y7bi1)w_j;}yWI>Dk!ER1kOU%ay+GjVwyq0O9?} zx&S7;u&`iiW(J_20AO^WnPXM%wxi>9x?2QWe17+g=oA1m0KA0&jD>10(;^BzFOs71 z@!38%ud`u~7eJ~kUGbK^B$RI=-}rtwFQ-l~T4~-#jB;_Z12=wDV;eEoI-})YXpo#E z(|&@_<9nv$)7SS^gknwqhJrDkz7ln&uC`?8_ivBsa&`Mghi#;Z>Lg?3_3z0lYZ(9> zyMv6LIFQVyh$*!DFD2|S>vZ>LCOpI2!44JIg=TFWpu8BOC6DTKZV^Q&qYJY(h)MmC z0AxrCDLfkm2mi!?H?YRV$>?_A?A=z3vGkRJVy8o6_=Fom3v=x>IkAolW{j~E04Me< zOD<4)wTmfC=A(BD@>vvw#(Me1za0{dOu~el4ROgAXjD=C=kDNq!ieDMX}uTJ(x#F1 zi$Q|pDj409wS>WL4v$aGf}|wLtCH9>6Q$O8Fp9~xz0jY`qIzVlv$9i5-4Kn{jYTn zpTeiWl!8Lm^(2`5E(oEw3R2%=$zYD7{h_} zsw4vM#CstJ2rbbbidF+#lbM;h{yRS|9522Z{5T^cBS%#uHZe^P2IqPXu`} ze)`Ywgfg#=3J-m(&IENEmK6x%nzO=9D^~G_`j9suBuFg?!gPIM~gC-PW z>jKL{ZK?KAtfQ1SF>bYi>NGI6BwGXQ)FG0ukV;V)3` zNf6*ObSn+Ba`&pnZ2#`6QFhe?OeZWNf|{CIqWwvZuUGQ*p37wgkb(B)XD>-;AD-{e zu{&q;;wKJ39E_G2l%N+p%HP`z}c`m8Zt5~9Qcze)*>Ewm19Oz6 zGeQ$Sm)6{IZqIQwrUa+=H3ycvyt;}{Ok806sI0f%aWW}kICH}>hN2r__44xa4CkBt z0kHSZiCoi@C%B^V-}6py>Jra#hwu4Sn|9*^d<#IY{&ihi zWdT7!kA4jg0|57(>w;fgc*)AjzT_~4gj3#kLHL|I`G~-BI_}-qo<^aTn5ikv#Kgqk zmw)M8GU`j_2IRlEt#8zD{~ZM&z2*-*xVTyWeXGmHdmA1S@WqkJzfQH^nMW|1wZv`^jJf1D2o41j%5;w3J}weiS*L{Z86}kP1b7kBLY}u8X8Ga6Slk2 zk-!HRHDzNyUTXArMiYzKFuYgBS@v^{K+7W~%gza``o2jAC%4*U%AdaGv+(@4#9JXCCsFObJ#-QP?ufjOhLHff3Cm00(9HEqw@70Cmdf&H2nKf- z z^%x4-teP(Ea=%+&9-HwdK)rj!nRUU7en|2zE25QrisHdY*9O#YtOaAEJ0Qrjt>nbp_#7RzhVPm0+(_`BxbG_`@> z{239e>0r8;Bnb4M`B3x&cDDlv>5Knbb&34&a+>F1*Uz7s)?>whc>V6%RQL52LB}CC zovv~Gk9h)ec1T2o^^DTn2!~H{qKa%{#bp>&t-zA}3+n4r0eahU<%_0qS>+<@7ku;= zBa`X)(C7f37x(dcg1~2gF&NR~oR1rO!S_FQ8<8&f3sag8Vob z>XwD0p_#P(@r~6Vv{iuwwG@>4`c|UoEsK}96dwnvi8-h~J6}vVul8mD-wzlM@K`m4lVYnVGZE3N9 z1N!{Cl5QqlBaiTA;iR?I^+7jx!-(k4L*|N zlE%i`c4cvUsrI13B=XY-r4kGn*&9#J>CWSweqV=yRDOcQp7tON0lx zxh08A6S|x7jz94mz9GYo(F)NlWdnnO>DoKh1~mhNOgj;fC8I5TQ5fP8nn8$bD9$op z+PvFQK{9QRyygMomK3`>8L7L6CNzdkjWx~DX+j?}_AhdZ19>e;!rnYY`Ytih@_*wx z^ZryBXJ<|z*(2+(nYo_r8?E*xWo6miJH;PFA3Pp>(zUCV`Wr}2)_ZbB5V9=hq}Wjj z23brjVJ5ZHj^Fz0t$@aD|0t}2H>PxBR&Ws^Ch=&kG#ePD! zT1SWR^`vdDMi!?qxli(T{GL0G-b|+8dEUHEah;#^+p(gI41eU!ammrvSk!ETH=HsI zI4kI2CE$(AmY_;92GLgXP%be&W;tTR60be^n4WW_-2pyeDrYzr zvl2&omrQ?%j=oo~F`VQ~_dqg+z6$1)+t>*Oe)9{U0kL@Bo;=BxW25PO*r=~i3I51< zovN_yy?{2_aF2_2x;9B|o>{z_V0yqSr{&ngh0y0pv->|l$kqK)%5Bf%y>Ye+bfPt> zr2>z@DB24M1%_Qh^sW2@GT+muNepP@1D(w0=LGR5sW%O|M@Sst@Q7znm6nzsT#~)B zn6(cTCY-@>CE3f?JuPu|rt9{sbak~@8}2iG-_U*gsRmw#hz{d~Vt{1j-U!Uehm-B| z%cSg+TD_g2f~@n!yTyEZ6`qwwpk&o^C}At3$T)W)OmWo%9m&hhsfDiBB@kLfZ79Ak zctERVLR~g8_MWIb;wKwEl6{jP;dbXdD&M*uM z_DTaFG3RGb50GKT==MWD5(KzgCp&F@{f+B=jD9N$EQzefa5T<`K9~7L+%kL@0&}Mx z3Dun+TIwur!cUVE&$`nkr%$Lxr>}Z7+4jMLC<=gta}rtBC<8&I{^`Utd`n z4}^NoGp=JXU%q6AP*eGeRr_?a&qnLpTpVIj*?EMi@N$(coV)Ycyak}44(I?q92J(G z@Q2s;=c0R$;)O_#ikzz1BlaD96ic1s2P;?#9w~}tr$DI+`5znwvd~KA@zYmR?sYmG zjS_y7T4NOzrDbM_^!(_>HH)+=ipP#WX5^DNf&m4?wFUyRF0Jb;)CcW*u22^{5G80_ zHMCo2ACn>SMcKU0o6ob}&Gj=DL8u+z`0?y}E7jLEd}|@=O&NX}-VI&71so2%75ElQ~t>dhA5+V4Of_17_qymz3`OZ=V zoB06Rh#;UZmBL>{TWfb8E(&zq_PLxuc%Rzk$R~*pd=sucnY6AFySpnpaTui)FD=qF zI=Xm6AO0vSiwPz7x{H5k6_SID))zSM&vC&Vt3Tr=Q7ZcSeSmZZ$V*pNk!)kz*mT73 zDnZ@|0H?greiLe?{oIFt9ipfXpggp1Jf~gijI!9DZ~l9(4s}G5BmsG7z`?yds%x)) zKZC_6g#BDUDvr=wfGm#^KKaTjbaw@ZhywRlqSHA)Q9xKDbo^U2C^#5M%)E;#c+Jer zroHzX+S=OodpQTzWgYK*bruj}I_x!^;^W~p4=afV4sWQb+`|F(ZoAXnGwySdedbFW zxiDG)7LfYaZ!^%-&jUYV%Z3Yw>n1D=2*Fh(zpMWu7@|oB?>D0h0hyTqAo~W$+#|&f z4ZjOb4%l8(ToK4qh?FD->WbQHpWMuqX-T}n42&*y;9#w1HXNJdQuyFuBmkf*CcpDk zropE(TFl8>_wFvk_3$R-N4G0;6K1R7ac|-qOjH1aNQrlQKr-1^rCxuuzO+4C9yVo< zOoNw<4jA3#j$Ub*raZ#lo^-ssQ@g}o$C02qHS~lhIGr7@?FaJ%NlkR?Bk)AxK*%G^ zJ|eWd-K-7*LeE9rWMh2Q?ivUAw?!q@`7!Dx!rkGC1FYpr8}boeM0I?kHtf z+nJLv0ne0~4aqQ=LZ5sdoYkyPK6d`F($%Qn_TANdaDaj`QvrYKG-dJ(*s_Gt*G%@L z+0QH}p6%S-+av38MoPwFz(2Hs%uvF7fL{ zMz+ybQ=3?B2`7R}UC5>qeBNp=lqaOc;%WY}?Ts97J-TuL_FoQVox_G1hU7#Y)|45A z^ytoI|zkcfTf$gNrvTjvhVQnE14=*ttCOrHcis&5qlw0<|-b9#O6IJ0DlqaaA2yXY? zgya0uQkBtP@fE@{mT37024z89Uy<%brhG`9KOdDw?x$Y&xwP>{bjYat|Cv zkjJlb210W)$n8~8cRND%re#ndHgWPkp~ZZvQ%myFlmb=M2d^e34e!1Q{uE{nCL?y+ z(((7DAwh;1Wky(KM*J6@AFGU_To&a&MsPAj=Ea{d0tGF^bsHIS^yz-`7GCs^VV!LR zGA2NztpyC|30hQi=B@FIdDgZp)7vs81wWDQL#k(eEbk`EYSQdP?ot zlQ-l9dgPWA=6QDjA@+04Y#qM!0539I^mx}1+G^9y!YIh2oM=P+0|zGg4YP%eW04kS zTq9ICcE3*1J2tgCBdyrxGsI-{U2i_^7&Lp#((<~Wrka5=ahh0}+#Hw?9PV{ItXFtX z?}AqJ!Z*+KUfQD&Z%awGAvcy`|9>k3dIMJ?Ny+8uP`4UXJv()s09_LTKC*&9$eE{Q zZdIK@yIXGC1m7q^v5;jqvzya$V7 zHLwR&hMh%qmCC<=El>ia11%3-d7J1~5C*9X7`*xQEl}B0Ypv!rpB6JN*#oOE+zc{x zJS$ruKa`5<_NMHUEKqkAI0ybyC-%R$?Y{BFSEnT-aN6NrI2cXDEDM}WL H8wUIr0kYA( literal 0 HcmV?d00001 diff --git a/docs/img/JobPageDetail3.png b/docs/img/JobPageDetail3.png new file mode 100644 index 0000000000000000000000000000000000000000..9f691e4ed2b6bcb3bc371622f9da09aab8c198e9 GIT binary patch literal 48152 zcmc$EgL`C4*KaUM2I0icOeUDv*2M1EwvCQ$+jcUsZKEf)ZFJ0=^Pcyd@7{mlyY)Q1 zpWeH=t9Gqbweb6`5E*F^gwHsiAs`?S#6$(pAcqBNA%o(Gx_YPuMSjYzoxc;7I>G=DqLQ>Nft{;*Y; zbr`{{p;ltIwwkHmX?NA4iEx%hPn=h21L%L4kkn*Vp1~7K8TnQ z5JV7Sg1;1iODC(&4hsG2gO_(_M$-bHe;|vB9<+ykh$hAVfb!*IaA+7Z}XK^CHnVfE7!Xu_}`WWRrqhucdO6Iq5t+Y{nVlOw`;5Lf9+3uvUYJshO1d3 zE*|Ej%U`MTm%C0*e+AeN2}u{v2j+J%^a(mFnNR-rJnKaiQq4k=^fj1${_N zgMQHV>yog`tUhj-wo@fA68KpdQPywy7N0&wwWfs+8mxZk+g++&g2#dyQ=n82^|T~* zY4Y74wmfYf(@qe#i?AUmKX9DYMy~cc%Ri09v&bw+X2>P=a7Qq}m)7Yk6`Uhgsak*s zCS7h9?x9Fp;)#A#!Wds3A%?S*q4(8>Rmv`1LPz<##gr)xWX{TCE2{?WaL5;1rn!_BWbOC?(klm@xl>Vb;jec>Ru9<;UAvc$YVoGrw z73Oed*ty;L7)Mn*0Gh{lZo17MV2}WpT?A}V4U%!yxTj%cyQ2qJnO%49!8kEAuHAJC zDIVT*{3?<{eh8}yf>MWSj4&4j;@s&3=|0Zz9k&;Oz7z`9i*NGG3=)};d)@bdUhfhG zx=JMLv7A9OuwXo#bb9_<%#a)zGS&d&>^NIht(76e(`?1RuUId>7XQ4$RCrbewV_%4 zJ?tqcD}4*^KqQZ%p!oULNoyF~l_(3*R2anhQ*V5UO=~f=KN23-sVGEl_vH{*KIUzf zo!#T@u99`qzAPa(6yHVURUw{0H3>w>prgc{8)7h9$*nET?X5n=;U6MjJ6NPO>9c)p&YPfh_7M-=;}`KQejE=XSEmx;Lv(Oo)2KMx&4I<_;G zO8Wk<@~KyUw1TOp0_f*09*f7vRRtC6+!yb_vGNnhfM)lmtBKq(8uY zNS9<&OI5#w*5Co#d$iNt$X#LEX4CWSE?ci`nfL4o_3zVWkoJ7{1Lm8fMfs-yGWn%3 zDyg|g$V9G0$87b$w?q`*gTLWZ*jcNB*+DB}H1@A!y|BPpN(p0VwrZJ!QxQKCvPo9q zDb7ClWbB2msGrrUSf8o!sBC;IW9G&M$U7-R;`0smNR)N-l7E$l{a3w1ip6~}lGH&p z3r29Lq-yn_xb?Y3p*y%Zis_?V(7oklN4LY^?u{B@S-3G@*x=F>0K1 ztC_QVWL|*MHV=h-Y^RzX?w$fOJeeF<$6$Q`nrp|h2R8YQ{w^XO;TD1l_@m&e6QLk^ z_J{w~8J3O1tQ|aOM@f4yZm!27W)>h90sa@y4ARvOEHA3`mCJUVW}EwrPw#RV>@3%C z_^9rAB=Ahk=Nwi<21!{h$lW#geAN`;qjyup(;=>fQ_-x=_dgkR9)$Q9g>jKSe-7lB zsxfOeZE$Ei^!V$Q&n_#V|rCVT~UKhoW-fuJgMpuYo#G0vx}yRC-Sqnh8^mUC-bfm7bZ zKU17zGsvb|sOHi0EhZZY{t|$8ltQ6+OMtxm@sFTk$|*5uC>#aD7mxbnH$HBjQtN@O zq5TBIpA`=H68feTKwPNgXB7|6wn{|Gpc#E}OKl3pYaLLvJ?&<6M)Ig^m&L{I-jKaE zhLEiY_G=MkCG}0OG#A&XHh4D(*(eVc$W!*PhOc(>zD&fLwQYg_pZzZps9`Vel(uriuv97bNxO&fDRl zRyQ1X?5QF2(=TOcd}xa&kQ6MR^C~{kCFJU}LsRN!pzW%O!p+yYH+riXY7lckf({t! zZY1`mxyZZ-#sVIVd8pUK2*M(p<_*gU?6zuk>-mG}4Xa2g!)RkO>OOHksgjM~+|`@{{G`V<8hJAMM@e>395>lgd3@s5QY3dW zu;w50!ra`#7_(HvB4tZ;k*O)}r}iscuh~JSgoMnYok)rkvZpnOgqssEpHAvq=a@1U zsXt`N(-a)Y0N7N5wS^%sy~=3+JzS3DiQb3VusIO>@Tr-UDxWuYt-GLYbjWJEK6Sj| zF*)k(+7wbEn|``-l)B-0)ktgo>yu*DvI{*fuA z*MN?zW50KCZI26lv_6M3LaQRA3~{NLOuL1!rbxf?EI_8^YN`q{a0v*yvU`)!YEWHt zWQ)tvV^po?4C9qM-6=A>xF!=RW^Vh+nrV-uXRK`a^;1RDH& zYzQ}yOt7ckIU>sIQVa?ePXUZJZpS(iX7BeX=vnG~gx?8pEBiAIzHY(BGO^Q?32@rl ztZO0vN5aZMdg{H`8o#E!J|{vcP@MX~!Kj1_J);>L+e9$Nsl-Qj+vO3ij=CmBTR5aU ze<>Oq-bfW`=CcZWelp92J%Y*f5-eXdE%e;xqj;fKhtGU1iZH25rs0V&o*q;+i*E;JpQRgGkhU%bdP7>D;j3hsdEsy8;V=H?O$FHbWc2l7x7o6#@jlLNhaG7d zuR7DMQAx4s!1-Xk?NwbtI2XQH1kjj!1El3j z8cyt&V8$jW%LLzcn#1OPGBiF9=dL*`UGu`@c_J@HWBfrA22>n~9be@DyRicdhe+-$ zZWWb^Ix+&Zt#U;^s-C59Ng5%)Rz<8Y1m(!UVpCO~78QSJb(#`V?HCVO*izD7CHlQz( zWu0UNK3w4nX2|{FIByD;<4o4bLSWbm4?|J|I+saE9`U-`ojZrAvY;%-(zeFIOT6c0 zmjY6T4GHC{nMUys>$l|_YBifwpI3b3T7$%>|0kX)T_Aj9D*in z|K`(C+i1V+D?p~jIgu{k=XrOon*db|g@yxuFp=+gQlwm`TKkp6-U`&wr_ssRji)?^ zU&J0`JIM6SN*M@mp95#bSBC85sh&uaZ#@y-Uam}7M51$a84=UdN0509|A3>zZa0

np@V`x*Cu-C-`WFH}Ae@Dpm-vQL!B~9H z>b0>b*tfSN{DJPKjv5~8qqR~B`x#1fftljY0|k3HmyY4|{&KO>Kw5S#f8-t;y>9Lv z_2kN_X;c0_+Ckdj1#9Dl9NiC674i>U5(#@LEt&gcM-!|fe|9!>U&khNHm*C1F<^fV zhKe@&7-!I>0&yUq4SUCdCw01`(z~zo;9|L=@01+z+s%(9*XwG19fZy@`?T64=x8;p z(}=>>OHPgv<`HHJnRA_|`id19ir|JR@)c--8iYzZfG8UXr?3!$)B$O`5oWk=Z|`yr zvlVz8K1DBEZJ*D{+g^-f;7atvndIX7$(dNr7JrM&3|ADSSfNm7SD7t|x zF|m>i^;Cb~#uw%b=@^1Uv+%d18&Y2Oh9qSjE)d^NA&)zcpm?9%kH62 z_pgOv4kuwyBv`v{ZlxJL|5#*CV%^iGiR9a}?00aWBaB!)x->!Nrw!Y7-~qtrG1pEN z{v~Wt0`p@pjhql=Ebnec*To}CBD3#my!1jEnidaR={G6rrmP`wx7$SRO+GU5=V%g= z(q>2L+K0u^1-Cx_j!q<2=iejN&Vd zH76(;$#aA%{%X*JtL(jmwILQ!THvyk$?-dcohNi|Z1t4b1Z9UJ>jV#O&8T>xje*hX zqUd)Js)XG)Y@dP@?kXJXy3cv3tXTG6*kugh-WbMCyco(=VPrT9DQa<_(ou$K)f*|o zp8Tg-R#4@wAl*<;l1CD9?=Etf&6h$&vU?A39Lr(#XhCv}<#8(ZUwbm!XUv zxlsT@yVrKrP2bcZ?-Td72g@>5;u_f4hj$O>oi21S?pI~}URM2Rtl(^ensuGNmPiB6 z2&bY$bYao+|IP7+im9S0_e7!E*H-!SO4PAE$GBnE6G8KEU#cL5HeV~?gf(GdpC3AL zHS%n}IFo_LgpcHw zNJ(PJ-4S{ihSToU@bYbdP;CZuklmuH$@fw-(*dO4y|w5x_FU_n{KIrNmT2 zozc35N*L~#AQsLA%0RH3BlYw@5StzIG|vAGs^RGVf5FlI_Yk}Pcc#G`*DdKHaA;At z4qY+qKX~F0y8Kr?1I)4}eKXko*<{M3;_oWuN_KX3L!+a;X7w0Pw&O;Oo}Qi`K77z& zNu!m3{M(-nB^Xz$h>D4QBPRCi>FGIUKx%7m|AdH0%EW{Q000uIM^wbUx9Mxb!zE+m z;>gL#yQ~_MhUy-lp5!~+U7(<#ytg7sG@T)*<^2kVhllU)?~##_ZB%It!ut88r4eCZ zVEh9DewzHPAUpgj6{?fb7v5XL?UBji^<3!lg5<@7P3-PNAw0E$(O4!iziUPplM5@sya-GBP?L0DCnxoS#2`&iq!bRjFi*I(QGl+lTJwHJ>C$Avc`;pGL|fZYTJqgY377TS1M_w% zyZig7SSm-QDx*wvbkR*sZ0XX*)mp~$mR&Pu^;*?Ao0*xJ?~1GF-PITR&%hO&0RaK= z6zDdqAUZD3Gf2qS`4yZjif!}G@Jhu&eR*Yx@DUTHEgQB_?skGwoc~&zeu>d)&8LTK z1|=Siai-#P2Vp~>{~8Mon$LwL;X>jZ5p@3`sVL2nrv8VAOg(?EMNqKfQ=~6?xW$s+ za4X(O_*aubu=5|?AwBV#qrx(>KOHyrzgs6p2M!&yn0b=~s>A8(Ym*vXS3zz1x2Toa zbf_B5-%bheMlTR!fMprT|3Pb;ZA{aW!F_CN>k1Kr^PL0bd*+S2d-=h;c`XcQja(3B z@X!9uZ^}0#g~JU=yX$+X_96mMinx{O{>)>*8)7AkMSxx=3}C%RaMnVsb1-PSye8_Z z$Un^eLbVWGcoTtbSQk8Re%Vn#S^H9?sAk8EPL6ze=_{IXO8 zb^Ju1X|e%s^>>5tz{$OmDAi=I7Qo4^UiEWQi@`O%bUtd&k;umJrgSB5E<=50B za$dejx@8}kp5v>5AjTz%@D>=*n+WKuGp9knAhWohl6#5eatJ9^-NbbR_ zn@j;JT-M~*MyCRSq9>rP&{v2=X24X%G??n%2=B#hMZ7Vs^^ftr28;ScY_e44AIE#= zAAJ)$nMNKiZ`sgf?9UH}SdY%8fHI|W6R)SL^ef{G3v+hHu7>N zG-|trT4|{#)0?7*RP-=#@sRL>ntiJ^Gj;1X0b0jXnIf2^n?HiMzYNk}%x%xl$Q05< z<{{HOK03wrHUI|ybedD@>FJrwHWLEI7y0KFzdDY@ySr`Pg`2q(%6qB~$wyu9PgI&WW2e)BIkla}N4?Ir=JU68=eC4#N&^`Sq{dI~nMd;x zw^I5hPV$(r5n5dn@=N(|Gc;M1nXh3bm$z*Ql5Rk(T9&j!28*xO9?$e(BQB{oSoYso zyx}411Df^nSF~OWpi!ILtJkZQ4~h9sdP_*hDsR(4@YG<=_r#TtVT1fWWuBCZsPhJ` zL*ogW@eH&Z*Z9`AckQr_b$7K_y#GMyj%Ba>+5=i||3&+>&&BTaN!2?h)B$)8*)r4Y zP;DU)6EStuYOma?_Q53o@2p;f^i|6BPwgBD);uueQGX)eI+<=u-JK8^wA=aIbHxt* z>E87FHZP}d9A2W7XF^1GRFzJ@>s;kya!vy4zIe!;-aPJ^I z;&RvZCa9)haMZo#0FLLtTxYfpu!scY;I1~J*?q!aO}Sb9o=BC+FL?yl#^Lgm!fC}7 zxafB*UwVSfqxFXEM}aH$W(5V9n8*WPSKdD)p^xFq29$#$Hp??GJDrR&MKlAYGDNeM zAWKi)x@1X1L%T1jvT8&9go>_bUyi1ll_eE!s(Rtki;w>w4a}gv4kX&+4~|3v66JS8 znyDUjNx%d&gMpW{V8v2Y&a@PEJB&+5Qo6S!gx{}LP>P7-JG+Zdnm2E}b)%RH09po< zLmP~m{c%>6)aA_m^;E&ExrpMgD8B^9aSu7R(6u}YiFd;v(3%=)gEfcaMa~OEiY6S+ z*201avU&*@5n&`V&`?@WJUW&z`TQc8>5%!3hrc{*Ww@1~>vZWv=334Vu<75~LC~GO z29*DdXm&Ip9@!E~+4E~=9k|=+uisWzB5d(-Be0e##>9Fqoi-OGbDw|f0yRsA`%fGF z;cHQ9_Afdhlsd)v*54TUcknQz_=k}5>V@?r>Y7koAHjZIZ~wcE3&4vvOeOlQ|L9jFCejV z=SmXMrEcMwisy6wQ*IqK^21sK^a|cH#xcjR{9biRbXJZY3`slEh%K1)X!`s(tBsj) zc68bzKMl$bbtNE`XB?0u6x@m2KUnU-cV1%+H7@d+0H`BWC%@Ma{9OwW3&L7dZ@n{F zQl42Hz(s9kD*4ekvD)v$Eq=XBg_dKR^de774W^je z7JVT;DGRNCs^SYEL`s2};zx!6oLyxO3q=G%RwY^fzKm;2WeLMLVLqVU;Z>yxyj6LA zf_HWCBU|Ao1po2M_%S60LutdM?^2Ab^O~3?fDX9}fmM-H)@uTpK6){XXY=qj2DmMh zvefv8lp{@_6U1G?SZqsZWnbB--k?s4liYB1TnHt}q&K510o*>s;&1Qb%x7<8`MkR@ zw~F>U=)``;KeC}Gl9Y;Ab#$kY>v~g|(}<8Z9=K{XXw|4+i&e7XLQd>D(=~e7XSF>1 z1ZW?*y?}ez7)kH8i$P2VZhEg9 z(VxHA3xBX!NDAzSK-<-$Hk0@k?Zy<*Hnl0(v`>t6k?ZPbGUnCtI>DXHhu6taVF0FPwrywtUtwLR`r^1ZI<^ zI%Z1_=R*1E(b~ie>w{t0RIN-Sa#a3Ag0~AF?s0 zUimUNx6({Jx7cM)Xn)GCNS$`)w|qDTB!J~cF)8~$h6c~;=Kb7Xsz?oE)3Sszr<1y( z<@ND8{9$nyVluzPw~&s^{`&5Q8oB`1_0r?vdf|S<_{4V@*3y9=@pK_N9V2-*oe3~w zEl@3sT(MK4$DbQUwZG7uGF1;~z)j}= zNNZ`JZ^j}m`Qx`+7Ph@oEo1M?LpppU?c<~N(k>d&m3xs#wU4MdL&%}@_wST?jmse5 zvQcfu{c4j3dnaB=FZX$U0X)X<%*pGHM*J^z!-!Ij25u}%^ z%lIGyEDI+8MG3ds{ds08&01e1VUNC|)xwL3Y5F&__>yjyGZ7PYnoSk#uz^(#lAGl) z24?GQ^Ksk*S2(l*Y8{1h)|C3+jFk>HmfLMZ!S&snq5KcbCgMBOX$vMHM%$5BOvz}X zt4%USZ#6;JC*wbqEya5ER7~bdTi~P_BShRM)L%w2E))Iy!Oq8wo&Wew6n|o5kY- z+8xr3eLt=QMo2!@wjl;96`SK~a!-wWY7<@!#O;4xCsZAZ$ntvGi;z3*J+Bo@ik24KZZ6QJR7z9ue z;|89@3zLWFHxqnF`YCi);gSE)xI^PyH6w=Oc2}6WaNXH3x<6hUrYSK)?0bQXE7CBK z2^=?Q_d&7Wr?Xz;+LJyOKo3Ik+cfZAIEkO}ow zrVSsfTih>9iGduqOcp{P3Dh5s;_wY1OODsJYRAd4fYz1M3AZRg56=bG8e!b~1V)En zkQj9qptfvY?sUhW2(Nk};r_-4lxuU-73AdPf&r90rggDNsr=&NpV(_b%t1bs8&;uTC53^(HXnLRT_gsV`LoMw+2wKA9&!oN#%9 z*lVYcKT`V{uh<#!KCyQWo*BJ}*sG*zhU|@NL@oSO=EQsIzOkrog8Kfn?=^n=Mz;h~ zI)YdSsSHnkw<&xZ6ZwMlbt(5jwP@ST)lHXj36xW35lSv4d2VD-?wJL0Hr&h)_BX6e zZlz7HaGp^Lcu7h}tG3)6l~lFc_TaXZFGiC1Ynz$g9SEcEgCG=X>GZQOS=#f3eKd z5n?h&`!jvo6uTV}u7s{>>P`4J>t6M?7ZC>OP|^29w%VC%s0{at)^$B3E4*-!o3@H! zZB|t6>s<{Q$gjqb=JYGbAM%mk4W4(&7}nbr1?t zNi=hAKf55x=6C7o?ft3Eg9a!v@AQ=2{U{FDx%i3X!a%?&TGLLIK#WGm9noEcE-i2R9-ttZsk7`z?XBavjHj`HyEu(Q9f&&U@;GI--0 zBjwEUAC%~41JP9KW&7b;J7**BEB-Kn*VU5c>fvNNCWvbEP5J4$@`YYtaXI;3iFa(2 za6eIL{9wlU^HW)r-c&G0vgA0{&5}U8Du)jbe(L1!T}`g`cCIMt2i_y7d~4m^SoD(- z9tpVFnP6SosBGa!JmI^Js88B^)9%kd&dN`YS^}L3COl?YFCF%2R%V@1X(I zat^e2{T8=LTl#zSAdO4V?KU0#@sl!cELvNy7 zX>HbSzJqScfqH2z0o#A=rYyd1T#|C-t`+8W0E7pYiE zia{YJWG2(W6cjiRbZ_HBIYn{eEvjMYf(?)A1YW0lBwJYx${>hIuFnoU-J#8V*CLB7bq2_U!6`S=@ zb1Tm*l4ot3f#JHIKD^Gf*?22(@)p+QNcLu~ebC7_?{clnC&W;M#VqES^|dCwa`h zb=>HJ%+1ZEOOY%>0ATzLg(^MilZ+gFsw1>+p#t^agF(EeD^2vm+bNJVoa!#dU0;4b zEHmm!&W&8V91FQ7w%;n=~fTSaEV(8 z)mR(JC83-gKT}Kx4kAZar#tg5OEsVAS1gRPXKMinN{YfX{>Hm6BRxKyL}`{u&KM81 z!NAKZY1G2UKqtTUdwVVP#C^7e1M4k!8+*))l1sr!XCLzI7A);uGpQrq_Db ztXaQN?aThQCJg}zyDenIx#1Dn-BIb^p{(qC{kR!tOagCNy3o!Q-Piv1FLM{#YO`Aw zNpI#5V16F$DZl)pE1878yiPd45>(lX1~%o zvf*Jx^z-Vzwq(SfA0JCdOHZJ)J6Tx~baZsI;QW_yfbQ<@&Mzu5|A1MMEh#Jv`+iHI zpHLl-%)`TjhK9D`z#;Jr+WkDBodhZG*X^B^MflG1<;!@bu&vjKnAA1 z%Das+FwkjSmawQ|dZ&C&PAuNZsZj1hBO5K_xA%J5VV1P_Y*qMufOkitQa-20+puUc zFh7r}t*t#~#7G+V$*()R>u*qbhlJ) zn;~&J1BX~bRFsH?C2?g%K(FLL_(e*Y=( zSE?jwxbSueGbY-ZM&D?I}-;5Y#UrDbkturJctqhFB-=Y;1+!1ezrLPHd zWdF~Y*E64O3%;oRcbC0)Hsc8RsrCLt!@U2Tfu7>aKiz_Y8eE~~gYQfJOFGpmHf(q! zrvGdWlx7p*Jd}6N&ezZH^5%y6b{l;({<*D<>zxCoaMh@be!+Gk8c*NLK3K*-%(vB>$12py0~?Nd5hQ z$ah_9*HC%*D$&zl<27of*aj@(Qv%ym8Pys+BOR4(2*1lQRGw2hwjc9+BN<+se|o3x zdN%;53D}PnHmd9low}miSW7FnJWp>@9kofUZuQxQFG0y!M<2Sx5W|Aj-1|rP=i@|& zH-AJ_!mS8~3IB!tZ}Xmz1xZyRzv$~xIpS$VWUkm6G zY27Ljksgn-)egkpX|_h6N*qttPXq*C-Q2XlF2R!tS!6n30xS*(2esFvl3V>=ohpM= zmfbP3B1-aV;>*r5Q(CV-S$Wh%$KYyhD>3@lwS~O~<_R*xw>g0`+S_K2J@@cyv+UCb zh&y)6J$J>e;6Qfa%+>(oiBPE%z)1hrp_r%%b?PdIfA<~h56PsYV>djJ1q}7zc5!lZ zYwPZz`!}UMuTHId^gwIwf?rd34YJDwyb`!IJpUT)Acy(Nd9zByy5K_=aVcJR;$mm~*k&>RYRqX8)1# zy8a_|j54W|;H2qHoyob9USv8i6L%ox&68p(>gB-3K2F#X5biO^kfUleV-B zN$YgwOCOqF8;tODMkm(qPeNV{#x=T{u(UPn=Ed6SHiX&}9WOLm5*HBM$WN3Jt#!`N zpQ<2WEB!2%<6+Q!ZEXA_o<3ia7mr3nfF`B=8+A8=S;$@;(hH)9G52d=B_qVrRBJnt zK^x01m#ft?j$UHT$8q~E#%Zj&?Cch)Oy1q=BtD-Kd6s+~BS@>CPa_S^pHfX}$2Rw& zLv}cQi|29)^9GT!b-f|qQ*KYzkRs!9QkvzcVwR{so!3yAFVE!VnOXy7jgMP(12yhf z3TrQpjJE7XJ(TtNZ*c%S<@d5{ZuaGUM%x}Y3(*Fd7FQ&KGoWs7O9oo?VXv)wOEpB; z=DBt1<Cj+JuFP+s^1Ud`!Yj?R!|MxHz8(%E$&}$c z+}pTK_*~aKPp;e8vfmJLUTM7&jF?pqq8tyAMBj1i^e^P2rfGU^UP}xuZ$9Q(rTK8Z z8qiX>b>AwV8~njMtiyX_gCy+eQ_G4OduT$NPUG%{1zAbHz5{RbQrR!TgqKJ6SbiQ! zm#-u325db|wi|2%>+{jjaaP=UkuPkcU8vO3i*Orb%cIJ1MT}mjPFq*$AC!xwMj?r@ zSIbz|(j_I1L#9m{96q1FU`uGpP@iFexro)06pz2pJMZs6ExB=uxNIl>az|edjyX2A z3+n7E79UKLu!jCbscp1@yKYZ3n{==T?XuO+Gjq$etLDDPengcv zvO-+~6mizu!F>yQgF^*Aq*)p~&gIX4k;htOLT=UX4>^A6C%Jj38<#kX@t$rjpKM&= zrHOTko7bU-{6d(0v!$+a+T*g+;N_MTZ9e~Sma+=wF_vjG#S6yjX|p~j{yJaRe7o)S zJ%9Sx{mGqCh1*9--|I*IGU`eadr}Io;{GLdoT+W(xzG)CWIA5=R&^w%+Q3C`ur+o9 zLnvex0hi?mf#;SKNjLjXX;Jz5?`m*eRd4os_N)8MtV8>pIY~`gQ}QSvi5=T}@-u8P z<}BZCdQ%cO>fpE?c@1{szJAF8GTx6)q6@wP(CT}OBr!r@TgHgd$ge=jWK<9f;d#E}24r|UhjAwh!rB`*e zC{4#b?iYrvvfhpqRx;{u6Gswx-xz_XuN=VEa?f>4&a8TW+j2D!)WfdCodv z`@|9w?cq8ZV03@Su#M@`vV%8SnD2TV-p%y;>Ua44O0vev{AJ=WQAQ}Y!f9~e!-@=X z3NPY9ck*B^3ibVApe>cl0hO(qaj>f)Te-#x6XT7CIuuaEKPG)71T z7oc}I>H=@=>`J)r6OeVGW3b{C8m8&&u{7#JXIwJnpNyaa_Sy$qH$=bik6kQ>D5;iD1;S8Jfo%fhL0*}|{>m&eAF*yh zMq~D)Y{}>YQtjtC?O@I|wXf$bY;Bs_Cv&{Kp_gF|@}-OVl2~4}inWR0$3Mmo?1L`4 ztW9KE#+K=E6d4426@jYm49570U!E&}t)><%+&rB>zcyy^dam>9Mz|`UV$wou2|5z? zX2bF!9{dfIsl*)Az^Q62LSd%}*ldUGj%RCrKrU#$;U(zFUxRSM3DidP23qq7 z5|J&3#CshlZN(M!iRSxE9U;SiJ_UmU~e%VW}b*1J&-?|&$O(Lf~YAg+b zLpsC6Cg-cyvX*C$@{O1b8h{qQaC&*ZbK1F!%yV9UI0>O1D_ec~)80Xubo(|IZMDs2 z_Q8Rpjsr&E#5z0+U7%jNyhN5L!oHP^!Ahvl-i`1(hR5(lz>_HLT);2wHZi`VPf7?6 zsRc0<7`ZA87ZRSlCIYiaaEvA@fDr>cqdDVV6&x33<6gYM30*%`PadR7c)Y;O zPmNC>w7qZaERwo=p@m%?d^0fq4Wrpj^k@JpQNFttfRlBbo0-9{HiXe2#Iwh2T5f9E z){*$I96IfBFSAB&zISp)51;Whz07`W9P3c^3{ct-Vv$J~_GJyd_@EG(uqMQGy@FV| z+_F0H+M|(_hGGH8v@uP~lP?wZ>%-$>cJWRqz~G-+fLnzFresiy($1N20Q;sn|86QfZ)-f>>1!zneI4SkBA z9Zxrh?}5*($yAiXB^2y^cw$Pk=q-O^cb_9rraUxc?~|ba*3+OQphdk6G|t-M(N9n+Gq~2T<2;^H0QT$tp3fO z5X6t*Ya-VR%gh|&>H#ZX&8MnWdFx|(f1Hn)Wk1Eb?k&(;gdTaNZ+DOLNy~a zU5(0{ZziL_ZMLNSd%+z&Lz8iha4K9Jqd3Mrd2idP*7eEp2xiv{2<@!L&8bg)(B8se zci)NzLy_NDG-#b<@3^|8?E0s92ZJ;ye|1sB#*Du)d*6*T*Z0frv!*OSgOr5z6< zks`(2-CEq;9fG?%E$+oVP#lW8I|O%^;2PY64*z@Kee>4LTJvSTBc^JchKRie6OqziVJqwRGXgU-8}7)57slwl5rQqM4cYY&>GHXCDb6$2f{3{#yo!t` zb_K0@U)_-LrbD(W*$YHTVH=;Vd8!VPL4h+C2HfetWe%xtPiJi=gZ6??K9$`d;@Wd7 zF7Do9NvN4@UFI@DEp^Dg@yeR0>OhWX?~6zJ=oF`m1S=cMf>cH{N~Y;C>Z>#W+G<@nq5d?qNgyZw!EA#oGet3EtV-mvnD(nf3oc@tbPK6BX33)9C9HU zK@5vh=|iE7a6ZB+Bq*i!6!73okU=CGb54G^&>dLVx?(s~1@7}s&Sdu^ukAxQDaX~5 zXbS)RJyomQ-cu>Lsb;3<^Ocg1t`O5+!+2lY14n4B-gbdA`eJDTwTi0eeGYGETZhk` zgpYIfr7^SGcGf1C`(bW>jupyW6RKwL)DQK={aRe33Q+JCe)#RRm;7`iWGXz8eo!L3 zoUG^LeDHbYE`N%wC>3aqoxZwO_zJH~mKUzk)!j*NcIL;TfIy4eXta=~*;Ap}1(~Wm zoc9=f`kVbS>${^2w%?2+ov%GeCp zkYou`6uvnxTi6wPV?g6*3+sv}_43wii*?j?dP6Y2!e`usfI}fc#mVhCJ9__FA<(IS z_fqQ}b9Yf}?af`_Q@s`T7-$rr+5Wgn;f;Hp;#PAFqKQ)AFlHy{#I=_LP-!1Wgo8FHXuzD5)p!;gu;V;Yr6HxDP_pnm?Yh@FNBF-^X&cvYpw0*K@km!Zu32J#ShY}HmsNm z^YW9?bh!2ARl|sFav9G>68>hFkKy1&^4&|QzZxU4z z4{QUtzqX@ms22nSf5R=Mq~UzSPV-{(QHkK3S_!9fbUjT5UU5~ORPdI4|7TC*c!B89 z=r;~Vth>6O7UV;R%FSUZ0?didzCum$t558P~)^T(yalmLuvfiBkL%Tk%~ z)v)SPr$quu7}Bu-Yv}?$YYl=4h)KD!H(qsq`zX>&aW@WgiMO~jg&BiYSY$blC1qa= zB~FBC_j$(S)?XwN)W)$8iY3RyZ4Y%X+)`)c&%)NeH?U2noQy`j*&nHXw2wI!9NlCC zaW`0^!*em1t4&yr`c*2A-^__wZ_KwRY5;umi&j?*6i??NokI7n*f9S3H5K_htOeLN zg{DZH=0TX!?0=?kMq7|ZkCy||w366&+CTRveZ-u?rzPMLYGVuRS)Mk#7}osm06=KE z{=FUgSxhn)HoD}qpkEq{KA(_ulQlP?8{Mloy5A{Wg!e^R=fw}t_7~p|5ZElqCAP?= z_2_*h%z0Kg!$en@*Pzu|He&2{wvlvleSOB_P{sr-e0XeNCqypP|M+J@w4TzF-rJ^y zW-Bx$t?VWK(K(gNX(pR6eJOj^v`MK(bT35`_qJzEopjZ3>p{rH&I5tR_Jz^ukQD?>+YYv^QO(_Y_JHdZ>I+W=6lJ#rKv@gsxy5NGQkCDRQ zsTC?JqR?z*Hfgs!cIb?YumGf_$)m=m#VOQHC)v|8BII#K85Gb>Ls_{apz2UA+z-Wvbj&V6!Pnb#^ z%_i9V0j}d1inJMfZ(a{IFh39CqE5GMsIKVhtflVl*lTqkW%We|q=1j=fc*GoYi-LQ zOx8xEA%hdlfzE8(D<8Gc$VRv?PlwMV#3uXMVa`gOAyAtJ_1&0^#?V) zWbz7!o3nvOoW{HAksu2;w*AqP2pXNQbJsZ`Ox6LlY6mc^+!1Y#mYmQFz2NfQ`Sv~yMGxJGQ}AjG zcP__VRyO3(Ic#5JdAaNN74XO`uGh!cGpa^4AEof4^36RIP+)t>=etMS{C!^+gGCyy z*fntQedG*&ie9Hnd4EGhWMtrxn9aNW<$Qa+&4lyX7d|ifDl?)chspWz&+W`8gck1B zUm&$)1@pq5_ZQ2W&(v|63f5CxYQ=lSG`r^`m?XwdZdQJ5>JK}R>G_{$!q*$8%7&`)V z6ZF*dRo8tCU@2|>T#ox>wARb+^8U;%eZ0TQV>xs<< zeZulRkvmGfEUZemSo7dY2o3#0OUVf2PT~2Aux~2)bmF7h^s(O-OHy8ZUoYlzcSL1c zt-JWh!DZO<)2U(~5j^%ZV~8_PEje~Nbj++-;b4pX?3sWuEqGrnRj<_etMQ3DJ$J@W zb(M_5`pb>2?U+c!JqYdTTRv|ZaQv>6Q3sQ}G=@>+PseKf24}q*nEN$0E3_dC!=XGc zc;4UZJv_C}c9Uvx+2FZI>0g@o1X^OfcYHafD+xE}8A(%Of(<_&3-H&%(Jx&suIWl+ z_x64W|5H>GW5nwnbdHQIHx&Rn%|W`T8PxNw6#J{f3^54{t4cmg9DVjYMQ~Z=RWH(#jz`Qex*#a1!KEy;eDM*3w6csTR3A?`h@fqbC)fR^YfZfy!x`wD z_9#aCgOmeuj68Uc{$(wjTR(IC2HaIUTu@(f6g-GcVo4N zO9{c6-$bi!EeTz~zi<}s&@4oS1Se6oP+mn}1)1Ka$l?Yzn|il)=Fc2XzL(M-CqgWA6`FuDWN0uEsb5*%@zHA2iHUy;T27ZPp1bb^21oY0sd2^PY6)TAexG1|y99CsiD<&(J z>bvV+(x2?fMUigZLzHxEeNa#B6`r8{cf+aLCr)31&2ih*mM!N?BA`g6pEYx`JqTW8 z*I}}+N`M|ex&zs-A4|GzDQ;M1z`>`V3z=z_aNRj0wVf;IL(+=!u+pZI@>=3OZUS5j z2A)H(khPI}_LTl-!tX2s zlz(9m6U6jA_g@>;@)%sx>!5)tiuOy0!Dwf=*mkDAcyWJtdLvknChIN2Cy25^VCZe{M7Myf_U#I*4g)@0m(8q1S5y4j$B(MY)N} zAn&`EZE6OvMGD^k(nD1{SS*Jva4GqC7Dmy#6=fa(A*zr5MIQZlXE^jSM3~B%MRe0S z;JaxE*1-#0hpc(z$&1Y9>t#x-PC<1@>e6|J?{oCaqG@Da1=uXz7o41G{mn|bCk$Vk zzg^q9M_HM*JE$s$aXSUf;9%c*jg|ZfjzaW1nD29N8h+xQ-J z=%gI>z8H@jRGuFwMc0ivYc&Z)A7wKEq+6xg{rba+yir+N*FEewIcp(}dh^#4WeYNO zp7$4D*bz2Dok2^2Zq(~y$yAnCB$=a}#w=N=l)BYLF|p>1a23sikX0@HI)*RGU23tr z%E;chj76q%&h&f^m~Z>~-dEFSd=gKG7ZS-;uE1^;Miq`5raeN1-uaH7H{A)}$8U;H zD>&PgKrIS4KdtR7Y(}QNO~1rkYl)m^p4Khtrn3UQFv!L~$S<6g`YGf_SABq7}Rpok5bQc4@=MY;(~3I#4fU36Z!- z^DCz*j8FxONvkCf?$wG(GyceF61>Omn}V6bL+&mTir?GP6#FfshZ`-?MlHf8Kx|vW z%>Rp9W~*$SxJ=ej0nj39CfBT&Dk8Ey>SYeUFv8n;;N_CTeu42P;~w`4r=98c%V*Vf z4GSR;UK2L1TgDlvEiDg3in}@v+h07s)73z*;wW71z^ZV9TQUx<)5L0`bY})=vaQhC zy!ZH?9G)vDOpiV^g>g4kFbueKE2`#@zjrIkkPV86I9iW2WeXLGlw=%3jR>4?!TB~DUn%_;eu>6O?u z)naq~;QZ1Nqwx%u)tua}1~BoX+1OZ3J8OQCZln){`pDX>c?9`z8zuz)yYJV+6u-Z`-c>erqr(lU~j%1FD~wv@gm=v`nKiTCrM zT=#>1!}RCwr-!tXYlV!5JimSY;)iU%a6ax z)J>Bf_sqPiF5jR#R(@IpBXlrcO7nYe?ZBCfH8An;Ipxbvdv(gXKQtkO;bFReg%a|> zn1w<`-it!gxf<-~Gz{T%PfB4h!qI`bQOOyR8FxRWed zeSvw=1NZMg*^g?Gd=M+0_8t99z(?I&Dsg5vQ~4yZu%T-CwRRsh?|T+2H)V+9c^3=g z7er-fyH8Y^LB54tgu)5;y-%J3A!5jBRHZK_0`{5@K71?%L-lDs}s+@bFuk_ zcg6&t>L0@gBX#-Ai2|$ai{1mbL^sK;gVlaQ`SOdNJ1K)YA4h8v6{4JKZWY$6M_qOg z#fnV$cMX;WB@faa5`3|(1KdT%L_liBTjdx_H8tNOYs@!z@cica?p9b)it^No|04NA zBJEa?G^_uueSZqyi6tilEkq+Rfbk|#Qtc=Q8>JQz8^KC5Z5ZS}N)<{Oh=5>;%`{YL%*OS;Do28x{?cT*o>w!fggY$^lL(wA)Qs{R0QpYER zg3Q+>aMqq9j{=SMowZC!;@aq1Qbf5orgmbSVab~fRMMb-e~`KBhXs@zEo3>a=d)Sv z0p?>Klam_8@6~KU61@+CcLrq?{}uC}H1~IHcxQmW=Kpta>*Kf0V9j0={c25OD84Wt<@8my`x`up?fBkk6{8?FWclp~|&<@hY z=w7UUI{gst|85P@Fj)*{?Ej~`A^zjP!nee#nOA#atO@9;EwIEeeuNbb=9XG0mS{}& ztn{0mMt>F!!%g`p8~jf*_aSYj$l$mc)V>#fud0^x1AU7|`5ks}r-|*L;+V@te9l{s zgyAOqcWfi~GLB60pnLaW9b6_U__rD|Q6i)w{!`O1++Y1=IdKT~qECFgV`Mt|Z@uE- zDAlDJwjWiR&tf^iqpnIbbXw|3Arj;dIll{>)?;^yJ-QA-`B&(nDSHHdA56yLK zAEjP#42>}J#N5(ifK=#yw8$6v%b*U6FBN14JP17kT)PY_um6Z-N`Iv=^dk4bUq{Uz zLT|m(YHEUktu4=tSQiR(6tde;)cml+BwN6Hc6E^xf)zKX#$<0!&lsmQ9R>sblEzc8{l%_^lh8+ic24G1kIo*pgCA0gc=yK2wqNp$T2qFZ*T}9R3PQm>z{ zm@{;-N}q1wCij$#pYf~NW>%R#ho`;c_`9vfZrn=$RRZsn?QW7c*nWR13%}E~cfo1@ zOl2O0!eJE5=VxwmT{sf&>2fvCKi~YmIO3;CUhDZ1iP%A>2|q)fd&j!Xm*IFW?~BOY zcJJkzl(?l^_Vvd5-@6y?;IEw>#bMGr_U`yz7L~hCib=D+yvs zjVoSRwtlP9^SLUWu?Vybm^wio;Aco=a*qaT4rubkqCIO0yq4C|miY%}-l$mUx7J^E zR%;I!WF3i%F{YGV`Sw0}!f>M}p5ATONtPNj39QJivDkBbOppSohI11Z&&4diYUjYU z$H^B8;H+%!3X^0Yai{MeN0VXB73Nh*(E~UG48C4>!Gh7|&N~}rYOlTvg>VdT+|hVm{%_uzg5$b@Z}E*zUxpadw09@cS;~;fZ_!F!X6}A8nvY!*-0TR-?*z z;GHyx$#iGoOLIlQH(R>A$1&3C>30AIo#8SOwQ#8C*KN0NT0;KLLMq-bvDpcT%A>cZ z<2L+XLVU)5jhI1w29*5_uCe82cYRn~$QNpMdp88ScaT1HiG}aQb{o{rO{Ux9R2Uv! zqtSRdk&#mpHKN}9@&|M+Vq!rL%;!&5hnBSjs@Go`44w$xyqW#C4AswXjP~I^bGqV@ z-*Hbl4*RInr+4W<-C2&d#eyt}+x4y?0$E9P8i41r`)v)l1(*Ij2MMvgzCvOot**oNVbIyi`wp z(a_sCT!Cn706>|Xt z5@gbLXQksEzvrUqZA`p|TUv;TT>jEgiTYn=Hmkk704rbHgF9u5u^%%>{cMm zOeaIXgmJeTfdylm?4!DULi8lH4`5#L+bHL(3rIx`HCfgUvfO+W$PIRM6zCo@rGzuu znU(7iPL^qVHVl(}Q&3QQ;lZwgN{McD)|YwF30 zvVuDL22oo1Lv~rCA14tCYH-$lt{BTQ-%^h8Ru}ja9EtRvG4BE4v4w6U-uB2|dn2J~ zezlFs$L;SkVH1iDv^$b#UB+!1gW7G|<9ej$m6F4%4!gA)y2S~1<1`WI-s2ZXN_@#a zs+pywaTd1vUhp{EL}-xxbqmGc+5QnG25rFu!FH^WoH#fJU(cf73_1JJoCTh}1eQ3l zgsRQmLkxV54p#*^S0|F$=>!AWbX<;<@?zPI@A^mhz*;!HcQt@3bh79iHCqRorQd+c z$AkuRbh-(Oc;cGW?26fo-0DSQ>+<7>mZIYP^!4m#OsOV!bKE1a&O+ zT#BfTVg8}RAnycOzaa(U3Nz&1D_LEUaJaNVi`hWf%C@GAR&x?2dh)j6J~4x7x*Ygz z_cZt5-XOubiK2jllkw>mwg`Q)h(fwgc|P+zXU6`7`s~kp{XB!S_PQ@n_(!2~;jVt> zc%qS+8hP|%`hXQ{_P^Xom9Yk7LfS7UNCEBD-n)niYASbY<7NWoL3gp}bvBe|9#)bu zGH~YZZF;(AC3H@_je=&%OOuTPY=j}TUaDCu*|jLUcS?{8iqr%Xf@kD$+*j(~KBnQZYRWg>L>?dPHh zEmiKSr=VSoAkUm}!zkg``15x}w`&lyH7?)={AeoZOWc}!mQ!1dr;Nfq3;Y2r)sQ4| zA{?cS;EzS$+O<(|K2U&LOw3!`5Y~nTR46)mhN8};uwLA&c`T&#oyaU2Z0G@QiUB9R z&27CwLg41Jb+1f8!hrcXSPcygh>$M7MaM>VG1_R8_b0-gRQ@bgXsgpn4>*)y)*y{r4-OM67kbGuXKq3-7uOeG&8vp+uH z{Rcd(tN!lDNuE7^lwqj_m~hX`5T(sE^6$eZnuht(K6B}Am*(b4a54S5Ca|h!lDUWZ zRd6W9mDuq17b%s9wOmv=(|RKa8o@SgN+MhvFAA4r)6!<9Uf=l8eW{^Ejq7 z1S}1V*o;F4l3$XVCOauG31^@_?XHD@;|mAyiglsf<2y!F>WM;ePoCzN4}p1Xr_C=! zUl`A!uHKaXiZNC1%~G}411%%T%h*J1i4Lv+@8DZJCaHOT#ujGh-bQ@#i{$_ik!G8} z2mb}4DPUu9pk?XYpIbIqI;ZEmC|CdS5QLS}MWl*%O>9D!njwkZWP9Pp#u2O9D@0bz zHa6PsX}I~AQF>wb3ide{`P`pJCOJq*U&=L6Wq0^~wlwR$$=%+o{goGWf1&1d5Cw70 zN=i?G!LFX~qWjD4vtj#kNC69WD*|TgV zWfKTUwzHh~W;C*y(O2T`k5a>fAgLgFe;&7(Y8W^EPjKZlk<9RO>Aev@$tWzWL#0ec zh~U8qv0`{J@AZ=W(AH5{?HKlW*2I%p56igTB~?xh+)Ubx?wA2$Cs|_#zir7hRx`f1 zMP`I=JzOF)q4C%}UA~FlL;cOCL=^f^JwTVk>n6MOERE&!`S)=-Onmvp=wFHS+b)l) zde79`_@Dx|1(IgHHnxM$-IpbN)QeaCvqL_eM@rH10@_N9SL=mXLXwYs4Gh z5{yl@p5nV*Y!~9Ya=4%eBfFXHMq}9}pjh-xm8d${>?eF+~{SqO_U0$3yTSW;_{&h`*^iEp}-^~ zp~ZpyN}-bwoht3(?)dp2N^D4_m9n7`Kn}j~O1GsxB!zP+kkMSXHwbORj?7!uUazE$ z4`J<9ZL2%6M17V5IYl4hd3f^`HEpPs@s=vq7)h!2cO44s4?&8~7tGfzma5RBwOuwN z&CDe`4Gec!9vtrKL<&E>=sF|H^|#FaW^fDgI2D*Nc)+R!V6ZW%Z#*lR?Z`hb#NE&# z*hr7coY{@vA7N3Sb_d;heyY`&)Ri(caUur$x4-M)6b82WDDnvjG07Q6SASk}c6@Td zrSgKV{Dh^iKvi`K+f&WNUv*x_oe9aAFDMDbG1q}~>Lj^tM9OjT6uh4x90(LE?C2dUHJ3TMs)!^9c9_WIvHD*BU^mO_7S4l? z|2EazflrL%KC{bd9W*rbOSoe@!A1C+dO@8lP0C$=zXrB7y&_LtV};nrP~^3s5xF2r z*Th*{5nL(oUizjusn+MSKw^=6+GHgTPfzcMkbmnUte=z1$gfs~w2c9505RM4!sZ4) zIY~73dJ5Qet>HDVP*4*~w)z_kAsGqe%B|EMTP1{DP;Gy>`i9hUiCOcILhZTDva58%p1vgE}_rJTswMKYK321zK z-|g(~{e;n_SIQ#b4Ngq_f#q$GS>tX&lvS;v93HKvMBA`6f^L3)Fw$u86l|rZKX-2} z&c*bRmioe+`^$+!g)U?rB@Y+iXOXRGPw3y{RChE(-yuva6Gu1ybG!8~pIyHmAsSX89jYVibVA&bw+{M7(iZV^dpGTx_iC4}!VwSkz%kju8E}y`RK&XLX@?a49hT!9DH=xvZxzIe);~X0%6ln*;lWDIyGTDdQ|r<%exCVdd*T;eXi)yo^y!)na(bb?D|#hq z%@pgo`Wvcn2Un{d?pDCIgK9Y08I@^f0;VdrODNwjn&q|-@6)2v{Lh|-NPG$s2~i9U zrVdx)aInNqb_D&Qlv#n?Rga%^aO^1c83OEuef9nOzCqoug<@a&0*o?-Ucp#!8Y8h0 zo8i+6CXC{f2QQff%+VE)r$@+H2IsqN4IyXIDu0%n!(F z9a?>v+G1Y;R5NgXBVAgUsVjN1>gV;py0?3bhOwnk>sI`a{5#!03Ph0Z z$-VU-4`eXpU5z)VF|Q9_EcO9YeYLhdvdX-&8w~a+M|vyP+IFYr%Rw`$guEHkyunyB z%Z-o;(el)$8nuRX4;Ir&Sl&DBv4*BX)ea-!j3S&fA7#*k;Y11kGgwdJZv77PRzJ{0HL1Wn8*EnvHE&WxE>-Lz_~=V(5(rO1 zWpL3`_FDWE#7I#Ml_g{Ik2Lps(~4pr@vT<1Wv3<(5NJu+iu{E}Lx(SZn-yhr+3^vz z)YC$#o!eh+SYZ?Xqd)T0!a)svI%Kzf@zQD!S?e9&m&;{~!4z8su{kg*uH9$bccx;e zDu$t89E=&(niliDo!PR_d0FiO+K-5J&?cJx(-!~IbWx86|BUaPwZ`VWu7U?5^-tRW zjB!7q*n><0I9z|L`s*T`-#FeubCL5^Q_tUd6Gt8gH8AUUtpzg(#gj;dE~QRH(*n%| zDq?Q)DX8L6wa_H(-o7^#nKgcDwdRi9TS>8ntTz2T+6iK`rR&An?JX%LuC=9pl((Q> z>O*=54{ts@>tScglra>ZuboMwD!DoOg#810vlFTxLfhYwYnR%$`jVNjQ|Qi_rn{Hr~%cYEO*`iiD=170*0mS_HiB|W2uC--lR6yarZ zpw8`5-kold@y2M>VJ6*5MhVB|F!4Zh&dqJFmUFmhOC%Rh<&tGIjt%OL{JM02A_0) zGUdR{d&sZh4yI9vJjlaxty0Jno(i`uOZS{nKfW^l`vIptY|)BqZ?{M&=Sl>nm}}om zKk(XmNTD39$oZmcW`^y`0N8YgFFL(B{?u_#UoM!GR3pki0W1O5mQpzT^au}RmJEh@ z8c0@M&p8aRv5k9hvSvl;#(=TwROwHKCe~mU=PJI6k(pS&nlO)X4md342wu1f z$$F)Sv+EFHKO?bJsY$uP?Se1}Cm3siPEf20@pWKuQM^c>Ytn}bH}^gYH3=d?ZE}h2 zTKiKgN?I23D`#;|+g9OT5`~$Qpm2gpDEjJ1c8S`RP9u7gK{`eMjF}euKuCv;#cu{p z#MCt+>Jkp8udL)NkM{8aDEgiSnfyuowv0b6$h5a&;1hBJtmL<@H<9v90W@;qtoL~1 z&eYCQo&R_iu50?=!ZiYa;>VYJQjw1dhv*5e64ndDg@aH(yRpI4c3*E`hOL2emmr_y z8o(ZW8t#Y(*rTzI3>SPLW(6PdzqV zF6!$L7zyWEUVia?N$9PEGUU4}2}=535D)scT+-;b>o;8VT$WenJZS}9UcfJEkYDf~ zdnyO7e8Fl^-6mQbyZ@afCpw|PpFgHNm8n8h= zeSz>z+WEX{NPkt>Zj@^mtujh9+_(-Bub=h{Y2jjCt0M;&Z?5g}C1(od z5Pa-8&2Jvl2xsHFGwy1QWIk%#`h5pc%2_Vg(VS47kV7w#TRz?zL+4Vw)3ZrU1>Q`Y zI)WOH(iwqN>aK+rq_B7K9XsI&H`&-R7>qz`4C<>a!JBRN%#YeS9LF@MQVaMiciJKPNw**-it4rGxR~A@((+*DA8*aco0~u3b=n z4L8dj5j~$y6EJan?mEglg>VJps3!YF1Vq(r!g30(Bxl$e8vM#D2g`n7qspkq_4~Kc z+w$wcbUfT>E90E6!@|1^Hnsot@MGb5@*)$;8wdbWTD-be0N&wK#^`Z4%;-lJMJF$e z@aFIl_0C%32LzwbD2>KUUE0Au@_jM+A_1l~8rkfutlA5~$HoMq>5^HDizVZ$C0=J7 zH&qn5gIF^Uw|c3*9iOFr^m?A=7$B6b@l(PU->YmDbzniG^jLA=nCqHTg;6RI$6yn# zcFfhW)VZ~ZF;rLO4VRLNdgO~c*n)Lx12|^un$z@I`?i=dfEsHG`XCI{1;=Km_cTxE zGXxch)WI2ul)`xoa#t~G%x5}Xi8Lyyytbs?@sWz-Bj{rFc^<-!)3Nl1M{Rn2DHURK zA2#W;Uj*3v^t;vv5I?HBQ5oHl&4+^9{J}1CGAL(Q4dXy7ftOjKFoxw+%3Tr$GP~yH zNXxvt4XBZjzPsF5>4g2<-FRg~euyKC3f0M$u@B)UUlE%v-R$vGgey(}Y1*4zdkixB z-!>*aiJ#4$pIoii(5p`+#C@LyS}Va{8C?W0DPNS#mEDe((x|O6^FdIK&dA+`MTn$LocA|zdp0ax;E+Kc@%y;9>8r;| zcQ1{)4@gi7nLmm6{50dCp4~c7f1pxE%#q^o+Q5E0#qZSWA#*TzjZ+?3CVb)F$Oc1qpM~6vizn$J= za9?CQ;m zrQn8;_-jp$1Zf%YmOtHD)_bFWKd-f(0rj^ITYR9GzGp-G-?_x9NvFd^>ibtZznZ|T73XRwvRv$kAR%*S#0=<1npcau$i@PKY>nRj zrEG6!Z$?$YfwU&!3#_VQQic!I_fzN*Y*j~DGGxEUvYPiNx1Qj@_069-B1ayi{TTgx z(5-2%ET_jk?oKP^Exq37fn|28foq1Ct7Jtl5^2!!Nq2VEDsdC?vk38B3y0BWqJbHp zPnbkIqkO-`d6x#Piio+i91KY@KDg4LhtFkWUb6b%m-RT>-88p!gbD0>+!PbZ z2-3>eXI!*#(EM6+?-#_zqHUxZiBN9P^0)6v%jo;_r%FLYyzx9I=A&%E_#n{yHa7pT zD5DlI0M3 zpUXSSDI_-A{EhJBb;OF#2~@SY79>Hlk~Bglyf4KR@{BQ;S6E*WNes$+;stydLDSnS zJUO|;RVrt@6rGnhU(O`Tlw!?Y#L9PhmcM>2`xe~bGtM_+eR0K9UJH2Hb+N;dV{fI3 z+6Rq+nv6$frrzF!h5q#>EMbSceleRvJ7jaP0)(ELXt6eJWx;a`WW}73kXvZ&dCQ^O zq;UhAR}`GTsr4`YMT{206VB$H$HzzGYt->K59xJKk1ch!|9Uh`2DbMOQf~##hTG&n||jT=k0|FLUb7YAy;@M<-G@wk1@}*Z`t^2>SS+ssf|^P(2!9_j3^(H*8~`hd zfQFNyTxywyjR=8p@UBO@j*u5S{iWpk2x&flxWoj>o9VCDgY|n2m_kw7Z=^x9_@a_8 zxUZ_q?%?0F;pBat7JfPHv(Zi`@pht@!S@Q6AR!{=`Xp1Ff8iHQALeq&AVfj;DZ?2Q zKmXw(QvVyD5XqY}WJ!O^@}6rrV_Y0VkFQGPXoFLiES1R4%Se>?vrqpj&b{E9x_nVQ31 zJK}i!ziFDSYD)@)|4Fv=NdK4Ef5{g86esn6x|QBgp=Ed!B&3AI{|TP(yZ)N_G6Z0RI+a`k(szm*@ZG zu1TW!cMwyAK~I!TtWwoG0{=3m=(bDrSxChWra zKZWuj)f`16So&iS=3msNWjccN4`jml#iz^##H?Nxj;i zb!=O=y}XO@Kb5n?z^@r1>kHHRi>>`f$a`X0n@GAmH=-fJs=fd)cc7>O&=CHKpQe9?F_Hi~aIr>??uKdgX#|d(*b(xWGRDKJaRcXMbt1RjzT$2IR8LEwq|5 zE{Y}7dcFU2e&a)6OrA#fgS0>y3#)dgH$l8IT$FEDj~&Iox4!h_CNLEuS0nrnaC9sL z{nEzi1AEq3GV#h=`IU6y3ek(P{549SY3|RX2D&nD4=qcYXWSCn(}m3bwpKR1%=Kvo zPeO2~nBvJ_m_$PV-%jgUdk{PmAQ#?J{RuK@0xAag3rXpd2QWLq%tiZmVi3_Nerp)> zNc*~V3&FtD$enxpj~CW(8f&uZ&%MbdnD;b2Zx*?;Fgv450$V4U>rh?wt;YWb?mJQ| zH6J!^kD27hF^OZp#LEy4dOiO zn`COh%(^qQl;H6w;b&4+UuUz$*;?c=dzbYk1Vg!!RPSNJP$BnHxHEE}vQt!2a*!bw z?quKbg%dQS`wC__^Q{(l-_xt&zpPu+=n&|wqf)<`f}B(=yy}0qAGbcd!cz1up}wT< zi?tckiU)9R4hI>qIBa{G>?uXDbQk}!WBjf+Z?F53|2~{fUg`yVI`p&X6YeSJcvH5Z z)%b3-&!v|)LC8F5ofcXM6C(LD>i)38>(Zzez}4fTu*2gyC4HuO!n{l%jRBVG{fzg= zO7(Wo-#oV*giGPP9U_l!wI_iKcn`JcBnC_Tx~mRP4#=1aha@%|LBMi86V3R<>;Pa~ z)p=79`JmPO<&03t5AfXrSGC!V)`0HRx!=S-H**~sCcks)?TlE%4gi#P+p=l(9~R)* z+~X=L;-@`7=;cp$l;fE3Xm)tMq?&KQLt5`#m@`Ex@fwTC3%Fk>#r5t6%t{o(Tr8LN z0cwY%!tK2N6xA+GMaiH)@c<=Jr13;B0ohUzRhioF&%9qx(1yU^lxvOUPSuBJKN1@p z%h&qpH)bDJ-fXU@+JCTp&9Wb|A%qemF=_1Pk!aMOE=*Ydc^VDP;{;<(BCTV_=**RR zpo7xmOqL4I`uegLY9Crf61iEzrxId67+?=b5PLE$3KP-oo} zvws3siglg*a=}mIPVvit{bKI}SerjPz|QyEv1B8BPB+RX$@Qo#DO+s%DQ3{{z0;L{ z8wcpWS7|GA>n2FV9r?FAFs2x{ zT?(4x9dF*s>T5V%<>Q3cp z*j&)<4+YKtWOr0H;&x2lSf8UOR7!@Mr;KowL7kasCv+@d<8FGvLqXA}ca`7&J7|MZ zx1m!&L8tARPyPk!A6VaMuGWbO~YmUwIgoq|i?mAUtK26j#b(Qev#JYzN(Xb0HUYi$g8 zQ-HqL;%* z8wY9>0P5_CW#8~&4 zY(Omg1-xMyi!-okR~}b-!EJW)t=me-o##$!Hg_cC+_Xi#*rXGCc7L=OtVatzvF6Vyh|gRva+)4y(m8ze{ZY{P->Pn#FFgbBeb?6`us9 z$t>lZ?q0OCR|IO_P%630QMor?mT;+OP(4`*;iM&t37i8Swtp#cBcG9mv3W z`D?C}HnU4~2(O2!+b`$F&X?oJrwJHME9XSb-Xd@uP4S`s8Q`^#+>%NBGsVw^&`KJY zMD(FWJe62RBNez~881*c?qh_({5FGF{{rAw{zu2E7Nle zarx=qt14fyLI{o~)xz(5LR2XzX9qC^Q|@m>4CZwCjX)ugQ12CMY@aYAD@7ea4BnjY zw_}c>2C9>n$z@qbS~;u|8d%qzJl07S++8tlCV(~KEN`*29jgdTvF}Lo70ZtCYuWen z4a&#Oo4kZAUjqjHF(j ze(dk$c3`@)vb=yumcr!^$w`W7vpID{sa6-49}sEpKW>K*08(j!)1-zzZOyLn+gFV0 zz6;~9G?J1XlytJ2Ly{oqT`as)7Wd|Io@WnzWgRlQz*8ZR_2JYz`v%yJ47}|mn(idN z8X@AX*8RKA(OxtW@x%lm~n1aJBnidroQNA0afj=N8T>!2GCWp zK$}IRAOI0}ywMfyNQcEZ_ZdKzU{_#pB|0^hSt#6iTHfW~wD(hgA@U|5d$2SsLoD<% zU!qf4JoB=}%%&!6Z)t6ck3+K7?GR0!=eL?vP@xiT>t14uSZd;Y*2Y--gD`!8$_Jxo zTy1!iQxY%WXVZQT3srW|n;^~eL#qQ(TvDm)XMcSJ-tl*=j>df0uKX!KIGUWnTAL-! z83;TSDR*qw$|(>fUbce+Smp7MI9^jH#FO@+;)w$xnI*Bh(^BQEiQA-tnBtvyzhi_i zl1{l6B|_xScon%XwQ8ApNnYqqIltdD4-1arSs52p->l==>fH!vNGnS*3Ge0+3BxS+L1~`+iWN2F{;}hC)O_Y2<%le%ZuB7wxv0sZtuW019#ew*k>16BV&Ewh3z81H+PBb zNz1t?qE@**)rwJ>{2UC(QAw(WX6M}3i> zlLgX3&^{Tc_A{usJ`%SQ-!Wiacx;ger~y{E=}&b>vV4EoAf z%w1N0NtgcUmA`FxtSo%qn^rS!fR7CX67_TfX{ zRMRGQSixGjxIIZkfdYA?vH%+xK;2t@X*d7uD9+^@M!HmM7qubW5N=uGFBlNHD*;o} z?aM5;$DeP4>YY_N5`V!54>k(O)NyWq#e;=nEiS6=s}GPQRKZ zs@#a6L{M`wO+c-FM3fh-& zrgJN=Dc0TGxNkedoSDGQ(`n6bsML!#Fe7z(8)lM@?%(yw%lXqs1}z>fsPpiSWRH4s z**hJFjj^Eqn@6U@ZB>P4gAG;*&)V-y?FF2GVjh<{yJD!Sr^DWhCY0xOt+nND?p5F* zz~qT6Q^0S5(q{yr+-IhHeYj0~_w@VpqaTl@N_jsLLXnyE%$6=1`MXbf*|u51&9A{u zIQ+P+D(;cNT(7j5k_|G0dtZYyz!vAElV2;P+j|-qnH?jv-R_pFSX-}`DdcxljoM-O`y;%BGr zX`%;et9Oq!DOZ)gA~!k@luuOm>_Qqk;U;%jKBvF$-)S73brDlk8>y|fT^(6`&-yj6 zVKzj#SZy@513QyquFCjy7NIbfekuAc(tm=FOl~;jm2$p_Axou{@$#G5v5aw)kJ;&5 z1u|WNqm05tXIv)mdm2L-jwB_K=JH!hpy zSH-C+E(ZOx_^=w!&M%#nm#r1kgO*q?Hm>Z+rdy9y7*w0!O3{%Bq(zIu&0I=KT|ko5 z=F<360Vn@6R)w&$#l@!knxFh%r@el4b)vuAUh*3pIBNAEWiN6xe$XrGydvL|E1K*z z1dS=7qoqZPQ|^C2db8uYdbfuiph4e1YmjrZ@QiLk#sBc(vdpHrbf7Jq-~7=pM{tRI zx_p21m(HqsQvmmxT;XynhbxOlf?a=<%Y`p`x72rb4+(xlEw<4rP1rDI3=3nUh~J%t ziEnK?cL%M=U7vCKnFl9IGF3w*$-vRB$l3E!hoZQs;$0<_5RiQ<7a0NyNtUgx z(`cp`tHcR)4FCRs=;;qq$C{8WTj(Y5b3O0*E%qA}3yT9%n<=wTOo&fp^luZRyx8Kn zGPPT|GE-5yKZa)_V`wc1o}(BaU-qv90)NEfp}wfACO4nSpQ*Se6OXfov8~Tnnh*(Y z3$sPT0^j7da`Dyc0eHE5adE=|W1Twc&iEJ}U?aanX zX;EqTj${t0yKc!?*PJbdvO-(MB9llPQ76}=-?~gbsU$8iV@w4>x$U9gfRv`{_a+AiAkBIS% z!VHI-;^xlSZHTfYtOiN$r}?g>=)O5l83yP6G##5Y?a1yxJVw0}ro$k^y(x93l=sa? zatox;v{l#vefu6Pw=@Gcqf48D7!PGEAY4H|v__WdK{Uu6sC+m-8 zbIuEYORLy295T;=3#^i&aI{rFZp7n!e-#H46Oe_SnI*enY4 zSS??DRz~{c&0xHVbx?U~SGRVIt^lJCog?#xq^D-Gd%`)|qk1K5J+Cq;7$t&%sPEjM ztbI$4(}%zp+^28FC*r|j5Y^sJaTUjJCq>&Wd5zVy*07eC9gL2Nj5%@RR z*wS|U8UR?sxf^_~s#?Hq(9m0DDx-`m&!7wLa8CX%cI!xa{Qw?npazo4j7_(i%H`m9 z{m~Io{^*Fh6f@x%+lHvr`=9tHy>3Tqopkh&I?$UOsicnCbX5a`Wy`ZM0O-Rf7s)-dkW@?;(Eh9S|ugZui>pfXk5_`<6Pow8RCCV83 zh6H=QtoZ}Op@JM1*_xd^d(XAN?DrTm^E-%bN5zC#>sk9T3DK4aWX@Z60iTtXw@qugW^yP+%3Xer zpGBj(cWr~T_6Rm%z5(bQt$`&H-TG9*HP5+%rU%S^yHFsjD&*EV6)54o3~*@cx#ke~ zUR2CsC53+a!QlEHvT@ZrY2JTpU>NIQCg)gwrLKDTvz?8W%jC*BKbUFL1mRuota_1t z^~#Kj?3&f;+1Jqu6KHViDRyMqpB%I#zYc@d%Sun0%Z;X%xclKE1e5S7E0XU z)tkDt5~b~->2cQv=pbfGo~bi~;O?ImS5S%8yr?`EXrE5#%p?Ln#5}EETOK7_*{$JJ zH(sx=wurdJkfiat@ew3ow>J17Lz_RW@EO!SHfmMVaB#$LeZ_3BMgo1;-rt2*&bdvu z0ir3+d3=mHrG=XVBNCqm3d|hDvo%b~gocyP_)>l_(GWvvsACO>*nVo;Qbp|Yo@`&k zmkl!!ZSNN}d7St=q8+ekv>k47o8u(9$%i_RM%5MgM{4KM=Yp%3-`Vl17n$s0XQe&R zbZ0BR778zg~{3xyTAzD0Oqtxoc*c>K)rS?7f#h;oMlU!Hhk zVyOCC8t&-NZJe}#1Skqb@nPF7`L0WxJbqP6SA_l%z?~H~N zJ67MFi3G1I2^CxO(C0Md>>1kcs>@SFsQ3W*O^{?B^QOlOn8qiREMrG0l1R+ ztBa*#B>2IT{U{Drk`Cv0Wy1xpMM&$*e=h9O$-c^wig=t2^MG38IvsvEeFPd?_>iiJ zd!g2)WEVioZ(iyvf|sw6G1H=aXbAL1qw{YvMI65gv$$fxQPc7f&!5#xqv_~XlH5ZC zThS3MEkisPi8(vla$>5WW<9dWHLLMP(jj#Z(&ZsSvPB;f*^aMZ42I>a+h8-l zxOuL^@S-Nz+tifWY-HIx@ih+04V5b@$rDfe4^4_O-L;UDyWGJ~59N>rA2$<-Bw+En zWx4d=*@cV@CV*f(#S#e_#YbFRj^kKjAM=x+Ds};{!RLNF5%0kiPCVbVDG+cMUb5|i zV@lqeg%W++Q;ds33wKNtTcim4WAN!(oC;kjp_VsnBhU?SI>e>uT=LniNz_8>wp2Q! zQjuyBl3jY7TKu2@cNi#s5|BC;n~LCBT-WLi1F&QRy4Epm-0bE3#88_n>Gqn%P~_6F z4-U3{hRl9>ob{~T=}{*KaB%!AFcRDmk@mZwy(Nk4})AVLbXQmi||oFjJPHVM$-_Ybo}<|LC|pC~S6p!`&f3?h-5b|3FO^-Tw1thSqGAm}rkhIRMGr)eDkg|O>H`j##x64< z8y;uNq0BH;3^CyFNN_5+uJiN}TRU|oYk@tOXziGqYUayG5n;07(O@{1kh1N*k1GHW z7a}KQV3-h4-2ezTnb?XLTlD>E*@;7A)7cl!;cyz%%xKsh<;TaG-`tJ?=rmi-68egO-5i8#6o%suoLq&)OC)tie^&iA zwvJ=bI5=&@>DGpH{VF$Qh`mv-7|QGy{jOg~HFb3_ckO01#ZKVd1}y+D-y@RuFPiX+ zWKQ>}eF!HaaGN^7fQfE{Samx2Ca(VmR-_%9aTN`?=)cFN%Y32PAf50`=CbYVPOZfj=bdeg73<2JqP>*MAB z0Ojfb0%b;S%sPvuQf4qJB{}qx_RlXW|AXjr(ES+kdUK(5Jj}05V#%2~m(;_hWbJ}M zy*WiQetYMbYo74v5ChQa_lQN-jl@xdK{}Nwr2je@#EpO1jpA5`87I8?2dr-d;z@sk z&d%Z|f~i~K0`os7|Garplg~XLM-=!NPbKlgx=}UqzgOT0T}r$dnm1{QelPq2pPh#| z2A<-~gqygK`=8Jde{X^{V9T4Bo-U)RihXo+^uk{>IiFtk1F3c~G(ICEve&4`xKF}FlB1~j>jxQMn1$e@_-Sjae6cm)@m6g-0D-Z;t zIHEnGjf*&EY+71sMNO?oBP2vpzo17C{e5r{4bNrA%F6n_7c`yI`o(J1$VyxsQ6dtz zcW`jKR&N+N7@<;A&fT4NFcL5EPq2`G>MD$ciHx*60W6k|r{WhBG`6|3Wxe7~m;dD| zSXW=qpxbnyhNqm!>3=(VW97;snUavuw>_H5ppyUG-rjzGejaz^f|icXWp9dRB$-JH z9xf#?D5_K>5kXBtMOs1O1>B|lytbHy@QonB-0Kp;oyIot*G=q_>TrA4Y z{P!tGPq7`ky4Vd)s)aj*$kSe~$g=?cje7&{{_r7*AS+dnIIcT<=i>hjrErV1G4{^% zvHuGyGoaR+(3zdrxWvAfB*9tm+HROu``I(1L77h<)FW>yaAHtwbCnP6+gwOveg}+n zS#;RqN_ZKQ$9)VOW9M;6X(cnTkg0&(x-fFLplB5T)GZ)fZh8J{3Oqrm$=}(a|s2CQgV<(*;A}y$zU*R4}#SWY2w^qV4A!&*&`$+IHATbsBRMRszyW zcGrwR^18C3n`ZC0i<~wP}%Q9l?Iz7d_*T69unjI9+ZkC zz(6<84|6HH(l}Y+jw9j=<+)=N) z!|IN5VJOo~-G|FyMY1~ryiYA(46n&|M`q-siTMy^MTVk8ZqGnZh&l6Ui7$u`p>XVY zco@^%%!km`%l%6vQft)Q(X!vlrl{y(GxKMsrAqZH%1h#@stF6*Hs^vb30LDgE4(qN zm2=3MIugY9Kx3dr+f7+RoIe#gpPZ;4H)+|%uZzT_oOZ6VFHSMT$6`9 zLqy8wPMRkp^0Ddu`j-jQf=E3U6&~aFL`IA}fF_GU+EFc3e^aHtmqd%vys5~G-_{@BiO zR~^9Xr6ES#-L@gA-wo78+&jp=LIB!=K@)egrAeV`=jpt=!ZZdCNe*2=NP7$>oULUa zTFttCV#^-jlgiq=0jqaqpT1!^?7uPZ6J!+&H1o8R@lUDCRv8yFFaAwo&h0SJUS5@- zyFFIM(J^_o$iwrklq4S<=xex!uA_L*osh#Kj)sQz$9YLq2Uqg;5IfmIrc1yAr{DU@ zd?ZK1ce|8tC*l7+KGpjqI)K3W7B=L^8ac}&q zSblZ46iHW5oOi=_mBZJ+JEpTl`?Z`ol`=1s56?P(?;$bA@L7-<>8yPi6DZ4r<)V%0 zwuA&SiCc3lb8hAZA|pZDCv15c>vVZf_Lv9>?0DMaB-WqQ3o83~^}CH0;Z ztrB|oWe<-OXZMo}E{o`w!n;%`owUNpt&=amSd=+btKMhA9UqIK|5Qne^V;hs3muTOVguFy#CcCYTbliA8qoA7{xo+W%Q;)?U85Q zs9l*&>8|fvI}%hr1`L(%GTW>?JSt>8-KQzx>w#4BX=nUc9qIdsN$UmtzCZ^N9Yx6@ zu;q2_ft;wD&I{qeGc`7F(7J*A{h2};!Naw@EYi>{Ss&q4Hs9~0o|Q_%4--_retB{=8=952x;jJZfCF;iP+mlY?n*x`H*+Ta4y#P2UgW5o(PN9Fs?K zhvdvJ1dl-CkPEbGycG8l*Y4B`ElhZtyt9>t`Z?4{x5HTT(+7%8u&4stEwFLf{I?fE56SAG*Im7F#S!-8*4LhPwd=6yN`v-hsljdv0zHV$&pn8x}%|N zTo8`T`c#`fI%xR${SfDDn{FN6L&lAZ*En!MIYn?n$a$|v;@L(YRZaOy$nMx@QYFM1CABZ9jT z3Q6yNybFnnPghbQNgfLXUT;p?zddxWW~gp+m~6C057W#Zd%Dcdqrni;i^PA?)9%dI zwb~W=IeGeeCRW{V8;@}z5S_rfQ~k*Mp;e-XWX!NRdNM|K{fme}vd(0#I&v-Dv2ZoC z(p+nC_o!L7!wjWtfpquy-NL+b@aJ+qP8V0y)vNV&mDDDljtCL|jI1!*)Pmz;%rl|( zY1#xbltk2nA0fYA(ZUlGo7WT7FGPcLX-gK~hc=*r3{==N6*-R!z+dmW2IIP)dMV0_ zCiMXQshdp+!(6UVIvX=S>{B}|@!efsRAeL&6+#EUEA~5Q4DNT<1od5j-ar>Hq52n5 zoPtL)=PzSSU{kMK>~HPlA3{X7%ag^=<1U}l2{yua96-|;pXFDP)zLPrgAgHSChz{W zNj!P+E^(>I=QTh2FkrZ?u^!{!nkuUHRa7Rp)Zi z#UTe>RYWo%_cZkBgS^ssrI(pi;(n6XG8a=@P+W|X(m;)rt!LuXEOdRseK+i7b2X)$vKR%^7Y^w>`KV(&D`pU&< z2?6a1)aTMsV%YnS!v^_#xD4l-3(d7|XUIpsSH);+mh!rZXZ|Sj`^ea~EL9=x)OYym za#u5*bd5Wkf;a1}=;-aMkIt_OFG7V#^@1%5Ne+>D$#1;N1hSF>gI+q_?VOJ1XJ^Te zKXmXW+rT~9z}&G+%|@}GR)v%)e4t>UQ(F0=*nAhQR^5kT%cqgdLT z&Npj9?x&s6*J56IH2iulFv&zTYliRA z#JgDzn!TonMQfKc&(bHUH7+I*`hD#WRV9r*DOh~2Rrbk;D`0|1l`}R$xruxR!>9mI zFfF3YJxzMX<>GU*e3lT>;!7s0@V2tJIv|X9deZG-zUJ@{ zVR(r~!P3aSo;m1R`YSSoKE>THMjvM7ae;pI<@dd>WbD$1bPgA7=!l>kx9=GfW~b-V zP@7Hr{cv@^jalN$*7E{|3_tk_v6O5^o0FMY)rjVRhm4d{tx2yNodxc93ilVO zv0aMoe{(f#&3nDM?^f)%1MB(2@+qzMwOqDxCd2HakHu1A=7zigmW}AKRjoc33+3!) zE`G2^Mj;|QP;eXqi@>SgI9HK;tc>I#z9;mce+lcmAN_GL?VkEU%rgw|fZ=fON^{Z} zE6ZD%JM;6?R1$~Qh?t^Jc!(nErdG#(qbzIaT_vnK7HHYQl^778$(<6bp9_r# zj{68~8w6WJ7=6pNLS4e(e~6_1mpK97b>&JN2ghY3QdswkcA-;sKlodJZXJat%1U|5 zu6kF!5`nBInHAJ`sz^>-(QK&Q44ACl&~8I80BeQkVgK8&>)ZCMpH0E2zp$ z_;NWS%E>6A*PEq@Pk00rzSCL)`1rv=Ica|NFu}1;Z`9Bk($Q0X=LuW*WX+6HNu$6G zDl^z5iAM*Ymi>opJT*7GoWjFS;W${Wf+x)3d;yzg=qs^_N zdH%F?D+<=<--kK-Wf+EMgBlRN;9ry>~U82Bt2E?!D0j`Ftv1sN-5jjdz8>@Jpc9no5mVa`noM&4EW{zxb^8 zhKIC`@T0gnwpRYD`n@Qb18LNqZREjYuh;|@-PeT8DGQlYV1%iHxs!g-^AUH?bnGuL zw@dGW`X*_7IG$@2G8&a@DNNL>z+wu0YPnWl_*95J@p+I`X)THQXF6 zQFP#dePLZ2iZ$fT;_VdOjU%*qXB93sn$D~n=_9hpf+>72tPCNelrPUU&E9Ti3!nk% zhp^JZVu3P39SMWQQ!Yz2$)tQDQY!>exu<8j(jb{vhGUYH>F+R#>LqqjWf-XYJ~oan z7aWG%qp)azrW!0shm#LS7cxwnL>+1e2iLF+4Gr(_J8IP`hTPCEdRb_4fspA0QT^|l8|xI&>7TWIFZO!3cys>ScK1<%sHdw6$#x?&Y|CNRX z*L?6CCT?0qe%O+HDE}eIeJC1oz8%K#S=!9j_BXpNXvx!#NNNsVxLP$=LH~oN$B2+R zf;8G@T+EU{vQs8PnZiqC1yXUx5vov+z3~=9w`nq=z`3A`k6LTy!F&3Ts0Eok#V*%B zncb}va2)Ui7$1UYY&n;*Wa1tJk~HJlwHy<_7kPFZ0UJ-R^Viqko< z70j@5XsN&IoSnw9lLxk*cHF?UIsgn${Rb7(+^SG)X(gWAw5gVi97!E(+FOgNCX(>Q z+K|Kc>gC7nx`*`hmA50bmBt6UE0)j_ycI)=iO94wOg-U{s_WWm9gPmAM00Mb>&>?r zUqh$bCfMk+yVs#dG}Yb3vX*A5Y(r|dA!V}@C8MfPww!Vzyb?8;hxcntJUlVe7wWE( z?B>NzJS;H{xFSYUjXP}l0ySAL?h0fS<53 z%&{W)D+M9H<4YL{@(%qSayzy!gZW$gpLuoZFMqc;Fu8}PBZ2B%@7_QL2C=f)KLz zK{cC2I2_Wg+7O>@R<9hj0uIUIO6k+a&GYl~>D7wlyF|-HYEyt$ptZ9OKO-&!2LndK zp~&cHYIgS5BqW)cMRb2<^Z}?KIg`43Y{8^<=8&^tbigAlEQ7};UvrIkozo@j*x(^c z|EWzXI))>>%B4G3|Fqbl!?ISJ-+2plLhR{Y?L7NP6C`_qz3StDRQu!&o-IV%+`Y_W z{zv;eFnru`&JQwquONtdBo{R`OF{~e-U7aG(Qeo$m66oFXKemC{zz8(yDEXxKjF9E z(PPowVr9pnXyubcyDy6*K0dyrvU0j_z$!>G2K0I`)(Zn>cFaMA!>O{+vY}Z-#m0Rs zAE(K4s$LP3nea$ZPkY}3Q3Dfl+b5MzCi@o%V%s{5T~|zE5KxsGbb1GZrfo6oBseL| z#Lp?SYfBVrRsC3y{K0~g&xhn(-A7=CI{u$Z|<-+jiXp?&sJ=CKl5hVr|-dz zH;G~-Eg7^l z9cKLL-$91#Ph+fL4_DVuhQ)Zz53LYxEY5Sh86$=;-Lw z8I$#r|2FAq()rbuWcds>2}zV=%)?d6hpawBw|CW{}4ZkNGluVeC!y9G&=qR?i`8uFZBqymh zskQ~MN(j)?BUMpRF*P?g=FP+~^VZkbhchQ;c;(o`oJcfR1NYL;<1L7%}j)R-t=YPAOa=;`X?lJg(_Yg;A`rndKDL?($a{RXw h|9tQN^KsP^QZH8C5)s|O literal 0 HcmV?d00001 diff --git a/docs/web-ui.md b/docs/web-ui.md index a15c114efc502..d6ee8eb707007 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -35,6 +35,64 @@ progress of all jobs and the overall event timeline. When you click on a job on page, you see the details page for that job. The details page further shows the event timeline, DAG visualization, and all stages of the job. +The information that is displayed in this section is +* User: Current Spark user +* Total uptime: Time since Spark application started +* Scheduling mode: See [job scheduling](job-scheduling.html#configuring-pool-properties) +* Number of jobs per status: Active, Completed, Failed + +

+ Basic info +

+ +* Event timeline: Displays in chronological order the events related to the executors (added, removed) and the jobs + +

+ Event timeline +

+ +* Details of jobs grouped by status: Displays detailed information of the jobs including Job ID, description (with a link to detailed job page), submitted time, duration, stages summary and tasks progress bar + +

+ Details of jobs grouped by status +

+ + +When you click on a specific job, you can see the detailed information of this job. + +### Jobs detail + +This page displays the details of a specific job identified by its job ID. +* Job Status: (running, succeeded, failed) +* Number of stages per status (active, pending, completed, skipped, failed) +* Associated SQL Query: Link to the sql tab for this job +* Event timeline: Displays in chronological order the events related to the executors (added, removed) and the stages of the job + +

+ Event timeline +

+ +* DAG visualization: Visual representation of the directed acyclic graph of this job where vertices represent the RDDs or DataFrames and the edges represent an operation to be applied on RDD. + +

+ DAG +

+ +* List of stages (grouped by state active, pending, completed, skipped, and failed) + * Stage ID + * Description of the stage + * Submitted timestamp + * Duration of the stage + * Tasks progress bar + * Input: Bytes read from storage in this stage + * Output: Bytes written in storage in this stage + * Shuffle read: Total shuffle bytes and records read, includes both data read locally and data read from remote executors + * Shuffle write: Bytes and records written to disk in order to be read by a shuffle in a future stage + +

+ DAG +

+ ## Stages Tab The Stages tab displays a summary page that shows the current state of all stages of all jobs in the Spark application, and, when you click on a stage, a details page for that stage. The details @@ -117,3 +175,4 @@ illustrate how Spark parses, analyzes, optimizes and performs the query. The web UI includes a Streaming tab if the application uses Spark streaming. This tab displays scheduling delay and processing time for each micro-batch in the data stream, which can be useful for troubleshooting the streaming application. + From ba5ee277069ca3d11b80b97bbb7235db0d0f8ff9 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 15 Aug 2019 10:21:26 -0500 Subject: [PATCH 073/149] [SPARK-28243][PYSPARK][ML][FOLLOW-UP] Move Python DecisionTreeParams to regression.py ## What changes were proposed in this pull request? Leave ```shared.py``` untouched. Move Python ```DecisionTreeParams``` to ```regression.py``` ## How was this patch tested? Use existing tests Closes #25406 from huaxingao/spark-28243. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 5 +- .../ml/param/_shared_params_code_gen.py | 41 ---------- python/pyspark/ml/param/shared.py | 54 -------------- python/pyspark/ml/regression.py | 74 +++++++++++++++++++ 4 files changed, 77 insertions(+), 97 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ce6543952bf6d..5a1202b0ba13c 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -22,8 +22,9 @@ from pyspark import since, keyword_only from pyspark.ml import Estimator, Model from pyspark.ml.param.shared import * -from pyspark.ml.regression import DecisionTreeModel, DecisionTreeRegressionModel, \ - GBTParams, HasVarianceImpurity, RandomForestParams, TreeEnsembleModel +from pyspark.ml.regression import DecisionTreeModel, DecisionTreeParams, \ + DecisionTreeRegressionModel, GBTParams, HasVarianceImpurity, RandomForestParams, \ + TreeEnsembleModel from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 1b0c8c5d28b78..ca2e4a0f5bde1 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -174,45 +174,4 @@ def get$Name(self): param_code = _gen_param_header(name, doc, defaultValueStr, typeConverter) code.append(param_code + "\n" + _gen_param_code(name, doc, defaultValueStr)) - decisionTreeParams = [ - ("maxDepth", "Maximum depth of the tree. (>= 0) E.g., depth 0 means 1 leaf node; " + - "depth 1 means 1 internal node + 2 leaf nodes.", "TypeConverters.toInt"), - ("maxBins", "Max number of bins for" + - " discretizing continuous features. Must be >=2 and >= number of categories for any" + - " categorical feature.", "TypeConverters.toInt"), - ("minInstancesPerNode", "Minimum number of instances each child must have after split. " + - "If a split causes the left or right child to have fewer than minInstancesPerNode, the " + - "split will be discarded as invalid. Should be >= 1.", "TypeConverters.toInt"), - ("minInfoGain", "Minimum information gain for a split to be considered at a tree node.", - "TypeConverters.toFloat"), - ("maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation. If too small," + - " then 1 node will be split per iteration, and its aggregates may exceed this size.", - "TypeConverters.toInt"), - ("cacheNodeIds", "If false, the algorithm will pass trees to executors to match " + - "instances with nodes. If true, the algorithm will cache node IDs for each instance. " + - "Caching can speed up training of deeper trees. Users can set how often should the " + - "cache be checkpointed or disable it by setting checkpointInterval.", - "TypeConverters.toBoolean")] - - decisionTreeCode = '''class DecisionTreeParams(Params): - """ - Mixin for Decision Tree parameters. - """ - - $dummyPlaceHolders - - def __init__(self): - super(DecisionTreeParams, self).__init__()''' - dtParamMethods = "" - dummyPlaceholders = "" - paramTemplate = """$name = Param($owner, "$name", "$doc", typeConverter=$typeConverterStr)""" - for name, doc, typeConverterStr in decisionTreeParams: - if typeConverterStr is None: - typeConverterStr = str(None) - variable = paramTemplate.replace("$name", name).replace("$doc", doc) \ - .replace("$typeConverterStr", typeConverterStr) - dummyPlaceholders += variable.replace("$owner", "Params._dummy()") + "\n " - dtParamMethods += _gen_param_code(name, doc, None) + "\n" - code.append(decisionTreeCode.replace("$dummyPlaceHolders", dummyPlaceholders) + "\n" + - dtParamMethods) print("\n\n\n".join(code)) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 56d6190723161..9527ef6f576b2 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -747,57 +747,3 @@ def getValidationIndicatorCol(self): Gets the value of validationIndicatorCol or its default value. """ return self.getOrDefault(self.validationIndicatorCol) - - -class DecisionTreeParams(Params): - """ - Mixin for Decision Tree parameters. - """ - - maxDepth = Param(Params._dummy(), "maxDepth", "Maximum depth of the tree. (>= 0) E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.", typeConverter=TypeConverters.toInt) - maxBins = Param(Params._dummy(), "maxBins", "Max number of bins for discretizing continuous features. Must be >=2 and >= number of categories for any categorical feature.", typeConverter=TypeConverters.toInt) - minInstancesPerNode = Param(Params._dummy(), "minInstancesPerNode", "Minimum number of instances each child must have after split. If a split causes the left or right child to have fewer than minInstancesPerNode, the split will be discarded as invalid. Should be >= 1.", typeConverter=TypeConverters.toInt) - minInfoGain = Param(Params._dummy(), "minInfoGain", "Minimum information gain for a split to be considered at a tree node.", typeConverter=TypeConverters.toFloat) - maxMemoryInMB = Param(Params._dummy(), "maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation. If too small, then 1 node will be split per iteration, and its aggregates may exceed this size.", typeConverter=TypeConverters.toInt) - cacheNodeIds = Param(Params._dummy(), "cacheNodeIds", "If false, the algorithm will pass trees to executors to match instances with nodes. If true, the algorithm will cache node IDs for each instance. Caching can speed up training of deeper trees. Users can set how often should the cache be checkpointed or disable it by setting checkpointInterval.", typeConverter=TypeConverters.toBoolean) - - - def __init__(self): - super(DecisionTreeParams, self).__init__() - - def getMaxDepth(self): - """ - Gets the value of maxDepth or its default value. - """ - return self.getOrDefault(self.maxDepth) - - def getMaxBins(self): - """ - Gets the value of maxBins or its default value. - """ - return self.getOrDefault(self.maxBins) - - def getMinInstancesPerNode(self): - """ - Gets the value of minInstancesPerNode or its default value. - """ - return self.getOrDefault(self.minInstancesPerNode) - - def getMinInfoGain(self): - """ - Gets the value of minInfoGain or its default value. - """ - return self.getOrDefault(self.minInfoGain) - - def getMaxMemoryInMB(self): - """ - Gets the value of maxMemoryInMB or its default value. - """ - return self.getOrDefault(self.maxMemoryInMB) - - def getCacheNodeIds(self): - """ - Gets the value of cacheNodeIds or its default value. - """ - return self.getOrDefault(self.cacheNodeIds) - diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 349130f22fade..aca9e6f53cd59 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -584,6 +584,80 @@ def predictions(self): return self._call_java("predictions") +class DecisionTreeParams(Params): + """ + Mixin for Decision Tree parameters. + """ + + maxDepth = Param(Params._dummy(), "maxDepth", "Maximum depth of the tree. (>= 0) E.g., " + + "depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.", + typeConverter=TypeConverters.toInt) + + maxBins = Param(Params._dummy(), "maxBins", "Max number of bins for discretizing continuous " + + "features. Must be >=2 and >= number of categories for any categorical " + + "feature.", typeConverter=TypeConverters.toInt) + + minInstancesPerNode = Param(Params._dummy(), "minInstancesPerNode", "Minimum number of " + + "instances each child must have after split. If a split causes " + + "the left or right child to have fewer than " + + "minInstancesPerNode, the split will be discarded as invalid. " + + "Should be >= 1.", typeConverter=TypeConverters.toInt) + + minInfoGain = Param(Params._dummy(), "minInfoGain", "Minimum information gain for a split " + + "to be considered at a tree node.", typeConverter=TypeConverters.toFloat) + + maxMemoryInMB = Param(Params._dummy(), "maxMemoryInMB", "Maximum memory in MB allocated to " + + "histogram aggregation. If too small, then 1 node will be split per " + + "iteration, and its aggregates may exceed this size.", + typeConverter=TypeConverters.toInt) + + cacheNodeIds = Param(Params._dummy(), "cacheNodeIds", "If false, the algorithm will pass " + + "trees to executors to match instances with nodes. If true, the " + + "algorithm will cache node IDs for each instance. Caching can speed " + + "up training of deeper trees. Users can set how often should the cache " + + "be checkpointed or disable it by setting checkpointInterval.", + typeConverter=TypeConverters.toBoolean) + + def __init__(self): + super(DecisionTreeParams, self).__init__() + + def getMaxDepth(self): + """ + Gets the value of maxDepth or its default value. + """ + return self.getOrDefault(self.maxDepth) + + def getMaxBins(self): + """ + Gets the value of maxBins or its default value. + """ + return self.getOrDefault(self.maxBins) + + def getMinInstancesPerNode(self): + """ + Gets the value of minInstancesPerNode or its default value. + """ + return self.getOrDefault(self.minInstancesPerNode) + + def getMinInfoGain(self): + """ + Gets the value of minInfoGain or its default value. + """ + return self.getOrDefault(self.minInfoGain) + + def getMaxMemoryInMB(self): + """ + Gets the value of maxMemoryInMB or its default value. + """ + return self.getOrDefault(self.maxMemoryInMB) + + def getCacheNodeIds(self): + """ + Gets the value of cacheNodeIds or its default value. + """ + return self.getOrDefault(self.cacheNodeIds) + + class TreeEnsembleParams(DecisionTreeParams): """ Mixin for Decision Tree-based ensemble algorithms parameters. From 2ac6163a5d04027ef4dbdf7d031cddf9415ed25e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 15 Aug 2019 09:38:31 -0700 Subject: [PATCH 074/149] [SPARK-23977][SQL] Support High Performance S3A committers [test-hadoop3.2] This patch adds the binding classes to enable spark to switch dataframe output to using the S3A zero-rename committers shipping in Hadoop 3.1+. It adds a source tree into the hadoop-cloud-storage module which only compiles with the hadoop-3.2 profile, and contains a binding for normal output and a specific bridge class for Parquet (as the parquet output format requires a subclass of `ParquetOutputCommitter`. Commit algorithms are a critical topic. There's no formal proof of correctness, but the algorithms are documented an analysed in [A Zero Rename Committer](https://github.com/steveloughran/zero-rename-committer/releases). This also reviews the classic v1 and v2 algorithms, IBM's swift committer and the one from EMRFS which they admit was based on the concepts implemented here. Test-wise * There's a public set of scala test suites [on github](https://github.com/hortonworks-spark/cloud-integration) * We have run integration tests against Spark on Yarn clusters. * This code has been shipping for ~12 months in HDP-3.x. Closes #24970 from steveloughran/cloud/SPARK-23977-s3a-committer. Authored-by: Steve Loughran Signed-off-by: Marcelo Vanzin --- .../internal/io/FileCommitProtocol.scala | 6 +- .../io/HadoopMapReduceCommitProtocol.scala | 54 +++++- docs/cloud-integration.md | 70 +++++++- hadoop-cloud/pom.xml | 39 ++++ .../cloud/BindingParquetOutputCommitter.scala | 122 +++++++++++++ .../io/cloud/PathOutputCommitProtocol.scala | 166 ++++++++++++++++++ .../io/cloud/CommitterBindingSuite.scala | 146 +++++++++++++++ .../io/cloud/StubPathOutputCommitter.scala | 120 +++++++++++++ .../src/test/resources/log4j.properties | 36 ++++ 9 files changed, 741 insertions(+), 18 deletions(-) create mode 100644 hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/BindingParquetOutputCommitter.scala create mode 100644 hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala create mode 100644 hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/CommitterBindingSuite.scala create mode 100644 hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/StubPathOutputCommitter.scala create mode 100644 hadoop-cloud/src/test/resources/log4j.properties diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index 854093851f5d0..0746e43babf9a 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.Utils * 3. When all necessary tasks completed successfully, the driver calls commitJob. If the job * failed to execute (e.g. too many failed tasks), the job should call abortJob. */ -abstract class FileCommitProtocol { +abstract class FileCommitProtocol extends Logging { import FileCommitProtocol._ /** @@ -129,7 +129,9 @@ abstract class FileCommitProtocol { * before the job has finished. These same task commit messages will be passed to commitJob() * if the entire job succeeds. */ - def onTaskCommit(taskCommit: TaskCommitMessage): Unit = {} + def onTaskCommit(taskCommit: TaskCommitMessage): Unit = { + logDebug(s"onTaskCommit($taskCommit)") + } } diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 7477e03bfaa76..11ce608f52ee2 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -17,6 +17,7 @@ package org.apache.spark.internal.io +import java.io.IOException import java.util.{Date, UUID} import scala.collection.mutable @@ -136,7 +137,7 @@ class HadoopMapReduceCommitProtocol( tmpOutputPath } - private def getFilename(taskContext: TaskAttemptContext, ext: String): String = { + protected def getFilename(taskContext: TaskAttemptContext, ext: String): String = { // The file name looks like part-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003-c000.parquet // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, // the file name is fine and won't overflow. @@ -205,11 +206,28 @@ class HadoopMapReduceCommitProtocol( } } + /** + * Abort the job; log and ignore any IO exception thrown. + * This is invariably invoked in an exception handler; raising + * an exception here will lose the root cause of the failure. + * + * @param jobContext job context + */ override def abortJob(jobContext: JobContext): Unit = { - committer.abortJob(jobContext, JobStatus.State.FAILED) - if (hasValidPath) { - val fs = stagingDir.getFileSystem(jobContext.getConfiguration) - fs.delete(stagingDir, true) + try { + committer.abortJob(jobContext, JobStatus.State.FAILED) + } catch { + case e: IOException => + logWarning(s"Exception while aborting ${jobContext.getJobID}", e) + } + try { + if (hasValidPath) { + val fs = stagingDir.getFileSystem(jobContext.getConfiguration) + fs.delete(stagingDir, true) + } + } catch { + case e: IOException => + logWarning(s"Exception while aborting ${jobContext.getJobID}", e) } } @@ -222,17 +240,35 @@ class HadoopMapReduceCommitProtocol( override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { val attemptId = taskContext.getTaskAttemptID + logTrace(s"Commit task ${attemptId}") SparkHadoopMapRedUtil.commitTask( committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId) new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet) } + /** + * Abort the task; log and ignore any failure thrown. + * This is invariably invoked in an exception handler; raising + * an exception here will lose the root cause of the failure. + * + * @param taskContext context + */ override def abortTask(taskContext: TaskAttemptContext): Unit = { - committer.abortTask(taskContext) + try { + committer.abortTask(taskContext) + } catch { + case e: IOException => + logWarning(s"Exception while aborting ${taskContext.getTaskAttemptID}", e) + } // best effort cleanup of other staged files - for ((src, _) <- addedAbsPathFiles) { - val tmp = new Path(src) - tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false) + try { + for ((src, _) <- addedAbsPathFiles) { + val tmp = new Path(src) + tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false) + } + } catch { + case e: IOException => + logWarning(s"Exception while aborting ${taskContext.getTaskAttemptID}", e) } } } diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index b64ffe55d8359..a8d40fe7456e4 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -125,7 +125,7 @@ consult the relevant documentation. ### Recommended settings for writing to object stores For object stores whose consistency model means that rename-based commits are safe -use the `FileOutputCommitter` v2 algorithm for performance: +use the `FileOutputCommitter` v2 algorithm for performance; v1 for safety. ``` spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 @@ -143,8 +143,30 @@ job failure: spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true ``` +The original v1 commit algorithm renames the output of successful tasks +to a job attempt directory, and then renames all the files in that directory +into the final destination during the job commit phase: + +``` +spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 1 +``` + +The slow performance of mimicked renames on Amazon S3 makes this algorithm +very, very slow. The recommended solution to this is switch to an S3 "Zero Rename" +committer (see below). + +For reference, here are the performance and safety characteristics of +different stores and connectors when renaming directories: + +| Store | Connector | Directory Rename Safety | Rename Performance | +|---------------|-----------|-------------------------|--------------------| +| Amazon S3 | s3a | Unsafe | O(data) | +| Azure Storage | wasb | Safe | O(files) | +| Azure Datalake Gen 2 | abfs | Safe | O(1) | +| Google GCS | gs | Safe | O(1) | + As storing temporary files can run up charges; delete -directories called `"_temporary"` on a regular basis to avoid this. +directories called `"_temporary"` on a regular basis. ### Parquet I/O Settings @@ -190,15 +212,49 @@ while they are still being written. Applications can write straight to the monit atomic `rename()` operation. Otherwise the checkpointing may be slow and potentially unreliable. +## Committing work into cloud storage safely and fast. + +As covered earlier, commit-by-rename is dangerous on any object store which +exhibits eventual consistency (example: S3), and often slower than classic +filesystem renames. + +Some object store connectors provide custom committers to commit tasks and +jobs without using rename. In versions of Spark built with Hadoop 3.1 or later, +the S3A connector for AWS S3 is such a committer. + +Instead of writing data to a temporary directory on the store for renaming, +these committers write the files to the final destination, but do not issue +the final POST command to make a large "multi-part" upload visible. Those +operations are postponed until the job commit itself. As a result, task and +job commit are much faster, and task failures do not affect the result. + +To switch to the S3A committers, use a version of Spark was built with Hadoop +3.1 or later, and switch the committers through the following options. + +``` +spark.hadoop.fs.s3a.committer.name directory +spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol +spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter +``` + +It has been tested with the most common formats supported by Spark. + +```python +mydataframe.write.format("parquet").save("s3a://bucket/destination") +``` + +More details on these committers can be found in the latest Hadoop documentation. + ## Further Reading Here is the documentation on the standard connectors both from Apache and the cloud providers. -* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+ -* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7 -* [Azure Data Lake](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8 -* [Amazon S3 via S3A and S3N](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+ +* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). +* [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). +* [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). +* [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html). +* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). * [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon * [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google - +* [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver) diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index dbf4b98d5fadb..31c729c501733 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -198,6 +198,45 @@ --> hadoop-3.2 + + src/hadoop-3/main/scala + src/hadoop-3/test/scala + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + ${extra.source.dir} + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + ${extra.testsource.dir} + + + + + + + ### What changes were proposed in this pull request? This PR proposes to improve the Github template for better and faster review iterations and better interactions between PR authors and reviewers. As suggested in the the [dev mailing list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-New-sections-in-Github-Pull-Request-description-template-td27527.html), this PR referred [Kubernates' PR template](https://raw.githubusercontent.com/kubernetes/kubernetes/master/.github/PULL_REQUEST_TEMPLATE.md). Therefore, those fields are newly added: ``` ### Why are the changes needed? ### Does this PR introduce any user-facing change? ``` and some comments were added. ### Why are the changes needed? Currently, many PR descriptions are poorly formatted, which causes some overheads between PR authors and reviewers. There are multiple problems by those poorly formatted PR descriptions: - Some PRs still write single line in PR description with 500+- code changes in a critical path. - Some PRs do not describe behaviour changes and reviewers need to find and document. - Some PRs are hard to review without outlines but they are not mentioned sometimes. - Spark is being old and sometimes we need to track the history deep. Due to poorly formatted PR description, sometimes it requires to read whole codes of whole commit histories to find the root cause of a bug. - Reviews take a while but the number of PR still grows. This PR targets to alleviate the problems and situation. ### Does this PR introduce any user-facing change? Yes, it changes the PR templates when PRs are open. This PR uses the template this PR proposes. ### How was this patch tested? Manually tested via Github preview feature. Closes #25310 from HyukjinKwon/SPARK-28578. Lead-authored-by: HyukjinKwon Co-authored-by: Hyukjin Kwon Signed-off-by: HyukjinKwon --- .github/PULL_REQUEST_TEMPLATE | 44 ++++++++++++++++++++++++++++++----- 1 file changed, 38 insertions(+), 6 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE index e7ed23dea0f9c..be57f007abbf0 100644 --- a/.github/PULL_REQUEST_TEMPLATE +++ b/.github/PULL_REQUEST_TEMPLATE @@ -1,10 +1,42 @@ -## What changes were proposed in this pull request? + -(Please fill in changes proposed in this fix) +### What changes were proposed in this pull request? + -## How was this patch tested? -(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) -(If this patch involves UI changes, please attach a screenshot; otherwise, remove this) +### Why are the changes needed? + -Please review https://spark.apache.org/contributing.html before opening a pull request. + +### Does this PR introduce any user-facing change? + + + +### How was this patch tested? + From babdba0f9ee3b70aad235734f67ef6563b0197f5 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 16 Aug 2019 03:40:41 -0700 Subject: [PATCH 079/149] [SPARK-28728][BUILD] Bump Jackson Databind to 2.9.9.3 ## What changes were proposed in this pull request? Update Jackson databind to the latest version for some latest changes. ## How was this patch tested? Pass the Jenkins. Closes #25451 from Fokko/fd-bump-jackson-databind. Lead-authored-by: Fokko Driesprong Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 2 +- dev/deps/spark-deps-hadoop-3.2 | 2 +- pom.xml | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 82389ea6aa58a..0022d2627c774 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -87,7 +87,7 @@ ivy-2.4.0.jar jackson-annotations-2.9.9.jar jackson-core-2.9.9.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.9.9.jar +jackson-databind-2.9.9.3.jar jackson-dataformat-yaml-2.9.9.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a02735d2be893..a46a56909a5e1 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -88,7 +88,7 @@ ivy-2.4.0.jar jackson-annotations-2.9.9.jar jackson-core-2.9.9.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.9.9.jar +jackson-databind-2.9.9.3.jar jackson-dataformat-yaml-2.9.9.jar jackson-jaxrs-base-2.9.5.jar jackson-jaxrs-json-provider-2.9.5.jar diff --git a/pom.xml b/pom.xml index b0372ce8a8e69..c9b5bd1628f55 100644 --- a/pom.xml +++ b/pom.xml @@ -171,6 +171,7 @@ true 1.9.13 2.9.9 + 2.9.9.3 1.1.7.3 1.1.2 1.10 @@ -700,7 +701,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.jackson.version} + ${fasterxml.jackson.databind.version} com.fasterxml.jackson.core From ef142371e7a2549259055f9d9e52016b390a7c9a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 16 Aug 2019 19:47:29 +0900 Subject: [PATCH 080/149] [SPARK-28736][SPARK-28735][PYTHON][ML] Fix PySpark ML tests to pass in JDK 11 ### What changes were proposed in this pull request? This PR proposes to fix both tests below: ``` ====================================================================== FAIL: test_raw_and_probability_prediction (pyspark.ml.tests.test_algorithms.MultilayerPerceptronClassifierTest) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/dongjoon/APACHE/spark-master/python/pyspark/ml/tests/test_algorithms.py", line 89, in test_raw_and_probability_prediction self.assertTrue(np.allclose(result.rawPrediction, expected_rawPrediction, atol=1E-4)) AssertionError: False is not true ``` ``` File "/Users/dongjoon/APACHE/spark-master/python/pyspark/mllib/clustering.py", line 386, in __main__.GaussianMixtureModel Failed example: abs(softPredicted[0] - 1.0) < 0.001 Expected: True Got: False ********************************************************************** File "/Users/dongjoon/APACHE/spark-master/python/pyspark/mllib/clustering.py", line 388, in __main__.GaussianMixtureModel Failed example: abs(softPredicted[1] - 0.0) < 0.001 Expected: True Got: False ``` to pass in JDK 11. The root cause seems to be different float values being understood via Py4J. This issue also was found in https://github.com/apache/spark/pull/25132 before. When floats are transferred from Python to JVM, the values are sent as are. Python floats are not "precise" due to its own limitation - https://docs.python.org/3/tutorial/floatingpoint.html. For some reasons, the floats from Python on JDK 8 and JDK 11 are different, which is already explicitly not guaranteed. This seems why only some tests in PySpark with floats are being failed. So, this PR fixes it by increasing tolerance in identified test cases in PySpark. ### Why are the changes needed? To fully support JDK 11. See, for instance, https://github.com/apache/spark/pull/25443 and https://github.com/apache/spark/pull/25423 for ongoing efforts. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested as described in JIRAs: ``` $ build/sbt -Phadoop-3.2 test:package $ python/run-tests --testnames 'pyspark.ml.tests.test_algorithms' --python-executables python ``` ``` $ build/sbt -Phadoop-3.2 test:package $ python/run-tests --testnames 'pyspark.mllib.clustering' --python-executables python ``` Closes #25475 from HyukjinKwon/SPARK-28735. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/ml/tests/test_algorithms.py | 2 +- python/pyspark/mllib/clustering.py | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index 4061fda3b19c9..1864a66487a28 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -86,7 +86,7 @@ def test_raw_and_probability_prediction(self): expected_rawPrediction = [-11.6081922998, -8.15827998691, 22.17757045] self.assertTrue(result.prediction, expected_prediction) self.assertTrue(np.allclose(result.probability, expected_probability, atol=1E-4)) - self.assertTrue(np.allclose(result.rawPrediction, expected_rawPrediction, atol=1E-4)) + self.assertTrue(np.allclose(result.rawPrediction, expected_rawPrediction, atol=1)) class OneVsRestTests(SparkSessionTestCase): diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 3524fcfeb7958..f220911993096 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -383,11 +383,11 @@ class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> model.predict([-0.1,-0.05]) 0 >>> softPredicted = model.predictSoft([-0.1,-0.05]) - >>> abs(softPredicted[0] - 1.0) < 0.001 + >>> abs(softPredicted[0] - 1.0) < 0.03 True - >>> abs(softPredicted[1] - 0.0) < 0.001 + >>> abs(softPredicted[1] - 0.0) < 0.03 True - >>> abs(softPredicted[2] - 0.0) < 0.001 + >>> abs(softPredicted[2] - 0.0) < 0.03 True >>> path = tempfile.mkdtemp() From c48e381214312de9c44eb7fc42ff5374595f8cba Mon Sep 17 00:00:00 2001 From: pavithra Date: Fri, 16 Aug 2019 22:46:04 +0900 Subject: [PATCH 081/149] [SPARK-28671][SQL] Throw NoSuchPermanentFunctionException for a non-exsistent permanent function in dropFunction/alterFunction ## What changes were proposed in this pull request? **Before Fix** When a non existent permanent function is dropped, generic NoSuchFunctionException was thrown.- which printed "This function is neither a registered temporary function nor a permanent function registered in the database" . This creates a ambiguity when a temp function in the same name exist. **After Fix** NoSuchPermanentFunctionException will be thrown, which will print "NoSuchPermanentFunctionException:Function not found in database " ## How was this patch tested? Unit test was run and corrected the UT. Closes #25394 from PavithraRamachandran/funcIssue. Lead-authored-by: pavithra Co-authored-by: pavithraramachandran Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 4 ++-- .../spark/sql/catalyst/catalog/SessionCatalogSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 74559f5d88796..2a6124a4079a1 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 @@ -1114,7 +1114,7 @@ class SessionCatalog( } externalCatalog.dropFunction(db, name.funcName) } else if (!ignoreIfNotExists) { - throw new NoSuchFunctionException(db = db, func = identifier.toString) + throw new NoSuchPermanentFunctionException(db = db, func = identifier.toString) } } @@ -1137,7 +1137,7 @@ class SessionCatalog( } externalCatalog.alterFunction(db, newFuncDefinition) } else { - throw new NoSuchFunctionException(db = db, func = identifier.toString) + throw new NoSuchPermanentFunctionException(db = db, func = identifier.toString) } } 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 bce85534ce7e4..59fb941f41f7e 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 @@ -1429,7 +1429,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { catalog.dropFunction( FunctionIdentifier("something", Some("unknown_db")), ignoreIfNotExists = false) } - intercept[NoSuchFunctionException] { + intercept[NoSuchPermanentFunctionException] { catalog.dropFunction(FunctionIdentifier("does_not_exist"), ignoreIfNotExists = false) } catalog.dropFunction(FunctionIdentifier("does_not_exist"), ignoreIfNotExists = true) From 92bfd9a317b928597c5ee295af93bf45d2c2952b Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 16 Aug 2019 22:27:27 +0800 Subject: [PATCH 082/149] [SPARK-28757][SQL] File table location should include both values of option `path` and `paths` ### What changes were proposed in this pull request? If both options `path` and `paths` are passed to file data source v2, both values of the options should be included as the target paths. ### Why are the changes needed? In V1 implementation, file table location includes both values of option `path` and `paths`. In the refactoring of https://github.com/apache/spark/pull/24025, the value of option `path` is ignored if "paths" are specified. We should make it consistent with V1. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test Closes #25473 from gengliangwang/fixPathOption. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../datasources/v2/FileDataSourceV2.scala | 7 +++--- .../spark/sql/FileBasedDataSourceSuite.scala | 24 +++++++++++++++++++ 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index bcb10ae5999fc..ac786bbaac6d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -43,11 +43,10 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { protected def getPaths(map: CaseInsensitiveStringMap): Seq[String] = { val objectMapper = new ObjectMapper() - Option(map.get("paths")).map { pathStr => + val paths = Option(map.get("paths")).map { pathStr => objectMapper.readValue(pathStr, classOf[Array[String]]).toSeq - }.getOrElse { - Option(map.get("path")).toSeq - } + }.getOrElse(Seq.empty) + paths ++ Option(map.get("path")).toSeq } protected def getTableName(paths: Seq[String]): String = { 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 51e26d42812ce..b1bde9098e096 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 @@ -29,6 +29,9 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -707,6 +710,27 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } } } + + test("File table location should include both values of option `path` and `paths`") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + withTempPaths(3) { paths => + paths.zipWithIndex.foreach { case (path, index) => + Seq(index).toDF("a").write.mode("overwrite").parquet(path.getCanonicalPath) + } + val df = spark + .read + .option("path", paths.head.getCanonicalPath) + .parquet(paths(1).getCanonicalPath, paths(2).getCanonicalPath) + df.queryExecution.optimizedPlan match { + case PhysicalOperation(_, _, DataSourceV2Relation(table: ParquetTable, _, _)) => + assert(table.paths.toSet == paths.map(_.getCanonicalPath).toSet) + case _ => + throw new AnalysisException("Can not match ParquetTable in the query.") + } + checkAnswer(df, Seq(0, 1, 2).map(Row(_))) + } + } + } } object TestingUDT { From 036fd3903f9d863fec02c810456229f41d0e4b1f Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 16 Aug 2019 23:24:32 +0800 Subject: [PATCH 083/149] [SPARK-27637][SHUFFLE][FOLLOW-UP] For nettyBlockTransferService, if IOException occurred while create client, check whether relative executor is alive before retry #24533 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? In pr #[24533](https://github.com/apache/spark/pull/24533/files) , it prevent retry to a removed Executor. In my test, I can't catch exceptions from ` new OneForOneBlockFetcher(client, appId, execId, blockIds, listener, transportConf, tempFileManager).start()` And I check the code carefully, method **start()** will handle exception of IOException in it's retry logical, won't throw it out. until it meet maxRetry times or meet exception that is not IOException. And if we meet the situation that when we fetch block , the executor is dead, when we rerun `RetryingBlockFetcher.BlockFetchStarter.createAndStart()` we may failed when we create a transport client to dead executor. it will throw a IOException. We should catch this IOException. ### Why are the changes needed? Old solution not comprehensive. Didn't cover more case. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existed Unit Test Closes #25469 from AngersZhuuuu/SPARK-27637-FLLOW-UP. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- .../apache/spark/network/netty/NettyBlockTransferService.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index b12cd4254f19e..1d27fe7db193f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -117,8 +117,8 @@ private[spark] class NettyBlockTransferService( try { val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) { - val client = clientFactory.createClient(host, port) try { + val client = clientFactory.createClient(host, port) new OneForOneBlockFetcher(client, appId, execId, blockIds, listener, transportConf, tempFileManager).start() } catch { From 2f04152921668893169fee0da2424fdd73437964 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 16 Aug 2019 10:01:59 -0700 Subject: [PATCH 084/149] [SPARK-28756][R] Fix checkJavaVersion to accept JDK8+ ### What changes were proposed in this pull request? Currently, `checkJavaVersion` only accepts JDK8 because it compares with the number in `SystemRequirements`. This PR changes it to accept the higher version, too. ### Why are the changes needed? Without this, two test suites are skipped on JDK11 environment due to this check. **BEFORE** ``` $ build/mvn -Phadoop-3.2 -Psparkr -DskipTests package $ R/install-dev.sh $ R/run-tests.sh ... basic tests for CRAN: SS Skipped ------------------------------------------------------------------------ 1. create DataFrame from list or data.frame (test_basic.R#21) - error on Java check 2. spark.glm and predict (test_basic.R#57) - error on Java check DONE =========================================================================== ``` **AFTER** ``` basic tests for CRAN: ............. DONE =========================================================================== ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually, build and test on JDK11. Closes #25472 from dongjoon-hyun/SPARK-28756. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/pkg/DESCRIPTION | 2 +- R/pkg/R/client.R | 16 +++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 3d31be809be61..6a83e00dff79f 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -13,7 +13,7 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), License: Apache License (== 2.0) URL: https://www.apache.org/ https://spark.apache.org/ BugReports: https://spark.apache.org/contributing.html -SystemRequirements: Java (== 8) +SystemRequirements: Java (>= 8) Depends: R (>= 3.1), methods diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R index 660f0864403e0..3299346bce007 100644 --- a/R/pkg/R/client.R +++ b/R/pkg/R/client.R @@ -91,11 +91,17 @@ checkJavaVersion <- function() { }, javaVersionOut) javaVersionStr <- strsplit(javaVersionFilter[[1]], "[\"]")[[1L]][2] - # javaVersionStr is of the form 1.8.0_92. - # Extract 8 from it to compare to sparkJavaVersion - javaVersionNum <- as.integer(strsplit(javaVersionStr, "[.]")[[1L]][2]) - if (javaVersionNum != sparkJavaVersion) { - stop(paste("Java version", sparkJavaVersion, "is required for this package; found version:", + # javaVersionStr is of the form 1.8.0_92/9.0.x/11.0.x. + # We are using 8, 9, 10, 11 for sparkJavaVersion. + versions <- strsplit(javaVersionStr, "[.]")[[1L]] + if ("1" == versions[1]) { + javaVersionNum <- as.integer(versions[2]) + } else { + javaVersionNum <- as.integer(versions[1]) + } + if (javaVersionNum < sparkJavaVersion) { + stop(paste("Java version", sparkJavaVersion, + ", or greater, is required for this package; found version:", javaVersionStr)) } return(javaVersionNum) From 7f44a6e367d79b5366df9899022c734c78b51de9 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 16 Aug 2019 10:03:14 -0700 Subject: [PATCH 085/149] [SPARK-28755][R][TESTS] Increase tolerance in 'spark.mlp' SparkR test for JDK 11 ### What changes were proposed in this pull request? This PR proposes to increase the tolerance for the exact value comparison in `spark.mlp` test. I don't know the root cause but some tolerance is already expected. I suspect it is not a big deal considering all other tests pass. The values are fairly close: JDK 8: ``` -24.28415, 107.8701, 16.86376, 1.103736, 9.244488 ``` JDK 11: ``` -24.33892, 108.0316, 16.89082, 1.090723, 9.260533 ``` ### Why are the changes needed? To fully support JDK 11. See, for instance, #25443 and #25423 for ongoing efforts. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested on the top of https://github.com/apache/spark/pull/25472 with JDK 11 ```bash ./build/mvn -DskipTests -Psparkr -Phadoop-3.2 package ./bin/sparkR ``` ```R absoluteSparkPath <- function(x) { sparkHome <- sparkR.conf("spark.home") file.path(sparkHome, x) } df <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), source = "libsvm") model <- spark.mlp(df, label ~ features, blockSize = 128, layers = c(4, 5, 4, 3), solver = "l-bfgs", maxIter = 100, tol = 0.00001, stepSize = 1, seed = 1) summary <- summary(model) head(summary$weights, 5) ``` Closes #25478 from HyukjinKwon/SPARK-28755. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- R/pkg/tests/fulltests/test_mllib_classification.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R index 1f1b187aef567..2da3a022f8941 100644 --- a/R/pkg/tests/fulltests/test_mllib_classification.R +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -308,7 +308,7 @@ test_that("spark.mlp", { expect_equal(summary$layers, c(4, 5, 4, 3)) expect_equal(length(summary$weights), 64) expect_equal(head(summary$weights, 5), list(-24.28415, 107.8701, 16.86376, 1.103736, 9.244488), - tolerance = 1e-6) + tolerance = 1e-1) # Test predict method mlpTestDF <- df From 0094b5fe727823e8f90c21825edf910e63364281 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 16 Aug 2019 12:39:12 -0500 Subject: [PATCH 086/149] [SPARK-28722][ML] Change sequential label sorting in StringIndexer fit to parallel ## What changes were proposed in this pull request? The `fit` method in `StringIndexer` sorts given labels in a sequential approach, if there are multiple input columns. When the number of input column increases, the time of label sorting dramatically increases too so it is hard to use in practice if dealing with hundreds of input columns. This patch tries to make the label sorting parallel. This runs benchmark like: ```scala import org.apache.spark.ml.feature.StringIndexer val numCol = 300 val data = (0 to 100).map { i => (i, 100 * i) } var df = data.toDF("id", "label0") (1 to numCol).foreach { idx => df = df.withColumn(s"label$idx", col("label0") + 1) } val inputCols = (0 to numCol).map(i => s"label$i").toArray val outputCols = (0 to numCol).map(i => s"labelIndex$i").toArray val t0 = System.nanoTime() val indexer = new StringIndexer().setInputCols(inputCols).setOutputCols(outputCols).setStringOrderType("alphabetDesc").fit(df) val t1 = System.nanoTime() println("Elapsed time: " + (t1 - t0) / 1000000000.0 + "s") ``` | numCol | 20 | 50 | 100 | 200 | 300 | |--:|---|---|---|---|---| | Before | 9.85 | 28.62 | 64.35 | 167.17 | 431.60 | | After | 2.44 | 2.71 | 3.34 | 4.83 | 6.90 | Unit: second ## How was this patch tested? Passed existing tests. Manually test for performance. Closes #25442 from viirya/improve_stringindexer2. Authored-by: Liang-Chi Hsieh Signed-off-by: Sean Owen --- .../spark/ml/feature/StringIndexer.scala | 92 +++++++++---------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 94f40c3529676..eab90a32a3f4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ThreadUtils import org.apache.spark.util.VersionUtils.majorMinorVersion import org.apache.spark.util.collection.OpenHashMap @@ -173,14 +174,12 @@ class StringIndexer @Since("1.4.0") ( @Since("3.0.0") def setOutputCols(value: Array[String]): this.type = set(outputCols, value) - private def countByValue( - dataset: Dataset[_], - inputCols: Array[String]): Array[OpenHashMap[String, Long]] = { - - val aggregator = new StringIndexerAggregator(inputCols.length) - implicit val encoder = Encoders.kryo[Array[OpenHashMap[String, Long]]] - - val selectedCols = inputCols.map { colName => + /** + * Gets columns from dataset. If a column is not string type, we replace NaN values + * with null. Columns are casted to string type. + */ + private def getSelectedCols(dataset: Dataset[_], inputCols: Seq[String]): Seq[Column] = { + inputCols.map { colName => val col = dataset.col(colName) if (col.expr.dataType == StringType) { col @@ -190,7 +189,16 @@ class StringIndexer @Since("1.4.0") ( new Column(If(col.isNaN.expr, Literal(null), col.expr)).cast(StringType) } } + } + + private def countByValue( + dataset: Dataset[_], + inputCols: Array[String]): Array[OpenHashMap[String, Long]] = { + val aggregator = new StringIndexerAggregator(inputCols.length) + implicit val encoder = Encoders.kryo[Array[OpenHashMap[String, Long]]] + + val selectedCols = getSelectedCols(dataset, inputCols) dataset.select(selectedCols: _*) .toDF .groupBy().agg(aggregator.toColumn) @@ -198,51 +206,43 @@ class StringIndexer @Since("1.4.0") ( .collect()(0) } - @Since("2.0.0") - override def fit(dataset: Dataset[_]): StringIndexerModel = { - transformSchema(dataset.schema, logging = true) + private def sortByFreq(dataset: Dataset[_], ascending: Boolean): Array[Array[String]] = { + val (inputCols, _) = getInOutCols() + + val sortFunc = StringIndexer.getSortFunc(ascending = ascending) + val orgStrings = countByValue(dataset, inputCols).toSeq + ThreadUtils.parmap(orgStrings, "sortingStringLabels", 8) { counts => + counts.toSeq.sortWith(sortFunc).map(_._1).toArray + }.toArray + } + private def sortByAlphabet(dataset: Dataset[_], ascending: Boolean): Array[Array[String]] = { val (inputCols, _) = getInOutCols() - // If input dataset is not originally cached, we need to unpersist it - // once we persist it later. - val needUnpersist = dataset.storageLevel == StorageLevel.NONE + val selectedCols = getSelectedCols(dataset, inputCols).map(collect_set(_)) + val allLabels = dataset.select(selectedCols: _*) + .collect().toSeq.flatMap(_.toSeq).asInstanceOf[Seq[Seq[String]]] + ThreadUtils.parmap(allLabels, "sortingStringLabels", 8) { labels => + val sorted = labels.filter(_ != null).sorted + if (ascending) { + sorted.toArray + } else { + sorted.reverse.toArray + } + }.toArray + } + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): StringIndexerModel = { + transformSchema(dataset.schema, logging = true) // In case of equal frequency when frequencyDesc/Asc, the strings are further sorted // alphabetically. val labelsArray = $(stringOrderType) match { - case StringIndexer.frequencyDesc => - val sortFunc = StringIndexer.getSortFunc(ascending = false) - countByValue(dataset, inputCols).map { counts => - counts.toSeq.sortWith(sortFunc).map(_._1).toArray - } - case StringIndexer.frequencyAsc => - val sortFunc = StringIndexer.getSortFunc(ascending = true) - countByValue(dataset, inputCols).map { counts => - counts.toSeq.sortWith(sortFunc).map(_._1).toArray - } - case StringIndexer.alphabetDesc => - import dataset.sparkSession.implicits._ - dataset.persist() - val labels = inputCols.map { inputCol => - dataset.select(inputCol).na.drop().distinct().sort(dataset(s"$inputCol").desc) - .as[String].collect() - } - if (needUnpersist) { - dataset.unpersist() - } - labels - case StringIndexer.alphabetAsc => - import dataset.sparkSession.implicits._ - dataset.persist() - val labels = inputCols.map { inputCol => - dataset.select(inputCol).na.drop().distinct().sort(dataset(s"$inputCol").asc) - .as[String].collect() - } - if (needUnpersist) { - dataset.unpersist() - } - labels + case StringIndexer.frequencyDesc => sortByFreq(dataset, ascending = false) + case StringIndexer.frequencyAsc => sortByFreq(dataset, ascending = true) + case StringIndexer.alphabetDesc => sortByAlphabet(dataset, ascending = false) + case StringIndexer.alphabetAsc => sortByAlphabet(dataset, ascending = true) } copyValues(new StringIndexerModel(uid, labelsArray).setParent(this)) } From 43101c7328339f34d4e16735c694ec002611adee Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 16 Aug 2019 11:33:02 -0700 Subject: [PATCH 087/149] [SPARK-28758][BUILD][SQL] Upgrade Janino to 3.0.15 ### What changes were proposed in this pull request? This PR aims to upgrade `Janino` from `3.0.13` to `3.0.15` in order to bring the bug fixes. Please note that `3.1.0` is a major refactoring instead of bug fixes. We had better use `3.0.15` and wait for the stabler 3.1.x. ### Why are the changes needed? This brings the following bug fixes. **3.0.15 (2019-07-28)** - Fix overloaded single static method import **3.0.14 (2019-07-05)** - Conflict in sbt-assembly - Overloaded static on-demand imported methods cause a CompileException: Ambiguous static method import - Handle overloaded static on-demand imports - Major refactoring of the Java 8 and Java 9 retrofit mechanism - Added tests for "JLS8 8.6 Instance Initializers" and "JLS8 8.7 Static Initializers" - Local variables in instance initializers don't work - Provide an option to keep generated code files - Added compile error handler and warning handler to ICompiler ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #25474 from dongjoon-hyun/SPARK-28758. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 4 ++-- dev/deps/spark-deps-hadoop-3.2 | 4 ++-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 0022d2627c774..18dd878786d54 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -30,7 +30,7 @@ commons-beanutils-1.9.3.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.13.jar +commons-compiler-3.0.15.jar commons-compress-1.8.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar @@ -96,7 +96,7 @@ jackson-module-paranamer-2.9.9.jar jackson-module-scala_2.12-2.9.9.jar jackson-xc-1.9.13.jar jakarta.xml.bind-api-2.3.2.jar -janino-3.0.13.jar +janino-3.0.15.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a46a56909a5e1..ec1e31a98b116 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -28,7 +28,7 @@ commons-beanutils-1.9.3.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.13.jar +commons-compiler-3.0.15.jar commons-compress-1.8.1.jar commons-configuration2-2.1.1.jar commons-crypto-1.0.0.jar @@ -97,7 +97,7 @@ jackson-module-jaxb-annotations-2.9.9.jar jackson-module-paranamer-2.9.9.jar jackson-module-scala_2.12-2.9.9.jar jakarta.xml.bind-api-2.3.2.jar -janino-3.0.13.jar +janino-3.0.15.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar diff --git a/pom.xml b/pom.xml index c9b5bd1628f55..72df4ce324ba5 100644 --- a/pom.xml +++ b/pom.xml @@ -181,7 +181,7 @@ 3.8.1 3.2.10 - 3.0.13 + 3.0.15 2.22.2 2.9.3 3.5.2 From c9b49f3978666af5d59a76e1cf2c23ed76f31a2d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 16 Aug 2019 15:08:04 -0700 Subject: [PATCH 088/149] [SPARK-28737][CORE] Update Jersey to 2.29 ## What changes were proposed in this pull request? Update Jersey to 2.27+, ideally 2.29, for possible JDK 11 fixes. ## How was this patch tested? Existing tests. Closes #25455 from srowen/SPARK-28737. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 8 +- core/pom.xml | 9 + dev/deps/spark-deps-hadoop-2.7 | 34 +-- dev/deps/spark-deps-hadoop-3.2 | 34 +-- .../LICENSE-jakarta-annotation-api | 277 ++++++++++++++++++ licenses-binary/LICENSE-jakarta-ws-rs-api | 277 ++++++++++++++++++ pom.xml | 13 +- 7 files changed, 615 insertions(+), 37 deletions(-) create mode 100644 licenses-binary/LICENSE-jakarta-annotation-api create mode 100644 licenses-binary/LICENSE-jakarta-ws-rs-api diff --git a/LICENSE-binary b/LICENSE-binary index 0d3dcd78fcbd8..65b1d61374ec7 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -464,10 +464,8 @@ javax.xml.stream:stax-api https://jcp.org/en/jsr/detail?id=173 Common Development and Distribution License (CDDL) 1.1 ------------------------------------------------------ -javax.annotation:javax.annotation-api https://jcp.org/en/jsr/detail?id=250 javax.servlet:javax.servlet-api https://javaee.github.io/servlet-spec/ javax.transaction:jta http://www.oracle.com/technetwork/java/index.html -javax.ws.rs:javax.ws.rs-api https://github.com/jax-rs javax.xml.bind:jaxb-api https://github.com/javaee/jaxb-v2 org.glassfish.hk2:hk2-api https://github.com/javaee/glassfish org.glassfish.hk2:hk2-locator (same) @@ -492,6 +490,12 @@ jakarta.xml.bind:jakarta.xml.bind-api com.sun.istack:istack-commons-runtime +Eclipse Public License (EPL) 2.0 +-------------------------------- + +jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca +jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api + Mozilla Public License (MPL) 1.1 -------------------------------- diff --git a/core/pom.xml b/core/pom.xml index b340c044a4cc4..b190ee213f38b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -260,6 +260,15 @@ org.glassfish.jersey.containers jersey-container-servlet-core + + org.glassfish.jersey.inject + jersey-hk2 + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-simple + test + io.netty netty-all diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 18dd878786d54..8638139d966d0 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -7,7 +7,7 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar +aopalliance-repackaged-2.5.0.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar @@ -75,9 +75,9 @@ hadoop-yarn-client-2.7.4.jar hadoop-yarn-common-2.7.4.jar hadoop-yarn-server-common-2.7.4.jar hadoop-yarn-server-web-proxy-2.7.4.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar +hk2-api-2.5.0.jar +hk2-locator-2.5.0.jar +hk2-utils-2.5.0.jar hppc-0.7.2.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar @@ -95,26 +95,26 @@ jackson-module-jaxb-annotations-2.9.9.jar jackson-module-paranamer-2.9.9.jar jackson-module-scala_2.12-2.9.9.jar jackson-xc-1.9.13.jar +jakarta.annotation-api-1.3.4.jar +jakarta.inject-2.5.0.jar +jakarta.ws.rs-api-2.1.5.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar +javassist-3.22.0-CR2.jar javax.inject-1.jar -javax.inject-2.4.0-b34.jar javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar jaxb-runtime-2.3.2.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar +jersey-client-2.29.jar +jersey-common-2.29.jar +jersey-container-servlet-2.29.jar +jersey-container-servlet-core-2.29.jar +jersey-hk2-2.29.jar +jersey-media-jaxb-2.29.jar +jersey-server-2.29.jar jetty-6.1.26.jar jetty-sslengine-6.1.26.jar jetty-util-6.1.26.jar @@ -160,7 +160,7 @@ orc-core-1.5.5-nohive.jar orc-mapreduce-1.5.5-nohive.jar orc-shims-1.5.5.jar oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar +osgi-resource-locator-1.0.3.jar paranamer-2.8.jar parquet-column-1.10.1.jar parquet-common-1.10.1.jar @@ -192,7 +192,7 @@ stream-2.9.6.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar +validation-api-2.0.1.Final.jar xbean-asm7-shaded-4.14.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index ec1e31a98b116..af93dd167b4d0 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -8,7 +8,7 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar +aopalliance-repackaged-2.5.0.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar arrow-format-0.12.0.jar @@ -76,9 +76,9 @@ hadoop-yarn-registry-3.2.0.jar hadoop-yarn-server-common-3.2.0.jar hadoop-yarn-server-web-proxy-3.2.0.jar hive-storage-api-2.6.0.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar +hk2-api-2.5.0.jar +hk2-locator-2.5.0.jar +hk2-utils-2.5.0.jar hppc-0.7.2.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar @@ -96,27 +96,27 @@ jackson-mapper-asl-1.9.13.jar jackson-module-jaxb-annotations-2.9.9.jar jackson-module-paranamer-2.9.9.jar jackson-module-scala_2.12-2.9.9.jar +jakarta.annotation-api-1.3.4.jar +jakarta.inject-2.5.0.jar +jakarta.ws.rs-api-2.1.5.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar +javassist-3.22.0-CR2.jar javax.inject-1.jar -javax.inject-2.4.0-b34.jar javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.11.jar jaxb-runtime-2.3.2.jar jcip-annotations-1.0-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar +jersey-client-2.29.jar +jersey-common-2.29.jar +jersey-container-servlet-2.29.jar +jersey-container-servlet-core-2.29.jar +jersey-hk2-2.29.jar +jersey-media-jaxb-2.29.jar +jersey-server-2.29.jar jetty-webapp-9.4.18.v20190429.jar jetty-xml-9.4.18.v20190429.jar jline-2.14.6.jar @@ -179,7 +179,7 @@ orc-core-1.5.5-nohive.jar orc-mapreduce-1.5.5-nohive.jar orc-shims-1.5.5.jar oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar +osgi-resource-locator-1.0.3.jar paranamer-2.8.jar parquet-column-1.10.1.jar parquet-common-1.10.1.jar @@ -212,7 +212,7 @@ stringtemplate-3.2.1.jar super-csv-2.2.0.jar token-provider-1.0.1.jar univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar +validation-api-2.0.1.Final.jar woodstox-core-5.0.3.jar xbean-asm7-shaded-4.14.jar xz-1.5.jar diff --git a/licenses-binary/LICENSE-jakarta-annotation-api b/licenses-binary/LICENSE-jakarta-annotation-api new file mode 100644 index 0000000000000..e23ece2c85241 --- /dev/null +++ b/licenses-binary/LICENSE-jakarta-annotation-api @@ -0,0 +1,277 @@ +Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + +"Contributor" means any person or entity that Distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which +are necessarily infringed by the use or sale of its Contribution alone +or when combined with the Program. + +"Program" means the Contributions Distributed in accordance with this +Agreement. + +"Recipient" means anyone who receives the Program under this Agreement +or any Secondary License (as applicable), including Contributors. + +"Derivative Works" shall mean any work, whether in Source Code or other +form, that is based on (or derived from) the Program and for which the +editorial revisions, annotations, elaborations, or other modifications +represent, as a whole, an original work of authorship. + +"Modified Works" shall mean any work in Source Code or other form that +results from an addition to, deletion from, or modification of the +contents of the Program, including, for purposes of clarity any new file +in Source Code form that contains any contents of the Program. Modified +Works shall not include works that contain only declarations, +interfaces, types, classes, structures, or files of the Program solely +in each case in order to link to, bind by name, or subclass the Program +or Modified Works thereof. + +"Distribute" means the acts of a) distributing or b) making available +in any manner that enables the transfer of a copy. + +"Source Code" means the form of a Program preferred for making +modifications, including but not limited to software source code, +documentation source, and configuration files. + +"Secondary License" means either the GNU General Public License, +Version 2.0, or any later versions of that license, including any +exceptions or additional permissions as identified by the initial +Contributor. + +2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + +3. REQUIREMENTS + +3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + +3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + +3.3 Contributors may not remove or alter any copyright, patent, +trademark, attribution notices, disclaimers of warranty, or limitations +of liability ("notices") contained within the Program from any copy of +the Program which they Distribute, provided that Contributors may add +their own appropriate notices. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities +with respect to end users, business partners and the like. While this +license is intended to facilitate the commercial use of the Program, +the Contributor who includes the Program in a commercial product +offering should do so in a manner which does not create potential +liability for other Contributors. Therefore, if a Contributor includes +the Program in a commercial product offering, such Contributor +("Commercial Contributor") hereby agrees to defend and indemnify every +other Contributor ("Indemnified Contributor") against any losses, +damages and costs (collectively "Losses") arising from claims, lawsuits +and other legal actions brought by a third party against the Indemnified +Contributor to the extent caused by the acts or omissions of such +Commercial Contributor in connection with its distribution of the Program +in a commercial product offering. The obligations in this section do not +apply to any claims or Losses relating to any actual or alleged +intellectual property infringement. In order to qualify, an Indemnified +Contributor must: a) promptly notify the Commercial Contributor in +writing of such claim, and b) allow the Commercial Contributor to control, +and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may +participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial +product offering, Product X. That Contributor is then a Commercial +Contributor. If that Commercial Contributor then makes performance +claims, or offers warranties related to Product X, those performance +claims and warranties are such Commercial Contributor's responsibility +alone. Under this section, the Commercial Contributor would have to +defend claims against the other Contributors related to those performance +claims and warranties, and if a court requires any other Contributor to +pay any damages as a result, the Commercial Contributor must pay +those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT +PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED 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. Each Recipient is solely responsible for determining the +appropriateness of using and distributing the Program and assumes all +risks associated with its exercise of rights under this Agreement, +including but not limited to the risks and costs of program errors, +compliance with applicable laws, damage to or loss of data, programs +or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT +PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS +SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE +EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under +applicable law, it shall not affect the validity or enforceability of +the remainder of the terms of this Agreement, and without further +action by the parties hereto, such provision shall be reformed to the +minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity +(including a cross-claim or counterclaim in a lawsuit) alleging that the +Program itself (excluding combinations of the Program with other software +or hardware) infringes such Recipient's patent(s), then such Recipient's +rights granted under Section 2(b) shall terminate as of the date such +litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it +fails to comply with any of the material terms or conditions of this +Agreement and does not cure such failure in a reasonable period of +time after becoming aware of such noncompliance. If all Recipient's +rights under this Agreement terminate, Recipient agrees to cease use +and distribution of the Program as soon as reasonably practicable. +However, Recipient's obligations under this Agreement and any licenses +granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, +but in order to avoid inconsistency the Agreement is copyrighted and +may only be modified in the following manner. The Agreement Steward +reserves the right to publish new versions (including revisions) of +this Agreement from time to time. No one other than the Agreement +Steward has the right to modify this Agreement. The Eclipse Foundation +is the initial Agreement Steward. The Eclipse Foundation may assign the +responsibility to serve as the Agreement Steward to a suitable separate +entity. Each new version of the Agreement will be given a distinguishing +version number. The Program (including Contributions) may always be +Distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to Distribute the Program (including its +Contributions) under the new version. + +Except as expressly stated in Sections 2(a) and 2(b) above, Recipient +receives no rights or licenses to the intellectual property of any +Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted +under this Agreement are reserved. Nothing in this Agreement is intended +to be enforceable by any entity that is not a Contributor or Recipient. +No third-party beneficiary rights are created under this Agreement. + +Exhibit A - Form of Secondary Licenses Notice + +"This Source Code may also be made available under the following +Secondary Licenses when the conditions for such availability set forth +in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), +version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. \ No newline at end of file diff --git a/licenses-binary/LICENSE-jakarta-ws-rs-api b/licenses-binary/LICENSE-jakarta-ws-rs-api new file mode 100644 index 0000000000000..e23ece2c85241 --- /dev/null +++ b/licenses-binary/LICENSE-jakarta-ws-rs-api @@ -0,0 +1,277 @@ +Eclipse Public License - v 2.0 + + THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE + PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION + OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + + a) in the case of the initial Contributor, the initial content + Distributed under this Agreement, and + + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from + and are Distributed by that particular Contributor. A Contribution + "originates" from a Contributor if it was added to the Program by + such Contributor itself or anyone acting on such Contributor's behalf. + Contributions do not include changes or additions to the Program that + are not Modified Works. + +"Contributor" means any person or entity that Distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which +are necessarily infringed by the use or sale of its Contribution alone +or when combined with the Program. + +"Program" means the Contributions Distributed in accordance with this +Agreement. + +"Recipient" means anyone who receives the Program under this Agreement +or any Secondary License (as applicable), including Contributors. + +"Derivative Works" shall mean any work, whether in Source Code or other +form, that is based on (or derived from) the Program and for which the +editorial revisions, annotations, elaborations, or other modifications +represent, as a whole, an original work of authorship. + +"Modified Works" shall mean any work in Source Code or other form that +results from an addition to, deletion from, or modification of the +contents of the Program, including, for purposes of clarity any new file +in Source Code form that contains any contents of the Program. Modified +Works shall not include works that contain only declarations, +interfaces, types, classes, structures, or files of the Program solely +in each case in order to link to, bind by name, or subclass the Program +or Modified Works thereof. + +"Distribute" means the acts of a) distributing or b) making available +in any manner that enables the transfer of a copy. + +"Source Code" means the form of a Program preferred for making +modifications, including but not limited to software source code, +documentation source, and configuration files. + +"Secondary License" means either the GNU General Public License, +Version 2.0, or any later versions of that license, including any +exceptions or additional permissions as identified by the initial +Contributor. + +2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free copyright + license to reproduce, prepare Derivative Works of, publicly display, + publicly perform, Distribute and sublicense the Contribution of such + Contributor, if any, and such Derivative Works. + + b) Subject to the terms of this Agreement, each Contributor hereby + grants Recipient a non-exclusive, worldwide, royalty-free patent + license under Licensed Patents to make, use, sell, offer to sell, + import and otherwise transfer the Contribution of such Contributor, + if any, in Source Code or other form. This patent license shall + apply to the combination of the Contribution and the Program if, at + the time the Contribution is added by the Contributor, such addition + of the Contribution causes such combination to be covered by the + Licensed Patents. The patent license shall not apply to any other + combinations which include the Contribution. No hardware per se is + licensed hereunder. + + c) Recipient understands that although each Contributor grants the + licenses to its Contributions set forth herein, no assurances are + provided by any Contributor that the Program does not infringe the + patent or other intellectual property rights of any other entity. + Each Contributor disclaims any liability to Recipient for claims + brought by any other entity based on infringement of intellectual + property rights or otherwise. As a condition to exercising the + rights and licenses granted hereunder, each Recipient hereby + assumes sole responsibility to secure any other intellectual + property rights needed, if any. For example, if a third party + patent license is required to allow Recipient to Distribute the + Program, it is Recipient's responsibility to acquire that license + before distributing the Program. + + d) Each Contributor represents that to its knowledge it has + sufficient copyright rights in its Contribution, if any, to grant + the copyright license set forth in this Agreement. + + e) Notwithstanding the terms of any Secondary License, no + Contributor makes additional grants to any Recipient (other than + those set forth in this Agreement) as a result of such Recipient's + receipt of the Program under the terms of a Secondary License + (if permitted under the terms of Section 3). + +3. REQUIREMENTS + +3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in + accordance with section 3.2, and the Contributor must accompany + the Program with a statement that the Source Code for the Program + is available under this Agreement, and informs Recipients how to + obtain it in a reasonable manner on or through a medium customarily + used for software exchange; and + + b) the Contributor may Distribute the Program under a license + different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all + warranties and conditions, express and implied, including + warranties or conditions of title and non-infringement, and + implied warranties or conditions of merchantability and fitness + for a particular purpose; + + ii) effectively excludes on behalf of all other Contributors all + liability for damages, including direct, indirect, special, + incidental and consequential damages, such as lost profits; + + iii) does not attempt to limit or alter the recipients' rights + in the Source Code under section 3.2; and + + iv) requires any subsequent distribution of the Program by any + party to be under a license that satisfies the requirements + of this section 3. + +3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the + Program (i) is combined with other material in a separate file or + files made available under a Secondary License, and (ii) the initial + Contributor attached to the Source Code the notice described in + Exhibit A of this Agreement, then the Program may be made available + under the terms of such Secondary Licenses, and + + b) a copy of this Agreement must be included with each copy of + the Program. + +3.3 Contributors may not remove or alter any copyright, patent, +trademark, attribution notices, disclaimers of warranty, or limitations +of liability ("notices") contained within the Program from any copy of +the Program which they Distribute, provided that Contributors may add +their own appropriate notices. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities +with respect to end users, business partners and the like. While this +license is intended to facilitate the commercial use of the Program, +the Contributor who includes the Program in a commercial product +offering should do so in a manner which does not create potential +liability for other Contributors. Therefore, if a Contributor includes +the Program in a commercial product offering, such Contributor +("Commercial Contributor") hereby agrees to defend and indemnify every +other Contributor ("Indemnified Contributor") against any losses, +damages and costs (collectively "Losses") arising from claims, lawsuits +and other legal actions brought by a third party against the Indemnified +Contributor to the extent caused by the acts or omissions of such +Commercial Contributor in connection with its distribution of the Program +in a commercial product offering. The obligations in this section do not +apply to any claims or Losses relating to any actual or alleged +intellectual property infringement. In order to qualify, an Indemnified +Contributor must: a) promptly notify the Commercial Contributor in +writing of such claim, and b) allow the Commercial Contributor to control, +and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may +participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial +product offering, Product X. That Contributor is then a Commercial +Contributor. If that Commercial Contributor then makes performance +claims, or offers warranties related to Product X, those performance +claims and warranties are such Commercial Contributor's responsibility +alone. Under this section, the Commercial Contributor would have to +defend claims against the other Contributors related to those performance +claims and warranties, and if a court requires any other Contributor to +pay any damages as a result, the Commercial Contributor must pay +those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT +PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED 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. Each Recipient is solely responsible for determining the +appropriateness of using and distributing the Program and assumes all +risks associated with its exercise of rights under this Agreement, +including but not limited to the risks and costs of program errors, +compliance with applicable laws, damage to or loss of data, programs +or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT +PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS +SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE +EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under +applicable law, it shall not affect the validity or enforceability of +the remainder of the terms of this Agreement, and without further +action by the parties hereto, such provision shall be reformed to the +minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity +(including a cross-claim or counterclaim in a lawsuit) alleging that the +Program itself (excluding combinations of the Program with other software +or hardware) infringes such Recipient's patent(s), then such Recipient's +rights granted under Section 2(b) shall terminate as of the date such +litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it +fails to comply with any of the material terms or conditions of this +Agreement and does not cure such failure in a reasonable period of +time after becoming aware of such noncompliance. If all Recipient's +rights under this Agreement terminate, Recipient agrees to cease use +and distribution of the Program as soon as reasonably practicable. +However, Recipient's obligations under this Agreement and any licenses +granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, +but in order to avoid inconsistency the Agreement is copyrighted and +may only be modified in the following manner. The Agreement Steward +reserves the right to publish new versions (including revisions) of +this Agreement from time to time. No one other than the Agreement +Steward has the right to modify this Agreement. The Eclipse Foundation +is the initial Agreement Steward. The Eclipse Foundation may assign the +responsibility to serve as the Agreement Steward to a suitable separate +entity. Each new version of the Agreement will be given a distinguishing +version number. The Program (including Contributions) may always be +Distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to Distribute the Program (including its +Contributions) under the new version. + +Except as expressly stated in Sections 2(a) and 2(b) above, Recipient +receives no rights or licenses to the intellectual property of any +Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted +under this Agreement are reserved. Nothing in this Agreement is intended +to be enforceable by any entity that is not a Contributor or Recipient. +No third-party beneficiary rights are created under this Agreement. + +Exhibit A - Form of Secondary Licenses Notice + +"This Source Code may also be made available under the following +Secondary Licenses when the conditions for such availability set forth +in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), +version(s), and exceptions or additional permissions here}." + + Simply including a copy of this Agreement, including this Exhibit A + is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular + file, then You may include the notice in a location (such as a LICENSE + file in a relevant directory) where a recipient would be likely to + look for such a notice. + + You may add additional accurate notices of copyright ownership. \ No newline at end of file diff --git a/pom.xml b/pom.xml index 72df4ce324ba5..2f028284c08cf 100644 --- a/pom.xml +++ b/pom.xml @@ -182,7 +182,7 @@ 3.8.1 3.2.10 3.0.15 - 2.22.2 + 2.29 2.9.3 3.5.2 3.0.0 @@ -751,6 +751,17 @@ jersey-container-servlet-core ${jersey.version} + + org.glassfish.jersey.inject + jersey-hk2 + ${jersey.version} + + + org.glassfish.jersey.test-framework.providers + jersey-test-framework-provider-simple + ${jersey.version} + test + org.glassfish.jersey jersey-client From 1819a6f22eee5314197aab4c169c74bd6ff6c17c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 16 Aug 2019 21:23:11 -0700 Subject: [PATCH 089/149] [SPARK-28759][BUILD] Upgrade scala-maven-plugin to 4.1.1 ### What changes were proposed in this pull request? This PR aims to upgrade `scala-maven-plugin` to 4.1.1 to bring the improvement (including Scala 2.13.0 support, Zinc update) and bug fixes in the plugin. ### Why are the changes needed? `4.1.1` uses the latest dependent plugins. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #25476 from dongjoon-hyun/SPARK-28759. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2f028284c08cf..37fe511a885db 100644 --- a/pom.xml +++ b/pom.xml @@ -2266,7 +2266,7 @@ net.alchim31.maven scala-maven-plugin - 3.4.4 + 4.1.1 eclipse-add-source From f7c9de90355dd5da63c401309a202f3afe7695fd Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 17 Aug 2019 10:16:22 -0700 Subject: [PATCH 090/149] [SPARK-28765][BUILD] Add explict exclusions to avoid JDK11 dependency issue ### What changes were proposed in this pull request? This PR adds explicit exclusions to avoid Maven `JDK11` dependency issues. ### Why are the changes needed? Maven/Ivy seems to be confused during dependency generation on `JDK11` environment. This is not only wrong, but also causes a Jenkins failure during dependency manifest check on `JDK11` environment. **JDK8** ``` $ cd core $ mvn -X dependency:tree -Dincludes=jakarta.activation:jakarta.activation-api ... [DEBUG] org.glassfish.jersey.core:jersey-server:jar:2.29:compile (version managed from 2.22.2) [DEBUG] org.glassfish.jersey.media:jersey-media-jaxb:jar:2.29:compile [DEBUG] javax.validation:validation-api:jar:2.0.1.Final:compile ``` **JDK11** ``` [DEBUG] org.glassfish.jersey.core:jersey-server:jar:2.29:compile (version managed from 2.22.2) [DEBUG] org.glassfish.jersey.media:jersey-media-jaxb:jar:2.29:compile [DEBUG] javax.validation:validation-api:jar:2.0.1.Final:compile [DEBUG] jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.2:compile [DEBUG] jakarta.activation:jakarta.activation-api:jar:1.2.1:compile ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Do the following in both `JDK8` and `JDK11` environment. The dependency manifest should not be changed. In the current `master` branch, `JDK11` changes the dependency manifest. ``` $ dev/test-dependencies.sh --replace-manifest ``` Closes #25481 from dongjoon-hyun/SPARK-28765. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 14 ++++++++++++++ resource-managers/kubernetes/core/pom.xml | 5 +++++ 2 files changed, 19 insertions(+) diff --git a/pom.xml b/pom.xml index 37fe511a885db..3b0383375c56b 100644 --- a/pom.xml +++ b/pom.xml @@ -730,11 +730,25 @@ org.glassfish.jersey.core jersey-server ${jersey.version} + + + + jakarta.xml.bind + jakarta.xml.bind-api + + org.glassfish.jersey.core jersey-common ${jersey.version} + + + + com.sun.activation + jakarta.activation + + org.glassfish.jersey.core diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 23106cb7ec68f..b350b050118fd 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -66,6 +66,11 @@ com.fasterxml.jackson.dataformat jackson-dataformat-yaml + + + javax.annotation + javax.annotation-api + From 5756a47a9fafca2d0b31de2b2374429f73b6e5e2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 17 Aug 2019 11:11:36 -0700 Subject: [PATCH 091/149] [SPARK-28766][R][DOC] Fix CRAN incoming feasibility warning on invalid URL MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This updates an URL in R doc to fix `Had CRAN check errors; see logs`. ### Why are the changes needed? Currently, this invalid link causes a warning during CRAN incoming feasibility. We had better fix this before submitting `3.0.0/2.4.4/2.3.4`. **BEFORE** ``` * checking CRAN incoming feasibility ... NOTE Maintainer: ‘Shivaram Venkataraman ’ Found the following (possibly) invalid URLs: URL: https://wiki.apache.org/hadoop/HCFS (moved to https://cwiki.apache.org/confluence/display/hadoop/HCFS) From: man/spark.addFile.Rd Status: 404 Message: Not Found ``` **AFTER** ``` * checking CRAN incoming feasibility ... Note_to_CRAN_maintainers Maintainer: ‘Shivaram Venkataraman ’ ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Check the warning message during R testing. ``` $ R/install-dev.sh $ R/run-tests.sh ``` Closes #25483 from dongjoon-hyun/SPARK-28766. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/pkg/R/context.R | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 619153645d925..51ae2d2954a9a 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -325,7 +325,8 @@ setCheckpointDirSC <- function(sc, dirName) { #' #' A directory can be given if the recursive option is set to true. #' Currently directories are only supported for Hadoop-supported filesystems. -#' Refer Hadoop-supported filesystems at \url{https://wiki.apache.org/hadoop/HCFS}. +#' Refer Hadoop-supported filesystems at +#' \url{https://cwiki.apache.org/confluence/display/HADOOP2/HCFS}. #' #' Note: A path can be added only once. Subsequent additions of the same path are ignored. #' From efbb0359024d1f77e207ca5fc78b3c32b4138558 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 17 Aug 2019 19:12:50 -0700 Subject: [PATCH 092/149] [SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server ## What changes were proposed in this pull request? This PR build a test framework that directly re-run all the tests in `SQLQueryTestSuite` via Thrift Server. But it's a little different from `SQLQueryTestSuite`: 1. Can not support [UDF testing](https://github.com/apache/spark/blob/44e607e9213bdceab970606fb15292db2fe157c2/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala#L293-L297). 2. Can not support `DESC` command and `SHOW` command because `SQLQueryTestSuite` [formatted the output](https://github.com/apache/spark/blob/1882912cca4921d3d8c8632b3bb34e69e8119791/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala#L38-L50.). When building this framework, found two bug: [SPARK-28624](https://issues.apache.org/jira/browse/SPARK-28624): `make_date` is inconsistent when reading from table [SPARK-28611](https://issues.apache.org/jira/browse/SPARK-28611): Histogram's height is different found two features that ThriftServer can not support: [SPARK-28636](https://issues.apache.org/jira/browse/SPARK-28636): ThriftServer can not support decimal type with negative scale [SPARK-28637](https://issues.apache.org/jira/browse/SPARK-28637): ThriftServer can not support interval type Also, found two inconsistent behavior: [SPARK-28620](https://issues.apache.org/jira/browse/SPARK-28620): Double type returned for float type in Beeline/JDBC [SPARK-28619](https://issues.apache.org/jira/browse/SPARK-28619): The golden result file is different when tested by `bin/spark-sql` ## How was this patch tested? N/A Closes #25373 from wangyum/SPARK-28527. Authored-by: Yuming Wang Signed-off-by: gatorsmile --- project/SparkBuild.scala | 3 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 67 ++-- sql/hive-thriftserver/pom.xml | 7 + .../hive/thriftserver/HiveThriftServer2.scala | 3 +- .../ThriftServerQueryTestSuite.scala | 362 ++++++++++++++++++ 5 files changed, 409 insertions(+), 33 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f1d2e3788918f..25c2fb4af5c34 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -474,7 +474,8 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", "org.apache.spark.ml.classification.LogisticRegressionSuite", "org.apache.spark.ml.classification.LinearSVCSuite", - "org.apache.spark.sql.SQLQueryTestSuite" + "org.apache.spark.sql.SQLQueryTestSuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" 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 4bdf25051127c..5c1ff9cd735eb 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 @@ -107,8 +107,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { import IntegratedUDFTestUtils._ private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + protected val isTestWithConfigSets: Boolean = true - private val baseResourcePath = { + protected val baseResourcePath = { // If regenerateGoldenFiles is true, we must be running this in SBT and we use hard-coded // relative path. Otherwise, we use classloader's getResource to find the location. if (regenerateGoldenFiles) { @@ -119,13 +120,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - private val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath - private val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + protected val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + protected val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath - private val validFileExtensions = ".sql" + protected val validFileExtensions = ".sql" + + private val notIncludedMsg = "[not included in comparison]" + private val clsName = this.getClass.getCanonicalName /** List of test cases to ignore, in lower cases. */ - private val blackList = Set( + protected def blackList: Set[String] = Set( "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. ) @@ -133,7 +137,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { listTestCases().foreach(createScalaTestCase) /** A single SQL query's output. */ - private case class QueryOutput(sql: String, schema: String, output: String) { + protected case class QueryOutput(sql: String, schema: String, output: String) { def toString(queryIndex: Int): String = { // We are explicitly not using multi-line string due to stripMargin removing "|" in output. s"-- !query $queryIndex\n" + @@ -146,7 +150,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** A test case. */ - private trait TestCase { + protected trait TestCase { val name: String val inputFile: String val resultFile: String @@ -156,35 +160,35 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, * PgSQL tests require to register some UDF functions. */ - private trait PgSQLTest + protected trait PgSQLTest - private trait UDFTest { + protected trait UDFTest { val udf: TestUDF } /** A regular test case. */ - private case class RegularTestCase( + protected case class RegularTestCase( name: String, inputFile: String, resultFile: String) extends TestCase /** A PostgreSQL test case. */ - private case class PgSQLTestCase( + protected case class PgSQLTestCase( name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest /** A UDF test case. */ - private case class UDFTestCase( + protected case class UDFTestCase( name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase with UDFTest /** A UDF PostgreSQL test case. */ - private case class UDFPgSQLTestCase( + protected case class UDFPgSQLTestCase( name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase with UDFTest with PgSQLTest - private def createScalaTestCase(testCase: TestCase): Unit = { + protected def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. @@ -222,7 +226,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** Run a test case. */ - private def runTest(testCase: TestCase): Unit = { + protected def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) @@ -235,7 +239,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // When we are regenerating the golden files, we don't need to set any config as they // all need to return the same result - if (regenerateGoldenFiles) { + if (regenerateGoldenFiles || !isTestWithConfigSets) { runQueries(queries, testCase, None) } else { val configSets = { @@ -271,7 +275,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - private def runQueries( + protected def runQueries( queries: Seq[String], testCase: TestCase, configSet: Option[Seq[(String, String)]]): Unit = { @@ -388,19 +392,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { try { val df = session.sql(sql) val schema = df.schema - val notIncludedMsg = "[not included in comparison]" - val clsName = this.getClass.getCanonicalName // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = hiveResultString(df.queryExecution.executedPlan) - .map(_.replaceAll("#\\d+", "#x") - .replaceAll( - s"Location.*/sql/core/spark-warehouse/$clsName/", - s"Location ${notIncludedMsg}sql/core/spark-warehouse/") - .replaceAll("Created By.*", s"Created By $notIncludedMsg") - .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") - .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") - .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") - .replaceAll("\\*\\(\\d+\\) ", "*")) // remove the WholeStageCodegen codegenStageIds + val answer = hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) // If the output is not pre-sorted, sort it. if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) @@ -418,7 +411,19 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - private def listTestCases(): Seq[TestCase] = { + protected def replaceNotIncludedMsg(line: String): String = { + line.replaceAll("#\\d+", "#x") + .replaceAll( + s"Location.*/sql/core/spark-warehouse/$clsName/", + s"Location ${notIncludedMsg}sql/core/spark-warehouse/") + .replaceAll("Created By.*", s"Created By $notIncludedMsg") + .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") + .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") + .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") + .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds + } + + protected def listTestCases(): Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" val absPath = file.getAbsolutePath @@ -444,7 +449,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** Returns all the files (not directories) in a directory, recursively. */ - private def listFilesRecursively(path: File): Seq[File] = { + protected def listFilesRecursively(path: File): Seq[File] = { val (dirs, files) = path.listFiles().partition(_.isDirectory) // Filter out test files with invalid extensions such as temp files created // by vi (.swp), Mac (.DS_Store) etc. diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 1abc65ad806bb..5b1352adddd89 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -47,6 +47,13 @@ test-jar test + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-hive_${scala.binary.version} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index b4d1d0d58aad6..abb53cf3429ff 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -53,7 +53,7 @@ object HiveThriftServer2 extends Logging { * Starts a new thrift server with the given context. */ @DeveloperApi - def startWithContext(sqlContext: SQLContext): Unit = { + def startWithContext(sqlContext: SQLContext): HiveThriftServer2 = { val server = new HiveThriftServer2(sqlContext) val executionHive = HiveUtils.newClientForExecution( @@ -69,6 +69,7 @@ object HiveThriftServer2 extends Logging { } else { None } + server } def main(args: Array[String]) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala new file mode 100644 index 0000000000000..ba3284462b460 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -0,0 +1,362 @@ +/* + * 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.thriftserver + +import java.io.File +import java.sql.{DriverManager, SQLException, Statement, Timestamp} +import java.util.Locale + +import scala.util.{Random, Try} +import scala.util.control.NonFatal + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.HiveSQLException + +import org.apache.spark.sql.{AnalysisException, SQLQueryTestSuite} +import org.apache.spark.sql.catalyst.util.fileToString +import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Re-run all the tests in SQLQueryTestSuite via Thrift Server. + * + * TODO: + * 1. Support UDF testing. + * 2. Support DESC command. + * 3. Support SHOW command. + */ +class ThriftServerQueryTestSuite extends SQLQueryTestSuite { + + private var hiveServer2: HiveThriftServer2 = _ + + override def beforeAll(): Unit = { + super.beforeAll() + // Chooses a random port between 10000 and 19999 + var listeningPort = 10000 + Random.nextInt(10000) + + // Retries up to 3 times with different port numbers if the server fails to start + (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) => + started.orElse { + listeningPort += 1 + Try(startThriftServer(listeningPort, attempt)) + } + }.recover { + case cause: Throwable => + throw cause + }.get + logInfo(s"HiveThriftServer2 started successfully") + } + + override def afterAll(): Unit = { + try { + hiveServer2.stop() + } finally { + super.afterAll() + } + } + + override val isTestWithConfigSets = false + + /** List of test cases to ignore, in lower cases. */ + override def blackList: Set[String] = Set( + "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + // Missing UDF + "pgSQL/boolean.sql", + "pgSQL/case.sql", + // SPARK-28624 + "date.sql", + // SPARK-28619 + "pgSQL/aggregates_part1.sql", + "group-by.sql", + // SPARK-28620 + "pgSQL/float4.sql", + // SPARK-28636 + "decimalArithmeticOperations.sql", + "literals.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/simple-in.sql", + "subquery/in-subquery/in-order-by.sql", + "subquery/in-subquery/in-set-operations.sql", + // SPARK-28637 + "cast.sql", + "ansi/interval.sql" + ) + + override def runQueries( + queries: Seq[String], + testCase: TestCase, + configSet: Option[Seq[(String, String)]]): Unit = { + // We do not test with configSet. + withJdbcStatement { statement => + + loadTestData(statement) + + testCase match { + case _: PgSQLTest => + // PostgreSQL enabled cartesian product by default. + statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.ANSI_SQL_PARSER.key} = true") + statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") + case _ => + } + + // Run the SQL queries preparing them for comparison. + val outputs: Seq[QueryOutput] = queries.map { sql => + val output = getNormalizedResult(statement, sql) + // We might need to do some query canonicalization in the future. + QueryOutput( + sql = sql, + schema = "", + output = output.mkString("\n").replaceAll("\\s+$", "")) + } + + // Read back the golden file. + val expectedOutputs: Seq[QueryOutput] = { + val goldenOutput = fileToString(new File(testCase.resultFile)) + val segments = goldenOutput.split("-- !query.+\n") + + // each query has 3 segments, plus the header + assert(segments.size == outputs.size * 3 + 1, + s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " + + s"Try regenerate the result files.") + Seq.tabulate(outputs.size) { i => + val sql = segments(i * 3 + 1).trim + val originalOut = segments(i * 3 + 3) + val output = if (isNeedSort(sql)) { + originalOut.split("\n").sorted.mkString("\n") + } else { + originalOut + } + QueryOutput( + sql = sql, + schema = "", + output = output.replaceAll("\\s+$", "") + ) + } + } + + // Compare results. + assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { + outputs.size + } + + outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) => + assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { + output.sql + } + + expected match { + // Skip desc command, see HiveResult.hiveResultString + case d if d.sql.toUpperCase(Locale.ROOT).startsWith("DESC ") + || d.sql.toUpperCase(Locale.ROOT).startsWith("DESC\n") + || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE ") + || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE\n") => + // Skip show command, see HiveResult.hiveResultString + case s if s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW ") + || s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW\n") => + // AnalysisException should exactly match. + // SQLException should not exactly match. We only assert the result contains Exception. + case _ if output.output.startsWith(classOf[SQLException].getName) => + assert(expected.output.contains("Exception"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + // HiveSQLException is usually a feature that our ThriftServer cannot support. + // Please add SQL to blackList. + case _ if output.output.startsWith(classOf[HiveSQLException].getName) => + assert(false, s"${output.output} for query #$i\n${expected.sql}") + case _ => + assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { + output.output + } + } + } + } + } + + override def createScalaTestCase(testCase: TestCase): Unit = { + if (blackList.exists(t => + testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { + // Create a test case to ignore this case. + ignore(testCase.name) { /* Do nothing */ } + } else { + // Create a test case to run this case. + test(testCase.name) { + runTest(testCase) + } + } + } + + override def listTestCases(): Seq[TestCase] = { + listFilesRecursively(new File(inputFilePath)).flatMap { file => + val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" + val absPath = file.getAbsolutePath + val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) + + if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { + Seq.empty + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { + PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else { + RegularTestCase(testCaseName, absPath, resultFile) :: Nil + } + } + } + + test("Check if ThriftServer can work") { + withJdbcStatement { statement => + val rs = statement.executeQuery("select 1L") + rs.next() + assert(rs.getLong(1) === 1L) + } + } + + private def getNormalizedResult(statement: Statement, sql: String): Seq[String] = { + try { + val rs = statement.executeQuery(sql) + val cols = rs.getMetaData.getColumnCount + val buildStr = () => (for (i <- 1 to cols) yield { + getHiveResult(rs.getObject(i)) + }).mkString("\t") + + val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq + .map(replaceNotIncludedMsg) + if (isNeedSort(sql)) { + answer.sorted + } else { + answer + } + } catch { + case a: AnalysisException => + // Do not output the logical plan tree which contains expression IDs. + // Also implement a crude way of masking expression IDs in the error message + // with a generic pattern "###". + val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage + Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x")).sorted + case NonFatal(e) => + // If there is an exception, put the exception class followed by the message. + Seq(e.getClass.getName, e.getMessage) + } + } + + private def startThriftServer(port: Int, attempt: Int): Unit = { + logInfo(s"Trying to start HiveThriftServer2: port=$port, attempt=$attempt") + val sqlContext = spark.newSession().sqlContext + sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, port.toString) + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + } + + private def withJdbcStatement(fs: (Statement => Unit)*) { + val user = System.getProperty("user.name") + + val serverPort = hiveServer2.getHiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname) + val connections = + fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } + + /** Load built-in test tables. */ + private def loadTestData(statement: Statement): Unit = { + // Prepare the data + statement.execute( + """ + |CREATE OR REPLACE TEMPORARY VIEW testdata as + |SELECT id AS key, CAST(id AS string) AS value FROM range(1, 101) + """.stripMargin) + statement.execute( + """ + |CREATE OR REPLACE TEMPORARY VIEW arraydata as + |SELECT * FROM VALUES + |(ARRAY(1, 2, 3), ARRAY(ARRAY(1, 2, 3))), + |(ARRAY(2, 3, 4), ARRAY(ARRAY(2, 3, 4))) AS v(arraycol, nestedarraycol) + """.stripMargin) + statement.execute( + """ + |CREATE OR REPLACE TEMPORARY VIEW mapdata as + |SELECT * FROM VALUES + |MAP(1, 'a1', 2, 'b1', 3, 'c1', 4, 'd1', 5, 'e1'), + |MAP(1, 'a2', 2, 'b2', 3, 'c2', 4, 'd2'), + |MAP(1, 'a3', 2, 'b3', 3, 'c3'), + |MAP(1, 'a4', 2, 'b4'), + |MAP(1, 'a5') AS v(mapcol) + """.stripMargin) + statement.execute( + s""" + |CREATE TEMPORARY VIEW aggtest + | (a int, b float) + |USING csv + |OPTIONS (path '${testFile("test-data/postgresql/agg.data")}', + | header 'false', delimiter '\t') + """.stripMargin) + statement.execute( + s""" + |CREATE OR REPLACE TEMPORARY VIEW onek + | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, + | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, + | stringu1 string, stringu2 string, string4 string) + |USING csv + |OPTIONS (path '${testFile("test-data/postgresql/onek.data")}', + | header 'false', delimiter '\t') + """.stripMargin) + statement.execute( + s""" + |CREATE OR REPLACE TEMPORARY VIEW tenk1 + | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, + | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, + | stringu1 string, stringu2 string, string4 string) + |USING csv + | OPTIONS (path '${testFile("test-data/postgresql/tenk.data")}', + | header 'false', delimiter '\t') + """.stripMargin) + } + + // Returns true if sql is retrieving data. + private def isNeedSort(sql: String): Boolean = { + val upperCase = sql.toUpperCase(Locale.ROOT) + upperCase.startsWith("SELECT ") || upperCase.startsWith("SELECT\n") || + upperCase.startsWith("WITH ") || upperCase.startsWith("WITH\n") || + upperCase.startsWith("VALUES ") || upperCase.startsWith("VALUES\n") || + // pgSQL/union.sql + upperCase.startsWith("(") + } + + private def getHiveResult(obj: Object): String = { + obj match { + case null => + HiveResult.toHiveString((null, StringType)) + case d: java.sql.Date => + HiveResult.toHiveString((d, DateType)) + case t: Timestamp => + HiveResult.toHiveString((t, TimestampType)) + case d: java.math.BigDecimal => + HiveResult.toHiveString((d, DecimalType.fromBigDecimal(d))) + case bin: Array[Byte] => + HiveResult.toHiveString((bin, BinaryType)) + case other => + other.toString + } + } +} From c308ab5a29c3f9f92bc73ac6ae984b465b177e97 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 18 Aug 2019 08:36:01 -0700 Subject: [PATCH 093/149] [MINOR][SQL] Make analysis error msg more meaningful on DISTINCT queries ## What changes were proposed in this pull request? This PR makes analysis error messages more meaningful when the function does not support the modifier DISTINCT: ```sql postgres=# select upper(distinct a) from (values('a'), ('b')) v(a); ERROR: DISTINCT specified, but upper is not an aggregate function LINE 1: select upper(distinct a) from (values('a'), ('b')) v(a); spark-sql> select upper(distinct a) from (values('a'), ('b')) v(a); Error in query: upper does not support the modifier DISTINCT; line 1 pos 7 spark-sql> ``` After this pr: ```sql spark-sql> select upper(distinct a) from (values('a'), ('b')) v(a); Error in query: DISTINCT specified, but upper is not an aggregate function; line 1 pos 7 spark-sql> ``` ## How was this patch tested? Unit test Closes #25486 from wangyum/DISTINCT. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- .../spark/sql/catalyst/analysis/AnalysisErrorSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 4 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 3a72988f8345d..a080a1b353e26 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 @@ -1647,7 +1647,8 @@ class Analyzer( // AggregateExpression. case wf: AggregateWindowFunction => if (isDistinct) { - failAnalysis(s"${wf.prettyName} does not support the modifier DISTINCT") + failAnalysis( + s"DISTINCT specified, but ${wf.prettyName} is not an aggregate function") } else { wf } @@ -1656,7 +1657,8 @@ class Analyzer( // This function is not an aggregate function, just return the resolved one. case other => if (isDistinct) { - failAnalysis(s"${other.prettyName} does not support the modifier DISTINCT") + failAnalysis( + s"DISTINCT specified, but ${other.prettyName} is not an aggregate function") } else { other } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 1cc5dd8ce1d54..f0356f5a42d67 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -165,12 +165,12 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "distinct function", CatalystSqlParser.parsePlan("SELECT hex(DISTINCT a) FROM TaBlE"), - "hex does not support the modifier DISTINCT" :: Nil) + "DISTINCT specified, but hex is not an aggregate function" :: Nil) errorTest( "distinct window function", CatalystSqlParser.parsePlan("SELECT percent_rank(DISTINCT a) over () FROM TaBlE"), - "percent_rank does not support the modifier DISTINCT" :: Nil) + "DISTINCT specified, but percent_rank is not an aggregate function" :: Nil) errorTest( "nested aggregate functions", From c097c555acd0c28d71a223e706e79f4dcf521f3b Mon Sep 17 00:00:00 2001 From: Yizhong Zhang Date: Sun, 18 Aug 2019 15:55:43 -0500 Subject: [PATCH 094/149] [SPARK-21067][DOC] Fix Thrift Server - CTAS fail with Unable to move source ## What changes were proposed in this pull request? This PR aims to fix CTAS fails after we closed a session of ThriftServer. - sql-distributed-sql-engine.md ![image](https://user-images.githubusercontent.com/25916266/62509628-6f854980-b83e-11e9-9bea-daaf76c8f724.png) It seems the simplest way to fix [[SPARK-21067]](https://issues.apache.org/jira/browse/SPARK-21067). For example : If we use HDFS, we can set the following property in hive-site.xml. `` ` fs.hdfs.impl.disable.cache` ` true` `` ## How was this patch tested Manual. Closes #25364 from Deegue/fix_add_doc_file_system. Authored-by: Yizhong Zhang Signed-off-by: Sean Owen --- docs/sql-distributed-sql-engine.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md index fc849d3912b98..13be6d51a7ece 100644 --- a/docs/sql-distributed-sql-engine.md +++ b/docs/sql-distributed-sql-engine.md @@ -85,6 +85,8 @@ To test, use beeline to connect to the JDBC/ODBC server in http mode with: beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= +If you closed a session and do CTAS, you must set `fs.%s.impl.disable.cache` to true in `hive-site.xml`. +See more details in [[SPARK-21067]](https://issues.apache.org/jira/browse/SPARK-21067). ## Running the Spark SQL CLI From f0834d3a7f945759bbbcba970d24a49ba1d08421 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 18 Aug 2019 16:54:24 -0700 Subject: [PATCH 095/149] Revert "[SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server" This reverts commit efbb0359024d1f77e207ca5fc78b3c32b4138558. --- project/SparkBuild.scala | 3 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 67 ++-- sql/hive-thriftserver/pom.xml | 7 - .../hive/thriftserver/HiveThriftServer2.scala | 3 +- .../ThriftServerQueryTestSuite.scala | 362 ------------------ 5 files changed, 33 insertions(+), 409 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 25c2fb4af5c34..f1d2e3788918f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -474,8 +474,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite", "org.apache.spark.ml.classification.LogisticRegressionSuite", "org.apache.spark.ml.classification.LinearSVCSuite", - "org.apache.spark.sql.SQLQueryTestSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite" + "org.apache.spark.sql.SQLQueryTestSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" 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 5c1ff9cd735eb..4bdf25051127c 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 @@ -107,9 +107,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { import IntegratedUDFTestUtils._ private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - protected val isTestWithConfigSets: Boolean = true - protected val baseResourcePath = { + private val baseResourcePath = { // If regenerateGoldenFiles is true, we must be running this in SBT and we use hard-coded // relative path. Otherwise, we use classloader's getResource to find the location. if (regenerateGoldenFiles) { @@ -120,16 +119,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - protected val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath - protected val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + private val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + private val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath - protected val validFileExtensions = ".sql" - - private val notIncludedMsg = "[not included in comparison]" - private val clsName = this.getClass.getCanonicalName + private val validFileExtensions = ".sql" /** List of test cases to ignore, in lower cases. */ - protected def blackList: Set[String] = Set( + private val blackList = Set( "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. ) @@ -137,7 +133,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { listTestCases().foreach(createScalaTestCase) /** A single SQL query's output. */ - protected case class QueryOutput(sql: String, schema: String, output: String) { + private case class QueryOutput(sql: String, schema: String, output: String) { def toString(queryIndex: Int): String = { // We are explicitly not using multi-line string due to stripMargin removing "|" in output. s"-- !query $queryIndex\n" + @@ -150,7 +146,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** A test case. */ - protected trait TestCase { + private trait TestCase { val name: String val inputFile: String val resultFile: String @@ -160,35 +156,35 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, * PgSQL tests require to register some UDF functions. */ - protected trait PgSQLTest + private trait PgSQLTest - protected trait UDFTest { + private trait UDFTest { val udf: TestUDF } /** A regular test case. */ - protected case class RegularTestCase( + private case class RegularTestCase( name: String, inputFile: String, resultFile: String) extends TestCase /** A PostgreSQL test case. */ - protected case class PgSQLTestCase( + private case class PgSQLTestCase( name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest /** A UDF test case. */ - protected case class UDFTestCase( + private case class UDFTestCase( name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase with UDFTest /** A UDF PostgreSQL test case. */ - protected case class UDFPgSQLTestCase( + private case class UDFPgSQLTestCase( name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase with UDFTest with PgSQLTest - protected def createScalaTestCase(testCase: TestCase): Unit = { + private def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. @@ -226,7 +222,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** Run a test case. */ - protected def runTest(testCase: TestCase): Unit = { + private def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) @@ -239,7 +235,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // When we are regenerating the golden files, we don't need to set any config as they // all need to return the same result - if (regenerateGoldenFiles || !isTestWithConfigSets) { + if (regenerateGoldenFiles) { runQueries(queries, testCase, None) } else { val configSets = { @@ -275,7 +271,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - protected def runQueries( + private def runQueries( queries: Seq[String], testCase: TestCase, configSet: Option[Seq[(String, String)]]): Unit = { @@ -392,8 +388,19 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { try { val df = session.sql(sql) val schema = df.schema + val notIncludedMsg = "[not included in comparison]" + val clsName = this.getClass.getCanonicalName // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + val answer = hiveResultString(df.queryExecution.executedPlan) + .map(_.replaceAll("#\\d+", "#x") + .replaceAll( + s"Location.*/sql/core/spark-warehouse/$clsName/", + s"Location ${notIncludedMsg}sql/core/spark-warehouse/") + .replaceAll("Created By.*", s"Created By $notIncludedMsg") + .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") + .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") + .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") + .replaceAll("\\*\\(\\d+\\) ", "*")) // remove the WholeStageCodegen codegenStageIds // If the output is not pre-sorted, sort it. if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) @@ -411,19 +418,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } - protected def replaceNotIncludedMsg(line: String): String = { - line.replaceAll("#\\d+", "#x") - .replaceAll( - s"Location.*/sql/core/spark-warehouse/$clsName/", - s"Location ${notIncludedMsg}sql/core/spark-warehouse/") - .replaceAll("Created By.*", s"Created By $notIncludedMsg") - .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") - .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") - .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") - .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds - } - - protected def listTestCases(): Seq[TestCase] = { + private def listTestCases(): Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" val absPath = file.getAbsolutePath @@ -449,7 +444,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } /** Returns all the files (not directories) in a directory, recursively. */ - protected def listFilesRecursively(path: File): Seq[File] = { + private def listFilesRecursively(path: File): Seq[File] = { val (dirs, files) = path.listFiles().partition(_.isDirectory) // Filter out test files with invalid extensions such as temp files created // by vi (.swp), Mac (.DS_Store) etc. diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5b1352adddd89..1abc65ad806bb 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -47,13 +47,6 @@ test-jar test - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - test-jar - test - org.apache.spark spark-hive_${scala.binary.version} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index abb53cf3429ff..b4d1d0d58aad6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -53,7 +53,7 @@ object HiveThriftServer2 extends Logging { * Starts a new thrift server with the given context. */ @DeveloperApi - def startWithContext(sqlContext: SQLContext): HiveThriftServer2 = { + def startWithContext(sqlContext: SQLContext): Unit = { val server = new HiveThriftServer2(sqlContext) val executionHive = HiveUtils.newClientForExecution( @@ -69,7 +69,6 @@ object HiveThriftServer2 extends Logging { } else { None } - server } def main(args: Array[String]) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala deleted file mode 100644 index ba3284462b460..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ /dev/null @@ -1,362 +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.hive.thriftserver - -import java.io.File -import java.sql.{DriverManager, SQLException, Statement, Timestamp} -import java.util.Locale - -import scala.util.{Random, Try} -import scala.util.control.NonFatal - -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.HiveSQLException - -import org.apache.spark.sql.{AnalysisException, SQLQueryTestSuite} -import org.apache.spark.sql.catalyst.util.fileToString -import org.apache.spark.sql.execution.HiveResult -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ - -/** - * Re-run all the tests in SQLQueryTestSuite via Thrift Server. - * - * TODO: - * 1. Support UDF testing. - * 2. Support DESC command. - * 3. Support SHOW command. - */ -class ThriftServerQueryTestSuite extends SQLQueryTestSuite { - - private var hiveServer2: HiveThriftServer2 = _ - - override def beforeAll(): Unit = { - super.beforeAll() - // Chooses a random port between 10000 and 19999 - var listeningPort = 10000 + Random.nextInt(10000) - - // Retries up to 3 times with different port numbers if the server fails to start - (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) => - started.orElse { - listeningPort += 1 - Try(startThriftServer(listeningPort, attempt)) - } - }.recover { - case cause: Throwable => - throw cause - }.get - logInfo(s"HiveThriftServer2 started successfully") - } - - override def afterAll(): Unit = { - try { - hiveServer2.stop() - } finally { - super.afterAll() - } - } - - override val isTestWithConfigSets = false - - /** List of test cases to ignore, in lower cases. */ - override def blackList: Set[String] = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. - // Missing UDF - "pgSQL/boolean.sql", - "pgSQL/case.sql", - // SPARK-28624 - "date.sql", - // SPARK-28619 - "pgSQL/aggregates_part1.sql", - "group-by.sql", - // SPARK-28620 - "pgSQL/float4.sql", - // SPARK-28636 - "decimalArithmeticOperations.sql", - "literals.sql", - "subquery/scalar-subquery/scalar-subquery-predicate.sql", - "subquery/in-subquery/in-limit.sql", - "subquery/in-subquery/simple-in.sql", - "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql", - // SPARK-28637 - "cast.sql", - "ansi/interval.sql" - ) - - override def runQueries( - queries: Seq[String], - testCase: TestCase, - configSet: Option[Seq[(String, String)]]): Unit = { - // We do not test with configSet. - withJdbcStatement { statement => - - loadTestData(statement) - - testCase match { - case _: PgSQLTest => - // PostgreSQL enabled cartesian product by default. - statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.ANSI_SQL_PARSER.key} = true") - statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") - case _ => - } - - // Run the SQL queries preparing them for comparison. - val outputs: Seq[QueryOutput] = queries.map { sql => - val output = getNormalizedResult(statement, sql) - // We might need to do some query canonicalization in the future. - QueryOutput( - sql = sql, - schema = "", - output = output.mkString("\n").replaceAll("\\s+$", "")) - } - - // Read back the golden file. - val expectedOutputs: Seq[QueryOutput] = { - val goldenOutput = fileToString(new File(testCase.resultFile)) - val segments = goldenOutput.split("-- !query.+\n") - - // each query has 3 segments, plus the header - assert(segments.size == outputs.size * 3 + 1, - s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " + - s"Try regenerate the result files.") - Seq.tabulate(outputs.size) { i => - val sql = segments(i * 3 + 1).trim - val originalOut = segments(i * 3 + 3) - val output = if (isNeedSort(sql)) { - originalOut.split("\n").sorted.mkString("\n") - } else { - originalOut - } - QueryOutput( - sql = sql, - schema = "", - output = output.replaceAll("\\s+$", "") - ) - } - } - - // Compare results. - assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { - outputs.size - } - - outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) => - assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { - output.sql - } - - expected match { - // Skip desc command, see HiveResult.hiveResultString - case d if d.sql.toUpperCase(Locale.ROOT).startsWith("DESC ") - || d.sql.toUpperCase(Locale.ROOT).startsWith("DESC\n") - || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE ") - || d.sql.toUpperCase(Locale.ROOT).startsWith("DESCRIBE\n") => - // Skip show command, see HiveResult.hiveResultString - case s if s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW ") - || s.sql.toUpperCase(Locale.ROOT).startsWith("SHOW\n") => - // AnalysisException should exactly match. - // SQLException should not exactly match. We only assert the result contains Exception. - case _ if output.output.startsWith(classOf[SQLException].getName) => - assert(expected.output.contains("Exception"), - s"Exception did not match for query #$i\n${expected.sql}, " + - s"expected: ${expected.output}, but got: ${output.output}") - // HiveSQLException is usually a feature that our ThriftServer cannot support. - // Please add SQL to blackList. - case _ if output.output.startsWith(classOf[HiveSQLException].getName) => - assert(false, s"${output.output} for query #$i\n${expected.sql}") - case _ => - assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { - output.output - } - } - } - } - } - - override def createScalaTestCase(testCase: TestCase): Unit = { - if (blackList.exists(t => - testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { - // Create a test case to ignore this case. - ignore(testCase.name) { /* Do nothing */ } - } else { - // Create a test case to run this case. - test(testCase.name) { - runTest(testCase) - } - } - } - - override def listTestCases(): Seq[TestCase] = { - listFilesRecursively(new File(inputFilePath)).flatMap { file => - val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" - val absPath = file.getAbsolutePath - val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) - - if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { - Seq.empty - } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { - PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil - } else { - RegularTestCase(testCaseName, absPath, resultFile) :: Nil - } - } - } - - test("Check if ThriftServer can work") { - withJdbcStatement { statement => - val rs = statement.executeQuery("select 1L") - rs.next() - assert(rs.getLong(1) === 1L) - } - } - - private def getNormalizedResult(statement: Statement, sql: String): Seq[String] = { - try { - val rs = statement.executeQuery(sql) - val cols = rs.getMetaData.getColumnCount - val buildStr = () => (for (i <- 1 to cols) yield { - getHiveResult(rs.getObject(i)) - }).mkString("\t") - - val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq - .map(replaceNotIncludedMsg) - if (isNeedSort(sql)) { - answer.sorted - } else { - answer - } - } catch { - case a: AnalysisException => - // Do not output the logical plan tree which contains expression IDs. - // Also implement a crude way of masking expression IDs in the error message - // with a generic pattern "###". - val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage - Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x")).sorted - case NonFatal(e) => - // If there is an exception, put the exception class followed by the message. - Seq(e.getClass.getName, e.getMessage) - } - } - - private def startThriftServer(port: Int, attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2: port=$port, attempt=$attempt") - val sqlContext = spark.newSession().sqlContext - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, port.toString) - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - } - - private def withJdbcStatement(fs: (Statement => Unit)*) { - val user = System.getProperty("user.name") - - val serverPort = hiveServer2.getHiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname) - val connections = - fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } - val statements = connections.map(_.createStatement()) - - try { - statements.zip(fs).foreach { case (s, f) => f(s) } - } finally { - statements.foreach(_.close()) - connections.foreach(_.close()) - } - } - - /** Load built-in test tables. */ - private def loadTestData(statement: Statement): Unit = { - // Prepare the data - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW testdata as - |SELECT id AS key, CAST(id AS string) AS value FROM range(1, 101) - """.stripMargin) - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW arraydata as - |SELECT * FROM VALUES - |(ARRAY(1, 2, 3), ARRAY(ARRAY(1, 2, 3))), - |(ARRAY(2, 3, 4), ARRAY(ARRAY(2, 3, 4))) AS v(arraycol, nestedarraycol) - """.stripMargin) - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW mapdata as - |SELECT * FROM VALUES - |MAP(1, 'a1', 2, 'b1', 3, 'c1', 4, 'd1', 5, 'e1'), - |MAP(1, 'a2', 2, 'b2', 3, 'c2', 4, 'd2'), - |MAP(1, 'a3', 2, 'b3', 3, 'c3'), - |MAP(1, 'a4', 2, 'b4'), - |MAP(1, 'a5') AS v(mapcol) - """.stripMargin) - statement.execute( - s""" - |CREATE TEMPORARY VIEW aggtest - | (a int, b float) - |USING csv - |OPTIONS (path '${testFile("test-data/postgresql/agg.data")}', - | header 'false', delimiter '\t') - """.stripMargin) - statement.execute( - s""" - |CREATE OR REPLACE TEMPORARY VIEW onek - | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, - | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, - | stringu1 string, stringu2 string, string4 string) - |USING csv - |OPTIONS (path '${testFile("test-data/postgresql/onek.data")}', - | header 'false', delimiter '\t') - """.stripMargin) - statement.execute( - s""" - |CREATE OR REPLACE TEMPORARY VIEW tenk1 - | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, - | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, - | stringu1 string, stringu2 string, string4 string) - |USING csv - | OPTIONS (path '${testFile("test-data/postgresql/tenk.data")}', - | header 'false', delimiter '\t') - """.stripMargin) - } - - // Returns true if sql is retrieving data. - private def isNeedSort(sql: String): Boolean = { - val upperCase = sql.toUpperCase(Locale.ROOT) - upperCase.startsWith("SELECT ") || upperCase.startsWith("SELECT\n") || - upperCase.startsWith("WITH ") || upperCase.startsWith("WITH\n") || - upperCase.startsWith("VALUES ") || upperCase.startsWith("VALUES\n") || - // pgSQL/union.sql - upperCase.startsWith("(") - } - - private def getHiveResult(obj: Object): String = { - obj match { - case null => - HiveResult.toHiveString((null, StringType)) - case d: java.sql.Date => - HiveResult.toHiveString((d, DateType)) - case t: Timestamp => - HiveResult.toHiveString((t, TimestampType)) - case d: java.math.BigDecimal => - HiveResult.toHiveString((d, DecimalType.fromBigDecimal(d))) - case bin: Array[Byte] => - HiveResult.toHiveString((bin, BinaryType)) - case other => - other.toString - } - } -} From 4ddad7906098ccde8d918f42afe856d9fe23b563 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 19 Aug 2019 11:41:13 +0900 Subject: [PATCH 096/149] [SPARK-28598][SQL] Few date time manipulation functions does not provide versions supporting Column as input through the Dataframe API ## What changes were proposed in this pull request? Add following functions: ``` def add_months(startDate: Column, numMonths: Column): Column def date_add(start: Column, days: Column): Column def date_sub(start: Column, days: Column): Column ``` ## How was this patch tested? UT. Please review https://spark.apache.org/contributing.html before opening a pull request. Closes #25334 from WeichenXu123/datefunc_impr. Authored-by: WeichenXu Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/functions.scala | 46 +++++++++++++++++-- .../apache/spark/sql/DateFunctionsSuite.scala | 11 +++++ 2 files changed, 53 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index afafde114a3ef..6b8127bab1cb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2594,8 +2594,21 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def add_months(startDate: Column, numMonths: Int): Column = withExpr { - AddMonths(startDate.expr, Literal(numMonths)) + def add_months(startDate: Column, numMonths: Int): Column = add_months(startDate, lit(numMonths)) + + /** + * Returns the date that is `numMonths` after `startDate`. + * + * @param startDate A date, timestamp or string. If a string, the data must be in a format that + * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param numMonths A column of the number of months to add to `startDate`, can be negative to + * subtract months + * @return A date, or null if `startDate` was a string that could not be cast to a date + * @group datetime_funcs + * @since 3.0.0 + */ + def add_months(startDate: Column, numMonths: Column): Column = withExpr { + AddMonths(startDate.expr, numMonths.expr) } /** @@ -2644,7 +2657,19 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def date_add(start: Column, days: Int): Column = withExpr { DateAdd(start.expr, Literal(days)) } + def date_add(start: Column, days: Int): Column = date_add(start, lit(days)) + + /** + * Returns the date that is `days` days after `start` + * + * @param start A date, timestamp or string. If a string, the data must be in a format that + * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days A column of the number of days to add to `start`, can be negative to subtract days + * @return A date, or null if `start` was a string that could not be cast to a date + * @group datetime_funcs + * @since 3.0.0 + */ + def date_add(start: Column, days: Column): Column = withExpr { DateAdd(start.expr, days.expr) } /** * Returns the date that is `days` days before `start` @@ -2656,7 +2681,20 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def date_sub(start: Column, days: Int): Column = withExpr { DateSub(start.expr, Literal(days)) } + def date_sub(start: Column, days: Int): Column = date_sub(start, lit(days)) + + /** + * Returns the date that is `days` days before `start` + * + * @param start A date, timestamp or string. If a string, the data must be in a format that + * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days A column of the number of days to subtract from `start`, can be negative to add + * days + * @return A date, or null if `start` was a string that could not be cast to a date + * @group datetime_funcs + * @since 3.0.0 + */ + def date_sub(start: Column, days: Column): Column = withExpr { DateSub(start.expr, days.expr) } /** * Returns the number of days from `start` to `end`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 69f17f5e9c2dd..a92c4177da1a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -239,6 +239,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { df.select(date_add(col("ss"), 7)), Seq(Row(Date.valueOf("2015-06-08")), Row(Date.valueOf("2015-06-09")))) + checkAnswer( + df.withColumn("x", lit(1)).select(date_add(col("d"), col("x"))), + Seq(Row(Date.valueOf("2015-06-02")), Row(Date.valueOf("2015-06-03")))) + checkAnswer(df.selectExpr("DATE_ADD(null, 1)"), Seq(Row(null), Row(null))) checkAnswer( df.selectExpr("""DATE_ADD(d, 1)"""), @@ -270,6 +274,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer( df.select(date_sub(lit(null), 1)).limit(1), Row(null)) + checkAnswer( + df.withColumn("x", lit(1)).select(date_sub(col("d"), col("x"))), + Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01")))) + checkAnswer(df.selectExpr("""DATE_SUB(d, null)"""), Seq(Row(null), Row(null))) checkAnswer( df.selectExpr("""DATE_SUB(d, 1)"""), @@ -318,6 +326,9 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer( df.selectExpr("add_months(d, -1)"), Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-28")))) + checkAnswer( + df.withColumn("x", lit(1)).select(add_months(col("d"), col("x"))), + Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-28")))) } test("function months_between") { From c96b6154b7fa9425130f7675e423aced185bd1d7 Mon Sep 17 00:00:00 2001 From: shivusondur Date: Mon, 19 Aug 2019 13:01:39 +0900 Subject: [PATCH 097/149] [SPARK-28390][SQL][PYTHON][TESTS][FOLLOW-UP] Update the TODO with actual blocking JIRA IDs ## What changes were proposed in this pull request? only todo message updated. Need to add udf() for GroupBy Tests, after resolving following jira [SPARK-28386] and [SPARK-26741] ## How was this patch tested? NA, only TODO message updated. Closes #25415 from shivusondur/jiraFollowup. Authored-by: shivusondur Signed-off-by: HyukjinKwon --- .../resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql index f52aa669e89db..c8e4346cedb89 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql @@ -6,7 +6,7 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql -- -- This test file was converted from inputs/pgSQL/select_having.sql --- TODO: We should add UDFs in GROUP BY clause when [SPARK-28445] is resolved. +-- TODO: We should add UDFs in GROUP BY clause when [SPARK-28386] and [SPARK-26741] is resolved. -- load test data CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; From d75a11d0596516e73e27aea8f5a7e85df0102d5c Mon Sep 17 00:00:00 2001 From: Eyal Zituny Date: Mon, 19 Aug 2019 14:12:48 +0800 Subject: [PATCH 098/149] [SPARK-27330][SS] support task abort in foreach writer ## What changes were proposed in this pull request? in order to address cases where foreach writer task is failing without calling the close() method, (for example when a task is interrupted) added the option to implement an abort() method that will be called when the task is aborted. users should handle resource cleanup (such as connections) in the abort() method ## How was this patch tested? update existing unit tests. Closes #24382 from eyalzit/SPARK-27330-foreach-writer-abort. Lead-authored-by: Eyal Zituny Co-authored-by: Jungtaek Lim (HeartSaVioR) Co-authored-by: eyalzit Signed-off-by: Wenchen Fan --- .../sources/ForeachWriterTable.scala | 17 +++++++++-- .../sources/ForeachWriterSuite.scala | 30 +++++++++++++++++++ 2 files changed, 44 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 6da1b3a49c442..838c7d497e35b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -21,6 +21,7 @@ import java.util import scala.collection.JavaConverters._ +import org.apache.spark.SparkException import org.apache.spark.sql.{ForeachWriter, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -133,6 +134,7 @@ class ForeachDataWriter[T]( // If open returns false, we should skip writing rows. private val opened = writer.open(partitionId, epochId) + private var closeCalled: Boolean = false override def write(record: InternalRow): Unit = { if (!opened) return @@ -141,17 +143,26 @@ class ForeachDataWriter[T]( writer.process(rowConverter(record)) } catch { case t: Throwable => - writer.close(t) + closeWriter(t) throw t } } override def commit(): WriterCommitMessage = { - writer.close(null) + closeWriter(null) ForeachWriterCommitMessage } - override def abort(): Unit = {} + override def abort(): Unit = { + closeWriter(new SparkException("Foreach writer has been aborted due to a task failure")) + } + + private def closeWriter(errorOrNull: Throwable): Unit = { + if (!closeCalled) { + closeCalled = true + writer.close(errorOrNull) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index e60c339bc9cc1..1e7fa8e91cdff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -154,6 +154,8 @@ class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAnd val errorEvent = allEvents(0)(2).asInstanceOf[ForeachWriterSuite.Close] assert(errorEvent.error.get.isInstanceOf[RuntimeException]) assert(errorEvent.error.get.getMessage === "ForeachSinkSuite error") + // 'close' shouldn't be called with abort message if close with error has been called + assert(allEvents(0).size == 3) } } @@ -258,6 +260,34 @@ class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAnd query.stop() } } + + testQuietly("foreach with error not caused by ForeachWriter") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val query = input.toDS().repartition(1).map(_ / 0).writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreach(new TestForeachWriter) + .start() + input.addData(1, 2, 3, 4) + + val e = intercept[StreamingQueryException] { + query.processAllAvailable() + } + + assert(e.getCause.isInstanceOf[SparkException]) + assert(e.getCause.getCause.getCause.getMessage === "/ by zero") + assert(query.isActive === false) + + val allEvents = ForeachWriterSuite.allEvents() + assert(allEvents.size === 1) + assert(allEvents(0)(0) === ForeachWriterSuite.Open(partition = 0, version = 0)) + // `close` should be called with the error + val errorEvent = allEvents(0)(1).asInstanceOf[ForeachWriterSuite.Close] + assert(errorEvent.error.get.isInstanceOf[SparkException]) + assert(errorEvent.error.get.getMessage === + "Foreach writer has been aborted due to a task failure") + } + } } /** A global object to collect events in the executor */ From a5df5ff0fdec0e1a727cb7f2f9bed178d37b3ee5 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sun, 18 Aug 2019 23:17:50 -0700 Subject: [PATCH 099/149] [SPARK-28734][DOC] Initial table of content in the left hand side bar for SQL doc ## What changes were proposed in this pull request? This is a initial PR that creates the table of content for SQL reference guide. The left side bar will displays additional menu items corresponding to supported SQL constructs. One this PR is merged, we will fill in the content incrementally. Additionally this PR contains a minor change to make the left sidebar scrollable. Currently it is not possible to scroll in the left hand side window. ## How was this patch tested? Used jekyll build and serve to verify. Closes #25459 from dilipbiswal/ref-doc. Authored-by: Dilip Biswal Signed-off-by: gatorsmile --- docs/_data/menu-sql.yaml | 164 +++++++++++++++++- docs/css/main.css | 5 +- docs/sql-ref-arithmetic-ops.md | 22 +++ ...{sql-reference.md => sql-ref-datatypes.md} | 28 +-- docs/sql-ref-functions-builtin-aggregate.md | 22 +++ docs/sql-ref-functions-builtin-scalar.md | 22 +++ docs/sql-ref-functions-builtin.md | 25 +++ docs/sql-ref-functions-udf-aggregate.md | 22 +++ docs/sql-ref-functions-udf-scalar.md | 22 +++ docs/sql-ref-functions-udf.md | 25 +++ docs/sql-ref-functions.md | 25 +++ docs/sql-ref-nan-semantics.md | 29 ++++ docs/sql-ref-syntax-aux-analyze-table.md | 22 +++ docs/sql-ref-syntax-aux-analyze.md | 25 +++ docs/sql-ref-syntax-aux-cache-cache-table.md | 22 +++ docs/sql-ref-syntax-aux-cache-clear-cache.md | 22 +++ .../sql-ref-syntax-aux-cache-uncache-table.md | 22 +++ docs/sql-ref-syntax-aux-cache.md | 25 +++ docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 22 +++ docs/sql-ref-syntax-aux-conf-mgmt-set.md | 22 +++ docs/sql-ref-syntax-aux-conf-mgmt.md | 25 +++ docs/sql-ref-syntax-aux-describe-database.md | 22 +++ docs/sql-ref-syntax-aux-describe-function.md | 22 +++ docs/sql-ref-syntax-aux-describe-query.md | 22 +++ docs/sql-ref-syntax-aux-describe-table.md | 22 +++ docs/sql-ref-syntax-aux-describe.md | 25 +++ ...l-ref-syntax-aux-resource-mgmt-add-file.md | 22 +++ ...ql-ref-syntax-aux-resource-mgmt-add-jar.md | 22 +++ docs/sql-ref-syntax-aux-resource-mgmt.md | 25 +++ docs/sql-ref-syntax-aux-show-columns.md | 22 +++ docs/sql-ref-syntax-aux-show-create-table.md | 22 +++ docs/sql-ref-syntax-aux-show-databases.md | 22 +++ docs/sql-ref-syntax-aux-show-functions.md | 22 +++ docs/sql-ref-syntax-aux-show-partitions.md | 22 +++ docs/sql-ref-syntax-aux-show-table.md | 22 +++ docs/sql-ref-syntax-aux-show-tables.md | 22 +++ docs/sql-ref-syntax-aux-show-tblproperties.md | 22 +++ docs/sql-ref-syntax-aux-show.md | 25 +++ docs/sql-ref-syntax-aux.md | 25 +++ docs/sql-ref-syntax-ddl-alter-database.md | 22 +++ docs/sql-ref-syntax-ddl-alter-table.md | 22 +++ docs/sql-ref-syntax-ddl-alter-view.md | 22 +++ docs/sql-ref-syntax-ddl-create-database.md | 22 +++ docs/sql-ref-syntax-ddl-create-function.md | 22 +++ docs/sql-ref-syntax-ddl-create-table.md | 22 +++ docs/sql-ref-syntax-ddl-create-view.md | 22 +++ docs/sql-ref-syntax-ddl-drop-database.md | 22 +++ docs/sql-ref-syntax-ddl-drop-function.md | 22 +++ docs/sql-ref-syntax-ddl-drop-table.md | 22 +++ docs/sql-ref-syntax-ddl-drop-view.md | 22 +++ docs/sql-ref-syntax-ddl-repair-table.md | 22 +++ docs/sql-ref-syntax-ddl-truncate-table.md | 22 +++ docs/sql-ref-syntax-ddl.md | 25 +++ docs/sql-ref-syntax-dml-insert.md | 22 +++ docs/sql-ref-syntax-dml-load.md | 22 +++ docs/sql-ref-syntax-dml.md | 25 +++ docs/sql-ref-syntax-qry-aggregation.md | 22 +++ docs/sql-ref-syntax-qry-explain.md | 22 +++ docs/sql-ref-syntax-qry-sampling.md | 22 +++ docs/sql-ref-syntax-qry-select-cte.md | 22 +++ docs/sql-ref-syntax-qry-select-distinct.md | 22 +++ docs/sql-ref-syntax-qry-select-groupby.md | 22 +++ docs/sql-ref-syntax-qry-select-having.md | 22 +++ docs/sql-ref-syntax-qry-select-hints.md | 22 +++ docs/sql-ref-syntax-qry-select-join.md | 22 +++ docs/sql-ref-syntax-qry-select-limit.md | 22 +++ docs/sql-ref-syntax-qry-select-orderby.md | 22 +++ docs/sql-ref-syntax-qry-select-setops.md | 22 +++ docs/sql-ref-syntax-qry-select-subqueries.md | 22 +++ docs/sql-ref-syntax-qry-select.md | 25 +++ docs/sql-ref-syntax-qry-window.md | 22 +++ docs/sql-ref-syntax-qry.md | 25 +++ docs/sql-ref-syntax.md | 25 +++ docs/sql-ref.md | 25 +++ 74 files changed, 1781 insertions(+), 33 deletions(-) create mode 100644 docs/sql-ref-arithmetic-ops.md rename docs/{sql-reference.md => sql-ref-datatypes.md} (94%) create mode 100644 docs/sql-ref-functions-builtin-aggregate.md create mode 100644 docs/sql-ref-functions-builtin-scalar.md create mode 100644 docs/sql-ref-functions-builtin.md create mode 100644 docs/sql-ref-functions-udf-aggregate.md create mode 100644 docs/sql-ref-functions-udf-scalar.md create mode 100644 docs/sql-ref-functions-udf.md create mode 100644 docs/sql-ref-functions.md create mode 100644 docs/sql-ref-nan-semantics.md create mode 100644 docs/sql-ref-syntax-aux-analyze-table.md create mode 100644 docs/sql-ref-syntax-aux-analyze.md create mode 100644 docs/sql-ref-syntax-aux-cache-cache-table.md create mode 100644 docs/sql-ref-syntax-aux-cache-clear-cache.md create mode 100644 docs/sql-ref-syntax-aux-cache-uncache-table.md create mode 100644 docs/sql-ref-syntax-aux-cache.md create mode 100644 docs/sql-ref-syntax-aux-conf-mgmt-reset.md create mode 100644 docs/sql-ref-syntax-aux-conf-mgmt-set.md create mode 100644 docs/sql-ref-syntax-aux-conf-mgmt.md create mode 100644 docs/sql-ref-syntax-aux-describe-database.md create mode 100644 docs/sql-ref-syntax-aux-describe-function.md create mode 100644 docs/sql-ref-syntax-aux-describe-query.md create mode 100644 docs/sql-ref-syntax-aux-describe-table.md create mode 100644 docs/sql-ref-syntax-aux-describe.md create mode 100644 docs/sql-ref-syntax-aux-resource-mgmt-add-file.md create mode 100644 docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md create mode 100644 docs/sql-ref-syntax-aux-resource-mgmt.md create mode 100644 docs/sql-ref-syntax-aux-show-columns.md create mode 100644 docs/sql-ref-syntax-aux-show-create-table.md create mode 100644 docs/sql-ref-syntax-aux-show-databases.md create mode 100644 docs/sql-ref-syntax-aux-show-functions.md create mode 100644 docs/sql-ref-syntax-aux-show-partitions.md create mode 100644 docs/sql-ref-syntax-aux-show-table.md create mode 100644 docs/sql-ref-syntax-aux-show-tables.md create mode 100644 docs/sql-ref-syntax-aux-show-tblproperties.md create mode 100644 docs/sql-ref-syntax-aux-show.md create mode 100644 docs/sql-ref-syntax-aux.md create mode 100644 docs/sql-ref-syntax-ddl-alter-database.md create mode 100644 docs/sql-ref-syntax-ddl-alter-table.md create mode 100644 docs/sql-ref-syntax-ddl-alter-view.md create mode 100644 docs/sql-ref-syntax-ddl-create-database.md create mode 100644 docs/sql-ref-syntax-ddl-create-function.md create mode 100644 docs/sql-ref-syntax-ddl-create-table.md create mode 100644 docs/sql-ref-syntax-ddl-create-view.md create mode 100644 docs/sql-ref-syntax-ddl-drop-database.md create mode 100644 docs/sql-ref-syntax-ddl-drop-function.md create mode 100644 docs/sql-ref-syntax-ddl-drop-table.md create mode 100644 docs/sql-ref-syntax-ddl-drop-view.md create mode 100644 docs/sql-ref-syntax-ddl-repair-table.md create mode 100644 docs/sql-ref-syntax-ddl-truncate-table.md create mode 100644 docs/sql-ref-syntax-ddl.md create mode 100644 docs/sql-ref-syntax-dml-insert.md create mode 100644 docs/sql-ref-syntax-dml-load.md create mode 100644 docs/sql-ref-syntax-dml.md create mode 100644 docs/sql-ref-syntax-qry-aggregation.md create mode 100644 docs/sql-ref-syntax-qry-explain.md create mode 100644 docs/sql-ref-syntax-qry-sampling.md create mode 100644 docs/sql-ref-syntax-qry-select-cte.md create mode 100644 docs/sql-ref-syntax-qry-select-distinct.md create mode 100644 docs/sql-ref-syntax-qry-select-groupby.md create mode 100644 docs/sql-ref-syntax-qry-select-having.md create mode 100644 docs/sql-ref-syntax-qry-select-hints.md create mode 100644 docs/sql-ref-syntax-qry-select-join.md create mode 100644 docs/sql-ref-syntax-qry-select-limit.md create mode 100644 docs/sql-ref-syntax-qry-select-orderby.md create mode 100644 docs/sql-ref-syntax-qry-select-setops.md create mode 100644 docs/sql-ref-syntax-qry-select-subqueries.md create mode 100644 docs/sql-ref-syntax-qry-select.md create mode 100644 docs/sql-ref-syntax-qry-window.md create mode 100644 docs/sql-ref-syntax-qry.md create mode 100644 docs/sql-ref-syntax.md create mode 100644 docs/sql-ref.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 9bbb115bcdda5..717911b5a4645 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -72,12 +72,164 @@ url: sql-migration-guide-hive-compatibility.html - text: SQL Reserved/Non-Reserved Keywords url: sql-reserved-and-non-reserved-keywords.html -- text: Reference - url: sql-reference.html + +- text: SQL Reference + url: sql-ref.html subitems: - text: Data Types - url: sql-reference.html#data-types + url: sql-ref-datatypes.html - text: NaN Semantics - url: sql-reference.html#nan-semantics - - text: Arithmetic operations - url: sql-reference.html#arithmetic-operations + url: sql-ref-nan-semantics.html + - text: SQL Syntax + url: sql-ref-syntax.html + subitems: + - text: Data Definition Statements + url: sql-ref-syntax-ddl.html + subitems: + - text: ALTER DATABASE + url: sql-ref-syntax-ddl-alter-database.html + - text: ALTER TABLE + url: sql-ref-syntax-ddl-alter-table.html + - text: ALTER VIEW + url: sql-ref-syntax-ddl-alter-view.html + - text: CREATE DATABASE + url: sql-ref-syntax-ddl-create-database.html + - text: CREATE FUNCTION + url: sql-ref-syntax-ddl-create-function.html + - text: CREATE TABLE + url: sql-ref-syntax-ddl-create-table.html + - text: CREATE VIEW + url: sql-ref-syntax-ddl-create-view.html + - text: DROP DATABASE + url: sql-ref-syntax-ddl-drop-database.html + - text: DROP FUNCTION + url: sql-ref-syntax-ddl-drop-function.html + - text: DROP TABLE + url: sql-ref-syntax-ddl-drop-table.html + - text: DROP VIEW + url: sql-ref-syntax-ddl-drop-view.html + - text: TRUNCATE TABLE + url: sql-ref-syntax-ddl-truncate-table.html + - text: REPAIR TABLE + url: sql-ref-syntax-ddl-repair-table.html + - text: Data Manipulation Statements + url: sql-ref-syntax-dml.html + subitems: + - text: INSERT + url: sql-ref-syntax-dml-insert.html + - text: LOAD + url: sql-ref-syntax-dml-load.html + - text: Data Retrieval(Queries) + url: sql-ref-syntax-qry.html + subitems: + - text: SELECT + url: sql-ref-syntax-qry-select.html + subitems: + - text: DISTINCT Clause + url: sql-ref-syntax-qry-select-distinct.html + - text: Joins + url: sql-ref-syntax-qry-select-join.html + - text: ORDER BY Clause + url: sql-ref-syntax-qry-select-orderby.html + - text: GROUP BY Clause + url: sql-ref-syntax-qry-select-groupby.html + - text: HAVING Clause + url: sql-ref-syntax-qry-select-having.html + - text: LIMIT Clause + url: sql-ref-syntax-qry-select-limit.html + - text: Set operations + url: sql-ref-syntax-qry-select-setops.html + - text: Common Table Expression(CTE) + url: sql-ref-syntax-qry-select-cte.html + - text: Subqueries + url: sql-ref-syntax-qry-select-subqueries.html + - text: Query hints + url: sql-ref-syntax-qry-select-hints.html + - text: SAMPLING + url: sql-ref-syntax-qry-sampling.html + - text: WINDOWING ANALYTIC FUNCTIONS + url: sql-ref-syntax-qry-window.html + - text: AGGREGATION (CUBE/ROLLUP/GROUPING) + url: sql-ref-syntax-qry-aggregation.html + - text: EXPLAIN + url: sql-ref-syntax-qry-explain.html + - text: Auxilarry Statements + url: sql-ref-syntax-aux.html + subitems: + - text: Analyze statement + url: sql-ref-syntax-aux-analyze.html + subitems: + - text: ANALYZE TABLE + url: sql-ref-syntax-aux-analyze-table.html + - text: Caching statements + url: sql-ref-syntax-aux-cache.html + subitems: + - text: CACHE TABLE + url: sql-ref-syntax-aux-cache-cache-table.html + - text: UNCACHE TABLE + url: sql-ref-syntax-aux-cache-uncache-table.html + - text: CLEAR CACHE + url: sql-ref-syntax-aux-cache-clear-cache.html + - text: Describe Commands + url: sql-ref-syntax-aux-describe.html + subitems: + - text: DESCRIBE DATABASE + url: sql-ref-syntax-aux-describe-database.html + - text: DESCRIBE TABLE + url: sql-ref-syntax-aux-describe-table.html + - text: DESCRIBE FUNCTION + url: sql-ref-syntax-aux-describe-function.html + - text: DESCRIBE QUERY + url: sql-ref-syntax-aux-describe-query.html + - text: Show commands + url: sql-ref-syntax-aux-show.html + subitems: + - text: SHOW COLUMNS + url: sql-ref-syntax-aux-show-columns.html + - text: SHOW DATABASES + url: sql-ref-syntax-aux-show-databases.html + - text: SHOW FUNCTIONS + url: sql-ref-syntax-aux-show-functions.html + - text: SHOW TABLE + url: sql-ref-syntax-aux-show-table.html + - text: SHOW TABLES + url: sql-ref-syntax-aux-show-tables.html + - text: SHOW TBLPROPERTIES + url: sql-ref-syntax-aux-show-tblproperties.html + - text: SHOW PARTITIONS + url: sql-ref-syntax-aux-show-partitions.html + - text: SHOW CREATE TABLE + url: sql-ref-syntax-aux-show-create-table.html + - text: Configuration Management Commands + url: sql-ref-syntax-aux-conf-mgmt.html + subitems: + - text: SET + url: sql-ref-syntax-aux-conf-mgmt-set.html + - text: RESET + url: sql-ref-syntax-aux-conf-mgmt-reset.html + - text: Resource Management Commands + url: sql-ref-syntax-aux-resource-mgmt.html + subitems: + - text: ADD FILE + url: sql-ref-syntax-aux-resource-mgmt-add-file.html + - text: ADD JAR + url: sql-ref-syntax-aux-resource-mgmt-add-jar.html + - text: Functions + url: sql-ref-functions.html + subitems: + - text: Builtin Functions + url: sql-ref-functions-builtin.html + subitems: + - text: Scalar functions + url: sql-ref-functions-builtin-scalar.html + - text: Aggregate functions + url: sql-ref-functions-builtin-aggregate.html + - text: User defined Functions + url: sql-ref-functions-udf.html + subitems: + - text: Scalar functions + url: sql-ref-functions-udf-scalar.html + - text: Aggregate functions + url: sql-ref-functions-udf-aggregate.html + - text: Arthmetic operations + url: sql-ref-arithmetic-ops.html diff --git a/docs/css/main.css b/docs/css/main.css index 7f1e99bf67224..fdfe4bda852a0 100755 --- a/docs/css/main.css +++ b/docs/css/main.css @@ -196,11 +196,14 @@ a.anchorjs-link:hover { text-decoration: none; } width: 210px; float: left; position: fixed; + overflow-y: scroll; + top: 0; + bottom: 0; } .left-menu { padding: 0px; - width: 199px; + width: 399px; } .left-menu h3 { diff --git a/docs/sql-ref-arithmetic-ops.md b/docs/sql-ref-arithmetic-ops.md new file mode 100644 index 0000000000000..7bc8ffe31c990 --- /dev/null +++ b/docs/sql-ref-arithmetic-ops.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Arithmetic Operations +displayTitle: Arithmetic Operations +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. +--- + +Operations performed on numeric types (with the exception of decimal) are not checked for overflow. This means that in case an operation causes an overflow, the result is the same that the same operation returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, the result is a negative number). diff --git a/docs/sql-reference.md b/docs/sql-ref-datatypes.md similarity index 94% rename from docs/sql-reference.md rename to docs/sql-ref-datatypes.md index 2ec26ecc2e1e9..0add62b10ed6b 100644 --- a/docs/sql-reference.md +++ b/docs/sql-ref-datatypes.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Data Types +displayTitle: Data Types license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,10 +19,6 @@ license: | limitations under the License. --- -* Table of contents -{:toc} - -## Data Types Spark SQL and DataFrames support the following data types: @@ -634,25 +630,5 @@ from pyspark.sql.types import * - - - -## NaN Semantics - -There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that -does not exactly match standard floating point semantics. -Specifically: - - - NaN = NaN returns true. - - In aggregations, all NaN values are grouped together. - - NaN is treated as a normal value in join keys. - - NaN values go last when in ascending order, larger than any other numeric value. - -## Arithmetic operations - -Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. -This means that in case an operation causes an overflow, the result is the same that the same operation -returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, -the result is a negative number). diff --git a/docs/sql-ref-functions-builtin-aggregate.md b/docs/sql-ref-functions-builtin-aggregate.md new file mode 100644 index 0000000000000..3fcd78245ca04 --- /dev/null +++ b/docs/sql-ref-functions-builtin-aggregate.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Builtin Aggregate Functions +displayTitle: Builtin Aggregate 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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-functions-builtin-scalar.md b/docs/sql-ref-functions-builtin-scalar.md new file mode 100644 index 0000000000000..1d818a25c4ac1 --- /dev/null +++ b/docs/sql-ref-functions-builtin-scalar.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Builtin Scalar Functions +displayTitle: Builtin Scalar 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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-functions-builtin.md b/docs/sql-ref-functions-builtin.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-functions-builtin.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-functions-udf-aggregate.md b/docs/sql-ref-functions-udf-aggregate.md new file mode 100644 index 0000000000000..49c7b5824dfd6 --- /dev/null +++ b/docs/sql-ref-functions-udf-aggregate.md @@ -0,0 +1,22 @@ +--- +layout: global +title: User defined Aggregate Functions (UDAF) +displayTitle: User defined Aggregate Functions (UDAF) +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-functions-udf-scalar.md b/docs/sql-ref-functions-udf-scalar.md new file mode 100644 index 0000000000000..cee135b572aea --- /dev/null +++ b/docs/sql-ref-functions-udf-scalar.md @@ -0,0 +1,22 @@ +--- +layout: global +title: User defined Scalar Functions (UDF) +displayTitle: User defined Scalar Functions (UDF) +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-functions-udf.md b/docs/sql-ref-functions-udf.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-functions-udf.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-functions.md b/docs/sql-ref-functions.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-functions.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-nan-semantics.md b/docs/sql-ref-nan-semantics.md new file mode 100644 index 0000000000000..f6a85728263a7 --- /dev/null +++ b/docs/sql-ref-nan-semantics.md @@ -0,0 +1,29 @@ +--- +layout: global +title: Nan Semantics +displayTitle: NaN Semantics +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. +--- + +There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that +does not exactly match standard floating point semantics. +Specifically: + + - NaN = NaN returns true. + - In aggregations, all NaN values are grouped together. + - NaN is treated as a normal value in join keys. + - NaN values go last when in ascending order, larger than any other numeric value. diff --git a/docs/sql-ref-syntax-aux-analyze-table.md b/docs/sql-ref-syntax-aux-analyze-table.md new file mode 100644 index 0000000000000..e23098903e886 --- /dev/null +++ b/docs/sql-ref-syntax-aux-analyze-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ANALYZE TABLE +displayTitle: ANALYZE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-analyze.md b/docs/sql-ref-syntax-aux-analyze.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-analyze.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md new file mode 100644 index 0000000000000..16f26929280ff --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CACHE TABLE +displayTitle: CACHE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md new file mode 100644 index 0000000000000..88d126f0f528e --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CLEAR CACHE +displayTitle: CLEAR CACHE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md new file mode 100644 index 0000000000000..15081ff7dfad1 --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: UNCACHE TABLE +displayTitle: UNCACHE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-cache.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md new file mode 100644 index 0000000000000..ad2d7f9a83316 --- /dev/null +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -0,0 +1,22 @@ +--- +layout: global +title: RESET +displayTitle: RESET +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set.md b/docs/sql-ref-syntax-aux-conf-mgmt-set.md new file mode 100644 index 0000000000000..c38d68dbb4f1d --- /dev/null +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SET +displayTitle: SET +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-conf-mgmt.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-describe-database.md b/docs/sql-ref-syntax-aux-describe-database.md new file mode 100644 index 0000000000000..5d1c9deb13a7a --- /dev/null +++ b/docs/sql-ref-syntax-aux-describe-database.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DESCRIBE DATABASE +displayTitle: DESCRIBE DATABASE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-describe-function.md b/docs/sql-ref-syntax-aux-describe-function.md new file mode 100644 index 0000000000000..55e8fc8e1dad3 --- /dev/null +++ b/docs/sql-ref-syntax-aux-describe-function.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DESCRIBE FUNCTION +displayTitle: DESCRIBE FUNCTION +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-describe-query.md b/docs/sql-ref-syntax-aux-describe-query.md new file mode 100644 index 0000000000000..e1c5c54a5bfbb --- /dev/null +++ b/docs/sql-ref-syntax-aux-describe-query.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DESCRIBE QUERY +displayTitle: DESCRIBE QUERY +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md new file mode 100644 index 0000000000000..110a5e4c78171 --- /dev/null +++ b/docs/sql-ref-syntax-aux-describe-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DESCRIBE TABLE +displayTitle: DESCRIBE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-describe.md b/docs/sql-ref-syntax-aux-describe.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-describe.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md new file mode 100644 index 0000000000000..f57b81ead6f49 --- /dev/null +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ADD FILE +displayTitle: ADD FILE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md new file mode 100644 index 0000000000000..cd91119c8c2ba --- /dev/null +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ADD JAR +displayTitle: ADD JAR +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-resource-mgmt.md b/docs/sql-ref-syntax-aux-resource-mgmt.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-resource-mgmt.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-show-columns.md b/docs/sql-ref-syntax-aux-show-columns.md new file mode 100644 index 0000000000000..521ec0002c785 --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-columns.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW COLUMNS +displayTitle: SHOW COLUMNS +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-create-table.md b/docs/sql-ref-syntax-aux-show-create-table.md new file mode 100644 index 0000000000000..2cf40915774c4 --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-create-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW CREATE TABLE +displayTitle: SHOW CREATE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md new file mode 100644 index 0000000000000..e7aedf8cc79a6 --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW DATABASE +displayTitle: SHOW DATABASE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-functions.md b/docs/sql-ref-syntax-aux-show-functions.md new file mode 100644 index 0000000000000..ae689fdf227e9 --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-functions.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW FUNCTIONS +displayTitle: SHOW 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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md new file mode 100644 index 0000000000000..c6499de9cbb9e --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-partitions.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW PARTITIONS +displayTitle: SHOW PARTITIONS +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-table.md b/docs/sql-ref-syntax-aux-show-table.md new file mode 100644 index 0000000000000..ad549b6b11ecb --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW TABLE +displayTitle: SHOW TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md new file mode 100644 index 0000000000000..e4340d608bf5e --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW TABLES +displayTitle: SHOW TABLES +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md new file mode 100644 index 0000000000000..586993eeb5cdd --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -0,0 +1,22 @@ +--- +layout: global +title: SHOW TBLPROPERTIES +displayTitle: SHOW TBLPROPERTIES +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-aux-show.md b/docs/sql-ref-syntax-aux-show.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux-show.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux.md b/docs/sql-ref-syntax-aux.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-aux.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md new file mode 100644 index 0000000000000..29e7ffb940107 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ALTER DATABASE +displayTitle: ALTER DATABASE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md new file mode 100644 index 0000000000000..7fcd397915825 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ALTER TABLE +displayTitle: ALTER TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md new file mode 100644 index 0000000000000..7a7947a3e0ac4 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ALTER VIEW +displayTitle: ALTER VIEW +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md new file mode 100644 index 0000000000000..bbcd34a6d6853 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-database.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CREATE DATABASE +displayTitle: CREATE DATABASE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md new file mode 100644 index 0000000000000..f95a9eba42c2f --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CREATE FUNCTION +displayTitle: CREATE FUNCTION +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-create-table.md b/docs/sql-ref-syntax-ddl-create-table.md new file mode 100644 index 0000000000000..4ce8ef697c2c2 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CREATE TABLE +displayTitle: CREATE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md new file mode 100644 index 0000000000000..eff7df91f59c5 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -0,0 +1,22 @@ +--- +layout: global +title: CREATE VIEW +displayTitle: CREATE VIEW +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-drop-database.md b/docs/sql-ref-syntax-ddl-drop-database.md new file mode 100644 index 0000000000000..cd900a7e393db --- /dev/null +++ b/docs/sql-ref-syntax-ddl-drop-database.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DROP DATABASE +displayTitle: DROP DATABASE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-drop-function.md b/docs/sql-ref-syntax-ddl-drop-function.md new file mode 100644 index 0000000000000..9cf1f445e91f4 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-drop-function.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DROP FUNCTION +displayTitle: DROP FUNCTION +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-drop-table.md b/docs/sql-ref-syntax-ddl-drop-table.md new file mode 100644 index 0000000000000..a036e66c3906d --- /dev/null +++ b/docs/sql-ref-syntax-ddl-drop-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DROP TABLE +displayTitle: DROP TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md new file mode 100644 index 0000000000000..9ad22500fd9e4 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -0,0 +1,22 @@ +--- +layout: global +title: DROP VIEW +displayTitle: DROP VIEW +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-repair-table.md b/docs/sql-ref-syntax-ddl-repair-table.md new file mode 100644 index 0000000000000..f21de558b8a07 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-repair-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: REPAIR TABLE +displayTitle: REPAIR TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md new file mode 100644 index 0000000000000..2704259391e94 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -0,0 +1,22 @@ +--- +layout: global +title: TRUNCATE TABLE +displayTitle: TRUNCATE TABLE +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl.md b/docs/sql-ref-syntax-ddl.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-ddl.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-dml-insert.md b/docs/sql-ref-syntax-dml-insert.md new file mode 100644 index 0000000000000..200be07a33095 --- /dev/null +++ b/docs/sql-ref-syntax-dml-insert.md @@ -0,0 +1,22 @@ +--- +layout: global +title: INSERT +displayTitle: INSERT +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md new file mode 100644 index 0000000000000..fd25ba314e0b6 --- /dev/null +++ b/docs/sql-ref-syntax-dml-load.md @@ -0,0 +1,22 @@ +--- +layout: global +title: LOAD +displayTitle: LOAD +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-dml.md b/docs/sql-ref-syntax-dml.md new file mode 100644 index 0000000000000..7fd537e0a0f5c --- /dev/null +++ b/docs/sql-ref-syntax-dml.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Data Manipulation Statements +displayTitle: Data Manipulation Statements +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-qry-aggregation.md b/docs/sql-ref-syntax-qry-aggregation.md new file mode 100644 index 0000000000000..fbe6e61c84fd1 --- /dev/null +++ b/docs/sql-ref-syntax-qry-aggregation.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Aggregation (CUBE/ROLLUP/GROUPING) +displayTitle: Aggregation (CUBE/ROLLUP/GROUPING) +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-explain.md b/docs/sql-ref-syntax-qry-explain.md new file mode 100644 index 0000000000000..501f2083f0f8c --- /dev/null +++ b/docs/sql-ref-syntax-qry-explain.md @@ -0,0 +1,22 @@ +--- +layout: global +title: EXPLAIN +displayTitle: EXPLAIN +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-sampling.md b/docs/sql-ref-syntax-qry-sampling.md new file mode 100644 index 0000000000000..a5efb361ee847 --- /dev/null +++ b/docs/sql-ref-syntax-qry-sampling.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Sampling Queries +displayTitle: Sampling Queries +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-cte.md b/docs/sql-ref-syntax-qry-select-cte.md new file mode 100644 index 0000000000000..2bd7748291a9f --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-cte.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Common Table Expression (CTE) +displayTitle: Common Table Expression (CTE) +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-distinct.md b/docs/sql-ref-syntax-qry-select-distinct.md new file mode 100644 index 0000000000000..2ed7931c202f3 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-distinct.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Distinct operator +displayTitle: Distinct operator +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md new file mode 100644 index 0000000000000..8ba7d155f3716 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -0,0 +1,22 @@ +--- +layout: global +title: GROUPBY Clause +displayTitle: GROUPBY Clause +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md new file mode 100644 index 0000000000000..ca92eb0d4daf0 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -0,0 +1,22 @@ +--- +layout: global +title: HAVING Clause +displayTitle: HAVING Clause +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-hints.md b/docs/sql-ref-syntax-qry-select-hints.md new file mode 100644 index 0000000000000..eb9655fc0d731 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-hints.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Query Hints +displayTitle: Query Hints +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-join.md b/docs/sql-ref-syntax-qry-select-join.md new file mode 100644 index 0000000000000..ee66dbdb8c823 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-join.md @@ -0,0 +1,22 @@ +--- +layout: global +title: JOIN +displayTitle: JOIN +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md new file mode 100644 index 0000000000000..d7fac3bb98929 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -0,0 +1,22 @@ +--- +layout: global +title: LIMIT operator +displayTitle: LIMIT operator +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md new file mode 100644 index 0000000000000..1f7c031e4aa88 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -0,0 +1,22 @@ +--- +layout: global +title: ORDER BY Clause +displayTitle: ORDER BY Clause +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-setops.md b/docs/sql-ref-syntax-qry-select-setops.md new file mode 100644 index 0000000000000..526f857520aa2 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-setops.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Set Operations +displayTitle: Set Operations +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select-subqueries.md b/docs/sql-ref-syntax-qry-select-subqueries.md new file mode 100644 index 0000000000000..976e201f2714b --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-subqueries.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Subqueries +displayTitle: Subqueries +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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md new file mode 100644 index 0000000000000..41972ef070831 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select.md @@ -0,0 +1,25 @@ +--- +layout: global +title: SELECT +displayTitle: SELECT +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-qry-window.md b/docs/sql-ref-syntax-qry-window.md new file mode 100644 index 0000000000000..767f47715e89c --- /dev/null +++ b/docs/sql-ref-syntax-qry-window.md @@ -0,0 +1,22 @@ +--- +layout: global +title: Windowing Analytic Functions +displayTitle: Windowing Analytic 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. +--- + +**This page is under construction** diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax-qry.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref-syntax.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. diff --git a/docs/sql-ref.md b/docs/sql-ref.md new file mode 100644 index 0000000000000..eb0e73d00e848 --- /dev/null +++ b/docs/sql-ref.md @@ -0,0 +1,25 @@ +--- +layout: global +title: Reference +displayTitle: Reference +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 SQL is a Apache Spark's module for working with structured data. +This guide is a reference for Structured Query Language (SQL) for Apache +Spark. This document describes the SQL constructs supported by Spark in detail +along with usage examples when applicable. From f999e00e9fc03889106a305eeec6d05b6a2c1de1 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Mon, 19 Aug 2019 15:53:43 +0800 Subject: [PATCH 100/149] [SPARK-28356][SHUFFLE][FOLLOWUP] Fix case with different pre-shuffle partition numbers ### What changes were proposed in this pull request? This PR reverts some of the latest changes in `ReduceNumShufflePartitions` to fix the case when there are different pre-shuffle partition numbers in the plan. Please see the new UT for an example. ### Why are the changes needed? Eliminate a bug. ### Does this PR introduce any user-facing change? Yes, some queries that failed will succeed now. ### How was this patch tested? Added new UT. Closes #25479 from peter-toth/SPARK-28356-followup. Authored-by: Peter Toth Signed-off-by: Wenchen Fan --- .../adaptive/ReduceNumShufflePartitions.scala | 7 ++++++- .../ReduceNumShufflePartitionsSuite.scala | 18 ++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 78923433eaab9..1a85d5c02075b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -82,7 +82,12 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. val validMetrics = shuffleMetrics.filter(_ != null) - if (validMetrics.nonEmpty) { + // We may have different pre-shuffle partition numbers, don't reduce shuffle partition number + // in that case. For example when we union fully aggregated data (data is arranged to a single + // partition) and a result of a SortMergeJoin (multiple partitions). + val distinctNumPreShufflePartitions = + validMetrics.map(stats => stats.bytesByPartitionId.length).distinct + if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) // This transformation adds new nodes, so we must use `transformUp` here. plan.transformUp { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 35c33a7157d38..b5dbdd0b18b49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -587,4 +587,22 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA } withSparkSession(test, 200, None) } + + test("Union two datasets with different pre-shuffle partition number") { + val test: SparkSession => Unit = { spark: SparkSession => + val df1 = spark.range(3).join(spark.range(3), "id").toDF() + val df2 = spark.range(3).groupBy().sum() + + val resultDf = df1.union(df2) + + checkAnswer(resultDf, Seq((0), (1), (2), (3)).map(i => Row(i))) + + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + // As the pre-shuffle partition number are different, we will skip reducing + // the shuffle partition numbers. + assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 0) + } + withSparkSession(test, 100, None) + } } From 97dc4c0bfc3a15d364a376c6f87cb921d8d6980d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 19 Aug 2019 19:01:56 +0800 Subject: [PATCH 101/149] [SPARK-28744][SQL][TEST] rename SharedSQLContext to SharedSparkSession ## What changes were proposed in this pull request? The Spark SQL test framework needs to support 2 kinds of tests: 1. tests inside Spark to test Spark itself (extends `SparkFunSuite`) 2. test outside of Spark to test Spark applications (introduced at https://github.com/apache/spark/commit/b57ed2245c705fb0964462cf4492b809ade836c6) The class hierarchy of the major testing traits: ![image](https://user-images.githubusercontent.com/3182036/63088526-c0f0af80-bf87-11e9-9bed-c144c2486da9.png) `PlanTestBase`, `SQLTestUtilsBase` and `SharedSparkSession` intentionally don't extend `SparkFunSuite`, so that they can be used for tests outside of Spark. Tests in Spark should extends `QueryTest` and/or `SharedSQLContext` in most cases. However, the name is a little confusing. As a result, some test suites extend `SharedSparkSession` instead of `SharedSQLContext`. `SharedSparkSession` doesn't work well with `SparkFunSuite` as it doesn't have the special handling of thread auditing in `SharedSQLContext`. For example, you will see a warning starting with `===== POSSIBLE THREAD LEAK IN SUITE` when you run `DataFrameSelfJoinSuite`. This PR proposes to rename `SharedSparkSession` to `SharedSparkSessionBase`, and rename `SharedSQLContext` to `SharedSparkSession`. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review https://spark.apache.org/contributing.html before opening a pull request. Closes #25463 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../AvroCatalystDataConversionSuite.scala | 4 +- .../spark/sql/avro/AvroFunctionsSuite.scala | 4 +- .../spark/sql/avro/AvroLogicalTypeSuite.scala | 4 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 4 +- .../sql/jdbc/DockerJDBCIntegrationSuite.scala | 10 +---- .../sql/jdbc/OracleIntegrationSuite.scala | 4 +- .../kafka010/CachedKafkaProducerSuite.scala | 4 +- .../sql/kafka010/KafkaDataConsumerSuite.scala | 4 +- .../KafkaDontFailOnDataLossSuite.scala | 4 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 4 +- .../sql/kafka010/KafkaRelationSuite.scala | 4 +- .../spark/sql/kafka010/KafkaSinkSuite.scala | 4 +- .../sql/kafka010/KafkaSourceOffsetSuite.scala | 4 +- ...xCountDistinctForIntervalsQuerySuite.scala | 4 +- .../sql/ApproximatePercentileQuerySuite.scala | 4 +- .../apache/spark/sql/BenchmarkQueryTest.scala | 6 +-- .../apache/spark/sql/CachedTableSuite.scala | 4 +- .../spark/sql/ColumnExpressionSuite.scala | 4 +- .../apache/spark/sql/ComplexTypesSuite.scala | 4 +- .../spark/sql/ConfigBehaviorSuite.scala | 4 +- .../sql/CountMinSketchAggQuerySuite.scala | 4 +- .../apache/spark/sql/CsvFunctionsSuite.scala | 4 +- .../spark/sql/DataFrameAggregateSuite.scala | 4 +- .../spark/sql/DataFrameComplexTypeSuite.scala | 4 +- .../spark/sql/DataFrameFunctionsSuite.scala | 4 +- .../apache/spark/sql/DataFrameHintSuite.scala | 4 +- .../spark/sql/DataFrameImplicitsSuite.scala | 4 +- .../apache/spark/sql/DataFrameJoinSuite.scala | 4 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 4 +- .../spark/sql/DataFramePivotSuite.scala | 4 +- .../spark/sql/DataFrameRangeSuite.scala | 4 +- .../sql/DataFrameSetOperationsSuite.scala | 4 +- .../apache/spark/sql/DataFrameStatSuite.scala | 6 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- .../sql/DataFrameTimeWindowingSuite.scala | 4 +- .../spark/sql/DataFrameTungstenSuite.scala | 4 +- .../sql/DataFrameWindowFramesSuite.scala | 4 +- .../sql/DataFrameWindowFunctionsSuite.scala | 4 +- .../spark/sql/DatasetAggregatorSuite.scala | 4 +- .../apache/spark/sql/DatasetCacheSuite.scala | 4 +- .../spark/sql/DatasetOptimizationSuite.scala | 4 +- .../spark/sql/DatasetPrimitiveSuite.scala | 4 +- .../DatasetSerializerRegistratorSuite.scala | 4 +- .../org/apache/spark/sql/DatasetSuite.scala | 4 +- .../apache/spark/sql/DateFunctionsSuite.scala | 4 +- .../org/apache/spark/sql/ExplainSuite.scala | 4 +- .../spark/sql/ExtraStrategiesSuite.scala | 4 +- .../spark/sql/FileBasedDataSourceSuite.scala | 5 +-- .../spark/sql/GeneratorFunctionSuite.scala | 4 +- .../org/apache/spark/sql/JoinHintSuite.scala | 4 +- .../org/apache/spark/sql/JoinSuite.scala | 4 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 4 +- .../apache/spark/sql/MathFunctionsSuite.scala | 4 +- .../apache/spark/sql/MetadataCacheSuite.scala | 4 +- .../apache/spark/sql/MiscFunctionsSuite.scala | 4 +- .../org/apache/spark/sql/QueryTest.scala | 2 +- ...ullWithFalseInPredicateEndToEndSuite.scala | 4 +- .../scala/org/apache/spark/sql/RowSuite.scala | 4 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +- .../sql/ScalaReflectionRelationSuite.scala | 4 +- .../apache/spark/sql/SerializationSuite.scala | 4 +- .../spark/sql/ShowCreateTableSuite.scala | 4 +- .../spark/sql/StatisticsCollectionSuite.scala | 4 +- .../spark/sql/StringFunctionsSuite.scala | 4 +- .../org/apache/spark/sql/SubquerySuite.scala | 4 +- .../sql/TypedImperativeAggregateSuite.scala | 4 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 4 +- .../spark/sql/UserDefinedTypeSuite.scala | 4 +- .../spark/sql/XPathFunctionsSuite.scala | 4 +- .../spark/sql/api/r/SQLUtilsSuite.scala | 4 +- .../execution/BroadcastExchangeSuite.scala | 4 +- .../DataSourceScanExecRedactionSuite.scala | 4 +- .../spark/sql/execution/ExchangeSuite.scala | 4 +- .../sql/execution/GlobalTempViewSuite.scala | 4 +- .../spark/sql/execution/HiveResultSuite.scala | 5 +-- .../OptimizeMetadataOnlyQuerySuite.scala | 4 +- .../spark/sql/execution/PlannerSuite.scala | 4 +- .../sql/execution/QueryExecutionSuite.scala | 4 +- .../QueryPlanningTrackerEndToEndSuite.scala | 4 +- .../spark/sql/execution/SQLViewSuite.scala | 4 +- .../execution/SQLWindowFunctionSuite.scala | 4 +- .../spark/sql/execution/SameResultSuite.scala | 4 +- .../spark/sql/execution/SortSuite.scala | 4 +- .../spark/sql/execution/SparkPlanSuite.scala | 4 +- .../sql/execution/SparkPlannerSuite.scala | 4 +- .../TakeOrderedAndProjectSuite.scala | 4 +- .../UnsafeFixedWidthAggregationMapSuite.scala | 4 +- .../UnsafeKVExternalSorterSuite.scala | 4 +- .../execution/WholeStageCodegenSuite.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 4 +- .../arrow/ArrowConvertersSuite.scala | 5 +-- .../columnar/InMemoryColumnarQuerySuite.scala | 4 +- .../columnar/PartitionBatchPruningSuite.scala | 11 +---- .../execution/command/DDLParserSuite.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 4 +- .../datasources/DataSourceStrategySuite.scala | 4 +- .../datasources/FileFormatWriterSuite.scala | 4 +- .../datasources/FileIndexSuite.scala | 4 +- .../datasources/FileSourceStrategySuite.scala | 4 +- .../HadoopFileLinesReaderSuite.scala | 4 +- .../datasources/ReadSchemaTest.scala | 4 +- .../RowDataSourceStrategySuite.scala | 4 +- .../SaveIntoDataSourceCommandSuite.scala | 4 +- .../datasources/SchemaPruningSuite.scala | 4 +- .../binaryfile/BinaryFileFormatSuite.scala | 4 +- .../execution/datasources/csv/CSVSuite.scala | 4 +- .../json/JsonParsingOptionsSuite.scala | 4 +- .../datasources/json/JsonSuite.scala | 4 +- .../datasources/noop/NoopSuite.scala | 4 +- .../orc/OrcColumnarBatchReaderSuite.scala | 4 +- .../orc/OrcPartitionDiscoverySuite.scala | 6 +-- .../datasources/orc/OrcQuerySuite.scala | 4 +- .../datasources/orc/OrcSourceSuite.scala | 4 +- .../ParquetAvroCompatibilitySuite.scala | 4 +- ...rquetCompressionCodecPrecedenceSuite.scala | 4 +- .../parquet/ParquetEncodingSuite.scala | 4 +- .../parquet/ParquetFileFormatSuite.scala | 4 +- .../parquet/ParquetFilterSuite.scala | 4 +- .../datasources/parquet/ParquetIOSuite.scala | 4 +- .../ParquetInteroperabilitySuite.scala | 4 +- .../ParquetPartitionDiscoverySuite.scala | 4 +- .../ParquetProtobufCompatibilitySuite.scala | 4 +- .../parquet/ParquetQuerySuite.scala | 4 +- .../parquet/ParquetSchemaSuite.scala | 4 +- .../ParquetThriftCompatibilitySuite.scala | 4 +- .../datasources/text/TextSuite.scala | 4 +- .../datasources/text/WholeTextFileSuite.scala | 4 +- .../datasources/v2/FileTableSuite.scala | 4 +- .../v2/V2SessionCatalogSuite.scala | 4 +- .../v2/V2StreamingScanSupportCheckSuite.scala | 3 +- .../sql/execution/debug/DebuggingSuite.scala | 5 +-- .../execution/joins/ExistenceJoinSuite.scala | 4 +- .../execution/joins/HashedRelationSuite.scala | 4 +- .../sql/execution/joins/InnerJoinSuite.scala | 4 +- .../sql/execution/joins/OuterJoinSuite.scala | 4 +- .../execution/metric/SQLMetricsSuite.scala | 5 +-- .../python/BatchEvalPythonExecSuite.scala | 4 +- .../python/ExtractPythonUDFsSuite.scala | 4 +- .../sql/execution/python/PythonUDFSuite.scala | 4 +- .../CheckpointFileManagerSuite.scala | 2 +- .../CompactibleFileStreamLogSuite.scala | 5 +-- .../streaming/FileStreamSinkLogSuite.scala | 4 +- .../streaming/HDFSMetadataLogSuite.scala | 5 +-- .../streaming/OffsetSeqLogSuite.scala | 5 +-- .../sources/ForeachWriterSuite.scala | 4 +- .../sources/TextSocketStreamSuite.scala | 6 +-- .../execution/ui/AllExecutionsPageSuite.scala | 4 +- .../ui/SQLAppStatusListenerSuite.scala | 4 +- .../spark/sql/internal/CatalogSuite.scala | 10 +---- .../spark/sql/internal/SQLConfSuite.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 4 +- .../spark/sql/sources/BucketedReadSuite.scala | 4 +- .../sql/sources/BucketedWriteSuite.scala | 4 +- .../sources/CreateTableAsSelectSuite.scala | 9 +--- .../sql/sources/DDLSourceLoadSuite.scala | 4 +- .../spark/sql/sources/FilteredScanSuite.scala | 5 +-- .../spark/sql/sources/InsertSuite.scala | 4 +- .../sql/sources/PartitionedWriteSuite.scala | 4 +- .../spark/sql/sources/PathOptionSuite.scala | 4 +- .../spark/sql/sources/PrunedScanSuite.scala | 4 +- .../sql/sources/ResolvedDataSourceSuite.scala | 5 +-- .../spark/sql/sources/SaveLoadSuite.scala | 4 +- .../spark/sql/sources/TableScanSuite.scala | 4 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 4 +- .../v2/DataSourceV2DataFrameSuite.scala | 4 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 4 +- .../sql/sources/v2/DataSourceV2Suite.scala | 4 +- .../v2/FileDataSourceV2FallBackSuite.scala | 4 +- .../sql/streaming/FileStreamSourceSuite.scala | 4 +- .../spark/sql/streaming/StreamTest.scala | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 2 +- .../spark/sql/test/GenericFlatSpecSuite.scala | 2 +- .../spark/sql/test/GenericFunSpecSuite.scala | 2 +- .../spark/sql/test/GenericWordSpecSuite.scala | 2 +- .../spark/sql/test/SharedSQLContext.scala | 44 ------------------- .../spark/sql/test/SharedSparkSession.scala | 28 +++++++++++- .../sql/util/DataFrameCallbackSuite.scala | 4 +- .../datasources/orc/OrcFilterSuite.scala | 4 +- .../datasources/orc/OrcFilterSuite.scala | 4 +- 181 files changed, 380 insertions(+), 437 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 27915562fded0..4b39e711aa287 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -24,11 +24,11 @@ import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ class AvroCatalystDataConversionSuite extends SparkFunSuite - with SharedSQLContext + with SharedSparkSession with ExpressionEvalHelper { private def roundTripTest(data: Literal): Unit = { diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index ffd77c5ff6101..c4f995015dfca 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.LocalTableScanExec import org.apache.spark.sql.functions.{col, struct} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession -class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class AvroFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("roundtrip in to_avro and from_avro - int and string") { diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 96382764b053c..d4649912f1e7a 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -28,10 +28,10 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{StructField, StructType, TimestampType} -abstract class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession { import testImplicits._ val dateSchema = s""" diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index a7c9e3fb7d329..366d1d233bc33 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -38,11 +38,11 @@ import org.apache.spark.sql._ import org.apache.spark.sql.TestingUDT.{IntervalData, NullData, NullUDT} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +abstract class AvroSuite extends QueryTest with SharedSparkSession { import testImplicits._ val episodesAvro = testFile("episodes.avro") diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index e9e547eb9a263..519a5cddb3639 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -26,12 +26,10 @@ import scala.util.control.NonFatal import com.spotify.docker.client._ import com.spotify.docker.client.exceptions.ImageNotFoundException import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} -import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.DockerUtils abstract class DatabaseOnDocker { @@ -66,11 +64,7 @@ abstract class DatabaseOnDocker { def getStartupProcessName: Option[String] } -abstract class DockerJDBCIntegrationSuite - extends SparkFunSuite - with BeforeAndAfterAll - with Eventually - with SharedSQLContext { +abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventually { val db: DatabaseOnDocker diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 8cdc4a1806b2b..6faa888cf18ed 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.{RowDataSourceScanExec, WholeStageCodegenE import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCRelation} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest @@ -54,7 +54,7 @@ import org.apache.spark.tags.DockerTest * repository. */ @DockerTest -class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext { +class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSparkSession { import testImplicits._ override val db = new DatabaseOnDocker { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala index 0b3355426df10..35c1379de160b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala @@ -24,9 +24,9 @@ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.PrivateMethodTester -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class CachedKafkaProducerSuite extends SharedSQLContext with PrivateMethodTester with KafkaTest { +class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTester with KafkaTest { type KP = KafkaProducer[Array[Byte], Array[Byte]] diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala index 0d0fb9c3ab5af..2aa869c02bc5d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala @@ -30,10 +30,10 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.scalatest.PrivateMethodTester import org.apache.spark.{TaskContext, TaskContextImpl} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.ThreadUtils -class KafkaDataConsumerSuite extends SharedSQLContext with PrivateMethodTester { +class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester { protected var testUtils: KafkaTestUtils = _ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index ba8340ea59c14..3e02580b53961 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamTest, Trigger} -import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} +import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} /** * This is a basic test trait which will set up a Kafka cluster that keeps only several records in @@ -43,7 +43,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} * does see missing offsets, you can check the earliest offset in `eventually` and make sure it's * not 0 rather than sleeping a hard-code duration. */ -trait KafkaMissingOffsetsTest extends SharedSQLContext { +trait KafkaMissingOffsetsTest extends SharedSparkSession { protected var testUtils: KafkaTestUtils = _ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 609c43803b591..8663a5d8d26cb 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -45,10 +45,10 @@ import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.util.CaseInsensitiveStringMap -abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest { +abstract class KafkaSourceTest extends StreamTest with SharedSparkSession with KafkaTest { protected var testUtils: KafkaTestUtils = _ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 84d1ab6f75c3e..ed41080eab459 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -32,10 +32,10 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils -abstract class KafkaRelationSuiteBase extends QueryTest with SharedSQLContext with KafkaTest { +abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { import testImplicits._ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 032adc70466fa..2746cf96c9cdb 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BinaryType, DataType} -abstract class KafkaSinkSuiteBase extends QueryTest with SharedSQLContext with KafkaTest { +abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { protected var testUtils: KafkaTestUtils = _ override def beforeAll(): Unit = { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala index efec51d09745f..ef902fcab3b50 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -21,9 +21,9 @@ import java.io.File import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.streaming.OffsetSuite -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class KafkaSourceOffsetSuite extends OffsetSuite with SharedSQLContext { +class KafkaSourceOffsetSuite extends OffsetSuite with SharedSparkSession { compare( one = KafkaSourceOffset(("t", 0, 1L)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala index c7d86bc955d67..171e93c1bf002 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala @@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.ApproxCountDistinctForIntervals import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSQLContext { +class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ // ApproxCountDistinctForIntervals is used in equi-height histogram generation. An equi-height 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 52708f5fe4108..a4b142b7ab78e 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 @@ -23,12 +23,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.PercentileDigest import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * End-to-end tests for approximate percentile aggregate function. */ -class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { +class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ private val table = "percentile_test" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala index c37d663941d8d..3fcb9892800b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala @@ -17,16 +17,14 @@ package org.apache.spark.sql -import org.scalatest.BeforeAndAfterAll - import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils -abstract class BenchmarkQueryTest extends QueryTest with SharedSQLContext with BeforeAndAfterAll { +abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { // When Utils.isTesting is true, the RuleExecutor will issue an exception when hitting // the max iteration of analyzer/optimizer batches. 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 267f255a11e86..6e1ee6da9200d 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 @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.storage.StorageLevel.{MEMORY_AND_DISK_2, MEMORY_ONLY} @@ -40,7 +40,7 @@ import org.apache.spark.util.{AccumulatorContext, Utils} private case class BigData(s: String) -class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext { +class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSession { import testImplicits._ setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index ee258f8ac3661..a52c6d503d147 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -30,10 +30,10 @@ import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class ColumnExpressionSuite extends QueryTest with SharedSQLContext { +class ColumnExpressionSuite extends QueryTest with SharedSparkSession { import testImplicits._ private lazy val booleanData = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala index b74fe2f90df23..4d0eb04be751b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions.CreateNamedStruct import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ComplexTypesSuite extends QueryTest with SharedSQLContext { +class ComplexTypesSuite extends QueryTest with SharedSparkSession { override def beforeAll() { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala index 276496be3d62c..431e797e1686e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ConfigBehaviorSuite extends QueryTest with SharedSQLContext { +class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala index dea0d4c0c6d40..5a80bdcdc0f35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.sketch.CountMinSketch /** * End-to-end test suite for count_min_sketch. */ -class CountMinSketchAggQuerySuite extends QueryTest with SharedSQLContext { +class CountMinSketchAggQuerySuite extends QueryTest with SharedSparkSession { test("count-min sketch") { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 6b67fccf86b9f..52cf91cfade51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -25,10 +25,10 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class CsvFunctionsSuite extends QueryTest with SharedSQLContext { +class CsvFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("from_csv with empty options") { 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 c56c93f708577..ec7b636c8f695 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 @@ -27,13 +27,13 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.DecimalData import org.apache.spark.sql.types.DecimalType case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) -class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { +class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { import testImplicits._ val absTol = 1e-8 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 1230b921aa279..e9179a39d3b6d 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 @@ -19,12 +19,12 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * A test suite to test DataFrame/SQL functionalities with complex types (i.e. array, struct, map). */ -class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { +class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("UDF on struct") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 69d442a86c29e..7d044638db571 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -30,13 +30,13 @@ import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** * Test suite for functions in [[org.apache.spark.sql.functions]]. */ -class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { +class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("array with column name") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala index 7ef8b542c79a8..b33c26a0b75a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameHintSuite extends AnalysisTest with SharedSQLContext { +class DataFrameHintSuite extends AnalysisTest with SharedSparkSession { import testImplicits._ lazy val df = spark.range(10) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala index 25e1d93ff0927..a454d91a8e62c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameImplicitsSuite extends QueryTest with SharedSQLContext { +class DataFrameImplicitsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("RDD of tuples") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index dc7928fde7791..3a217e6e28060 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameJoinSuite extends QueryTest with SharedSQLContext { +class DataFrameJoinSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("join - join using") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index e6983b6be555a..aeee4577d3483 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { +class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ def createDF(): DataFrame = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index 8c2c11be9b6fd..bcd0c3f0d64a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -22,10 +22,10 @@ import java.util.Locale import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class DataFramePivotSuite extends QueryTest with SharedSQLContext { +class DataFramePivotSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("pivot courses") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala index 6c1a66cae2279..fc549e307c80f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala @@ -27,10 +27,10 @@ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventually { +class DataFrameRangeSuite extends QueryTest with SharedSparkSession with Eventually { test("SPARK-7150 range api") { // numSlice is greater than length diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index 30452af1fad64..fbb7e903c3450 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -22,11 +22,11 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.plans.logical.Union import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} import org.apache.spark.sql.test.SQLTestData.NullStrings import org.apache.spark.sql.types._ -class DataFrameSetOperationsSuite extends QueryTest with SharedSQLContext { +class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("except") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 2a74bfe4d3789..0572728d68226 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -25,10 +25,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.functions.{col, lit, struct} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, StructField, StructType} -class DataFrameStatSuite extends QueryTest with SharedSQLContext { +class DataFrameStatSuite extends QueryTest with SharedSparkSession { import testImplicits._ private def toLetter(i: Int): String = (i + 97).toChar.toString @@ -449,7 +449,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { } -class DataFrameStatPerfSuite extends QueryTest with SharedSQLContext with Logging { +class DataFrameStatPerfSuite extends QueryTest with SharedSparkSession with Logging { // Turn on this test if you want to test the performance of approximate quantiles. ignore("computing quantiles should not take much longer than describe()") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 8782f95fe517f..ba8fced983c61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -37,13 +37,13 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} import org.apache.spark.sql.test.SQLTestData.{NullStrings, TestData2} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom -class DataFrameSuite extends QueryTest with SharedSQLContext { +class DataFrameSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("analysis error should be eagerly reported") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 2953425b1db49..8b0f46b9d1ddb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -21,10 +21,10 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.catalyst.plans.logical.Expand import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StringType -class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { +class DataFrameTimeWindowingSuite extends QueryTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala index 0881212a64de8..a9f7d5bfab23e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** @@ -27,7 +27,7 @@ import org.apache.spark.sql.types._ * This is here for now so I can make sure Tungsten project is tested without refactoring existing * end-to-end test infra. In the long run this should just go away. */ -class DataFrameTungstenSuite extends QueryTest with SharedSQLContext { +class DataFrameTungstenSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("test simple types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala index 002c17f4cce4f..fbd399917e390 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala @@ -21,12 +21,12 @@ import java.sql.Date import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * Window frame testing for DataFrame API. */ -class DataFrameWindowFramesSuite extends QueryTest with SharedSQLContext { +class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("lead/lag with empty data frame") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index a07553008fd78..835630bff7099 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -25,13 +25,13 @@ import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** * Window function testing for DataFrame API. */ -class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { +class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index e581211e4e76f..817387b2845f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType} @@ -221,7 +221,7 @@ case class OptionBooleanIntAggregator(colName: String) def OptionalBoolIntEncoder: Encoder[Option[(Boolean, Int)]] = ExpressionEncoder() } -class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { +class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { import testImplicits._ private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b) 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 d888585e120a4..33d9def0b44e5 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 @@ -22,11 +22,11 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.storage.StorageLevel -class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits { +class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimits { import testImplicits._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala index ae051e43fbcb5..892122b94b977 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.objects.ExternalMapToCatalyst import org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject import org.apache.spark.sql.functions.expr import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DatasetOptimizationSuite extends QueryTest with SharedSQLContext { +class DatasetOptimizationSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("SPARK-26619: Prune the unused serializers from SerializeFromObject") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index 4d7037f36b1fd..91a8f0a26b360 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -22,7 +22,7 @@ import scala.collection.immutable.Queue import scala.collection.mutable.{LinkedHashMap => LHMap} import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession case class IntClass(value: Int) @@ -47,7 +47,7 @@ package object packageobject { case class PackageClass(value: Int) } -class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { +class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("toDS") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala index 69728efcd1d9a..43de2663b1236 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -23,12 +23,12 @@ import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.SparkConf import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoRegistrator -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * Test suite to test Kryo custom registrators. */ -class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext { +class DatasetSerializerRegistratorSuite extends QueryTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index ff6143162ff2f..42e5ee58954e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2) @@ -49,7 +49,7 @@ object TestForTypeAlias { def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2)) } -class DatasetSuite extends QueryTest with SharedSQLContext { +class DatasetSuite extends QueryTest with SharedSparkSession { import testImplicits._ private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index a92c4177da1a0..2fef05f97e57c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -26,10 +26,10 @@ import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.unsafe.types.CalendarInterval -class DateFunctionsSuite extends QueryTest with SharedSQLContext { +class DateFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("function current_date") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index ec688282d5740..125cff0e6628a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType -class ExplainSuite extends QueryTest with SharedSQLContext { +class ExplainSuite extends QueryTest with SharedSparkSession { import testImplicits._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala index a41b465548622..9192370cfa620 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession case class FastOperator(output: Seq[Attribute]) extends SparkPlan { @@ -46,7 +46,7 @@ object TestStrategy extends Strategy { } } -class ExtraStrategiesSuite extends QueryTest with SharedSQLContext { +class ExtraStrategiesSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("insert an extraStrategy") { 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 b1bde9098e096..4b24add42d8d8 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 @@ -24,7 +24,6 @@ import java.util.Locale import scala.collection.mutable import org.apache.hadoop.fs.Path -import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} @@ -35,11 +34,11 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with BeforeAndAfterAll { +class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { import testImplicits._ override def beforeAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 8280a3ce39845..951e246072d39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -22,10 +22,10 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Generator} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} -class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { +class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("stack") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 6bda1afdfdd42..6b154253e6e6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class JoinHintSuite extends PlanTest with SharedSQLContext { +class JoinHintSuite extends PlanTest with SharedSparkSession { import testImplicits._ lazy val df = spark.range(10) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 059dbf892c653..1e97347f6751a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType -class JoinSuite extends QueryTest with SharedSQLContext { +class JoinSuite extends QueryTest with SharedSparkSession { import testImplicits._ setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index ae6e9037acd21..92a4acc130be5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -25,10 +25,10 @@ import collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class JsonFunctionsSuite extends QueryTest with SharedSQLContext { +class JsonFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("function get_json_object") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala index d3db3b40e59a1..567bcdd1878a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -21,14 +21,14 @@ import java.nio.charset.StandardCharsets import org.apache.spark.sql.functions._ import org.apache.spark.sql.functions.{log => logarithm} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession private object MathFunctionsTestData { case class DoubleData(a: java.lang.Double, b: java.lang.Double) case class NullDoubles(a: java.lang.Double) } -class MathFunctionsSuite extends QueryTest with SharedSQLContext { +class MathFunctionsSuite extends QueryTest with SharedSparkSession { import MathFunctionsTestData._ import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index 602951bc48a1e..a8993bca657c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -21,12 +21,12 @@ import java.io.File import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * Test suite to handle metadata cache related. */ -abstract class MetadataCacheSuite extends QueryTest with SharedSQLContext { +abstract class MetadataCacheSuite extends QueryTest with SharedSparkSession { /** Removes one data file in the given directory. */ protected def deleteOneFileInDirectory(dir: File): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala index a5b08f717767f..cad0821dbf5aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class MiscFunctionsSuite extends QueryTest with SharedSQLContext { +class MiscFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("reflect and java_method") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index d034e10e0d077..3039a4ccb677c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -416,7 +416,7 @@ object QueryTest { } } -class QueryTestSuite extends QueryTest with test.SharedSQLContext { +class QueryTestSuite extends QueryTest with test.SharedSparkSession { test("SPARK-16940: checkAnswer should raise TestFailedException for wrong results") { intercept[org.scalatest.exceptions.TestFailedException] { checkAnswer(sql("SELECT 1"), Row(2) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala index 1729c3c0c557a..bdbb741f24bc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.expressions.{CaseWhen, If, Literal} import org.apache.spark.sql.execution.LocalTableScanExec import org.apache.spark.sql.functions.{lit, when} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.BooleanType -class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSQLContext { +class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("SPARK-25860: Replace Literal(null, _) with FalseLiteral whenever possible") { 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 c53fd5bd7a158..fd9655fdbef42 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, SpecificInternalRow} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -class RowSuite extends SparkFunSuite with SharedSQLContext { +class RowSuite extends SparkFunSuite with SharedSparkSession { import testImplicits._ test("create row") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 972950669198c..a8664ad66c66e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -33,11 +33,11 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ -class SQLQuerySuite extends QueryTest with SharedSQLContext { +class SQLQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ setupTestData() 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 4bdf25051127c..a32bc8b2f8a35 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType /** @@ -102,7 +102,7 @@ import org.apache.spark.sql.types.StructType * Therefore, UDF test cases should have single input and output files but executed by three * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ -class SQLQueryTestSuite extends QueryTest with SharedSQLContext { +class SQLQueryTestSuite extends QueryTest with SharedSparkSession { import IntegratedUDFTestUtils._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index c9bd05d0e4e36..7e305e0504729 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession case class ReflectData( stringField: String, @@ -74,7 +74,7 @@ case class ComplexReflectData( mapFieldContainsNull: Map[Int, Option[Long]], dataField: Data) -class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { +class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession { import testImplicits._ test("query case class RDD") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index 1a1c956aed3d9..3ca21a3d02f08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SerializationSuite extends SparkFunSuite with SharedSQLContext { +class SerializationSuite extends SparkFunSuite with SharedSparkSession { test("[SPARK-5235] SQLContext should be serializable") { val spark = SparkSession.builder.getOrCreate() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index ada3baf3cd720..42307b1b9734e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.util.Utils -class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSQLContext +class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSparkSession abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 4c78f85a3ac69..e9ceab6724659 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.ArrayData import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -39,7 +39,7 @@ import org.apache.spark.util.Utils /** * End-to-end suite testing statistics collection and use on both entire table and columns. */ -class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSQLContext { +class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession { import testImplicits._ test("estimates the size of a limit 0 on outer join") { 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 cc5af327cf599..88b3e5ec61f8a 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 @@ -18,10 +18,10 @@ package org.apache.spark.sql import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class StringFunctionsSuite extends QueryTest with SharedSQLContext { +class StringFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("string concat") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index b46abdb48e738..a1d7792941ed9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.FileScanRDD import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SubquerySuite extends QueryTest with SharedSQLContext { +class SubquerySuite extends QueryTest with SharedSparkSession { import testImplicits._ setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala index c5fb17345222a..f2b608bbcb21b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggreg import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext { +class TypedImperativeAggregateSuite extends QueryTest with SharedSparkSession { import testImplicits._ 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 08a98e29d2ad0..2a034bcdc3f00 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectComm import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.functions.{lit, udf} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.QueryExecutionListener @@ -36,7 +36,7 @@ import org.apache.spark.sql.util.QueryExecutionListener private case class FunctionResult(f1: String, f2: String) -class UDFSuite extends QueryTest with SharedSQLContext { +class UDFSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("built-in fixed arity expressions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 49f0000212554..2b2fedd3ca218 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseVector) { @@ -101,7 +101,7 @@ private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType] override def userClass: Class[IExampleSubType] = classOf[IExampleSubType] } -class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest +class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with ParquetTest with ExpressionEvalHelper { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala index 1d33e7970be8e..a25cca7af50bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.sql -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * End-to-end tests for xpath expressions. */ -class XPathFunctionsSuite extends QueryTest with SharedSQLContext { +class XPathFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("xpath_boolean") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala index 7cfee49575570..3fb5a4aa4f3ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.api.r -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SQLUtilsSuite extends SharedSQLContext { +class SQLUtilsSuite extends SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala index 4e39df928603f..43e29c2d50786 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins.HashedRelation import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class BroadcastExchangeSuite extends SparkPlanTest with SharedSQLContext { +class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala index 20e864ab24b71..1051410cc0997 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala @@ -23,12 +23,12 @@ import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * Test suite base for testing the redaction of DataSourceScanExec/BatchScanExec. */ -abstract class DataSourceScanRedactionTest extends QueryTest with SharedSQLContext { +abstract class DataSourceScanRedactionTest extends QueryTest with SharedSparkSession { override protected def sparkConf: SparkConf = super.sparkConf .set("spark.redaction.string.regex", "file:/[^\\]\\s]+") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index bde2de5b39fd7..4828c4973731f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityB import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ExchangeSuite extends SparkPlanTest with SharedSQLContext { +class ExchangeSuite extends SparkPlanTest with SharedSparkSession { import testImplicits._ test("shuffling UnsafeRows in exchange") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala index 972b47e96fe06..7fbfa73623c85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalog.Table import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType -class GlobalTempViewSuite extends QueryTest with SharedSQLContext { +class GlobalTempViewSuite extends QueryTest with SharedSparkSession { import testImplicits._ override protected def beforeAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index bbce4705871df..104cf4c58d617 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.execution import java.sql.{Date, Timestamp} -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} -class HiveResultSuite extends SparkFunSuite with SharedSQLContext { +class HiveResultSuite extends SharedSparkSession { import testImplicits._ test("date formatting in hive result") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index e20a82ba9bc48..1ed3e07d1dbe9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_METADATA_ONLY -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { +class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ override def beforeAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index e30fb13d10df1..2bafc87bc25f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -29,10 +29,10 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class PlannerSuite extends SharedSQLContext { +class PlannerSuite extends SharedSparkSession { import testImplicits._ setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 39c87c9eeb47d..eca39f3f81726 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession case class QueryExecutionTestRecord( c0: Int, c1: Int, c2: Int, c3: Int, c4: Int, @@ -33,7 +33,7 @@ case class QueryExecutionTestRecord( c20: Int, c21: Int, c22: Int, c23: Int, c24: Int, c25: Int, c26: Int) -class QueryExecutionSuite extends SharedSQLContext { +class QueryExecutionSuite extends SharedSparkSession { import testImplicits._ def checkDumpedPlans(path: String, expected: Int): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala index e42177c156ee9..76006efda992f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext { +class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession { test("programmatic API") { val df = spark.range(1000).selectExpr("count(*)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 8d4a9ae6a5760..347a14c9129f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -21,9 +21,9 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} -class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext +class SimpleSQLViewSuite extends SQLViewSuite with SharedSparkSession /** * A suite for testing view related functionality. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala index 971fd842f046a..7aabf4d039f08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.TestUtils.assertSpilled import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession case class WindowData(month: Int, area: String, product: Int) @@ -28,7 +28,7 @@ case class WindowData(month: Int, area: String, product: Int) /** * Test suite for SQL window functions. */ -class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { +class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala index b252100d890e3..de1acd3ff2734 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala @@ -23,13 +23,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.IntegerType /** * Tests for the sameResult function for [[SparkPlan]]s. */ -class SameResultSuite extends QueryTest with SharedSQLContext { +class SameResultSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("FileSourceScanExec: different orders of data filters and partition filters") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index a7bbe34f4eedb..7654a9d982059 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -22,14 +22,14 @@ import scala.util.Random import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** * Test sorting. Many of the test cases generate random data and compares the sorted result with one * sorted by a reference implementation ([[ReferenceSort]]). */ -class SortSuite extends SparkPlanTest with SharedSQLContext { +class SortSuite extends SparkPlanTest with SharedSparkSession { import testImplicits.newProductEncoder import testImplicits.localSeqToDatasetHolder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index e9d0556ebb513..1a29c471a77af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala @@ -21,9 +21,9 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SparkPlanSuite extends QueryTest with SharedSQLContext { +class SparkPlanSuite extends QueryTest with SharedSparkSession { test("SPARK-21619 execution of a canonicalized plan should fail") { val plan = spark.range(10).queryExecution.executedPlan.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala index 5828f9783da42..9107f8afa83d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Strategy import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SparkPlannerSuite extends SharedSQLContext { +class SparkPlannerSuite extends SharedSparkSession { import testImplicits._ test("Ensure to go down only the first branch, not any other possible branches") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 7e317a4d80265..376d330ebeb70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -22,11 +22,11 @@ import scala.util.Random import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { +class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession { private var rand: Random = _ private var seed: Long = 0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 5c15ecd42fa0c..392cce54ebede 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String class UnsafeFixedWidthAggregationMapSuite extends SparkFunSuite with Matchers - with SharedSQLContext { + with SharedSparkSession { import UnsafeFixedWidthAggregationMap._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index c882a9dd2148c..8aa003a3dfeb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -27,14 +27,14 @@ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.map.BytesToBytesMap /** * Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data. */ -class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { +class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSparkSession { private val keyTypes = Seq(IntegerType, FloatType, DoubleType, StringType) private val valueTypes = Seq(IntegerType, FloatType, DoubleType, StringType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 55dff16887cb8..0ea16a1a15d66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} -class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { +class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { import testImplicits._ 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 d8efca323d519..720e7edb72f8d 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 @@ -23,9 +23,9 @@ import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { +class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession { import testImplicits._ setupTestData() 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 67c3fa0d3bf59..2eb4ac52aca90 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 @@ -27,20 +27,19 @@ import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot} import org.apache.arrow.vector.ipc.JsonFileReader import org.apache.arrow.vector.util.{ByteArrayReadableSeekableByteChannel, Validator} -import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BinaryType, Decimal, IntegerType, StructField, StructType} import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils -class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { +class ArrowConvertersSuite extends SharedSparkSession { import testImplicits._ private var tempDataPath: String = _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 711ecf1738ab1..2f6843cb5b6d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -27,13 +27,13 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel._ import org.apache.spark.util.Utils -class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { +class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession { import testImplicits._ setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 7b2ccca403bb2..a22cb66474493 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -17,19 +17,12 @@ package org.apache.spark.sql.execution.columnar -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.DataFrame import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData._ -class PartitionBatchPruningSuite - extends SparkFunSuite - with BeforeAndAfterEach - with SharedSQLContext { +class PartitionBatchPruningSuite extends SharedSparkSession { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 83452cdd8927b..74ef81f7181da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -35,10 +35,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, Log import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -class DDLParserSuite extends AnalysisTest with SharedSQLContext { +class DDLParserSuite extends AnalysisTest with SharedSparkSession { private lazy val parser = new SparkSqlParser(new SQLConf) private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b777db750a1bb..fd1da2011f28e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -33,12 +33,12 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with BeforeAndAfterEach { +class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { import testImplicits._ override def afterEach(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala index 2f5d5551c5df0..a44a94aaa4f94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala @@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.sources -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataSourceStrategySuite extends PlanTest with SharedSQLContext { +class DataSourceStrategySuite extends PlanTest with SharedSparkSession { test("translate simple expression") { val attrInt = 'cint.int diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala index e09ec0d7bbb49..ce511842e6356 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession class FileFormatWriterSuite extends QueryTest - with SharedSQLContext + with SharedSparkSession with CodegenInterpretedPlanTest{ import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 2a5c5a2dd0ff8..4b086e830e456 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -30,11 +30,11 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.KnownSizeEstimation -class FileIndexSuite extends SharedSQLContext { +class FileIndexSuite extends SharedSparkSession { private class TestInMemoryFileIndex( spark: SparkSession, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index eaff5a2352a0c..7845895f3ce1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -35,11 +35,11 @@ import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.util.Utils -class FileSourceStrategySuite extends QueryTest with SharedSQLContext with PredicateHelper { +class FileSourceStrategySuite extends QueryTest with SharedSparkSession with PredicateHelper { import testImplicits._ protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala index 508614a7e476c..3c83388461ce1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala @@ -24,9 +24,9 @@ import java.nio.file.Files import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class HadoopFileLinesReaderSuite extends SharedSQLContext { +class HadoopFileLinesReaderSuite extends SharedSparkSession { def getLines( path: File, text: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala index d42809529ceed..fd70b6529ff51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala @@ -21,7 +21,7 @@ import java.io.File import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} /** * The reader schema is said to be evolved (or projected) when it changed after the data is @@ -67,7 +67,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} * -> ToDecimalTypeTest */ -trait ReadSchemaTest extends QueryTest with SQLTestUtils with SharedSQLContext { +trait ReadSchemaTest extends QueryTest with SharedSparkSession { val format: String val options: Map[String, String] = Map.empty[String, String] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala index e8bf21a2a9dbe..6420081a9757b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala @@ -25,11 +25,11 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class RowDataSourceStrategySuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext { +class RowDataSourceStrategySuite extends SharedSparkSession with BeforeAndAfter { import testImplicits._ val url = "jdbc:h2:mem:testdb0" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index 8b06b175a2103..233978289f068 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class SaveIntoDataSourceCommandSuite extends SharedSQLContext { +class SaveIntoDataSourceCommandSuite extends SharedSparkSession { test("simpleString is redacted") { val URL = "connection.url" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 09ca42851836b..bb3cec579016e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -27,14 +27,14 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType abstract class SchemaPruningSuite extends QueryTest with FileBasedDataSourceTest with SchemaPruningTest - with SharedSQLContext { + with SharedSparkSession { case class FullName(first: String, middle: String, last: String) case class Company(name: String, address: String) case class Employer(id: Int, company: Company) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index a66b34fe367fa..70ec9bbf4819d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -34,11 +34,11 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf.SOURCES_BINARY_FILE_MAX_LENGTH import org.apache.spark.sql.sources._ -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class BinaryFileFormatSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { import BinaryFileFormat._ private var testDir: String = _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index cf17025980ee3..a6c2e5bfab2eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -39,10 +39,10 @@ import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with TestCsvData { +class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { import testImplicits._ private val carsFile = "test-data/cars.csv" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 316c5183fddf1..bafb6769af69c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.datasources.json import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.json.JSONOptions -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession /** * Test cases for various [[JSONOptions]]. */ -class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext { +class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("allowComments off") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 21b5cbb3ace29..2998e673bd45c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.ExternalRDD import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.types.StructType.fromDDL import org.apache.spark.util.Utils @@ -45,7 +45,7 @@ class TestFileFilter extends PathFilter { override def accept(path: Path): Boolean = path.getParent.getName != "p=2" } -class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { +class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { import testImplicits._ test("Type promotion") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala index 59de28688ec1d..c5a03cb8ef6d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.datasources.noop -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class NoopSuite extends SharedSQLContext { +class NoopSuite extends SharedSparkSession { import testImplicits._ test("materialisation of all rows") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala index c16fcc67f8dd1..719bf91e1786b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala @@ -22,11 +22,11 @@ import org.apache.orc.TypeDescription import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.unsafe.types.UTF8String.fromString -class OrcColumnarBatchReaderSuite extends QueryTest with SQLTestUtils with SharedSQLContext { +class OrcColumnarBatchReaderSuite extends QueryTest with SharedSparkSession { private val dataSchema = StructType.fromDDL("col1 int, col2 int") private val partitionSchema = StructType.fromDDL("p1 string, p2 string") private val partitionValues = InternalRow(fromString("partValue1"), fromString("partValue2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index 143e3f0997201..42ea161cb30c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession // The data where the partitioning key exists only in the directory structure. case class OrcParData(intField: Int, stringField: String) @@ -168,7 +168,7 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } } -class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { +class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSparkSession { test("read partitioned table - partition key included in orc file") { withTempDir { base => for { @@ -252,7 +252,7 @@ class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQ } } -class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { +class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSparkSession { override protected def sparkConf: SparkConf = super .sparkConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 18ec3e3ebed0c..b5b9e398f5ae7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -621,7 +621,7 @@ abstract class OrcQueryTest extends OrcTest { } } -class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { +class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { import testImplicits._ test("LZO compression options for writing to an ORC file") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index c9f5d9cb23e6b..0d904a09c07e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StructField, StructType} import org.apache.spark.util.Utils @@ -480,7 +480,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } -class OrcSourceSuite extends OrcSuite with SharedSQLContext { +class OrcSourceSuite extends OrcSuite with SharedSparkSession { protected override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala index 1b99fbedca047..d7727d93ddf98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala @@ -31,9 +31,9 @@ import org.apache.parquet.hadoop.ParquetWriter import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.parquet.test.avro._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { +class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession { private def withWriter[T <: IndexedRecord] (path: String, schema: Schema) (f: ParquetWriter[T] => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala index 09de715e87a11..900f4fdca395d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala @@ -25,9 +25,9 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSQLContext { +class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSparkSession { test("Test `spark.sql.parquet.compression.codec` config") { Seq("NONE", "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO", "LZ4", "BROTLI", "ZSTD").foreach { c => withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> c) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala index db73bfa149aa0..6d681afd23b18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala @@ -20,11 +20,11 @@ import scala.collection.JavaConverters._ import org.apache.parquet.hadoop.ParquetOutputFormat -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession // TODO: this needs a lot more testing but it's currently not easy to test with the parquet // writer abstractions. Revisit. -class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContext { +class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSparkSession { import testImplicits._ val ROW = ((1).toByte, 2, 3L, "abc") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala index 94abf115cef35..e65f4d12bf7f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala @@ -22,9 +22,9 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkException import org.apache.spark.sql.QueryTest import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSQLContext { +class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSparkSession { test("read parquet footers in parallel") { def testReadFooters(ignoreCorruptFiles: Boolean): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 577d1bc8d6a4f..365cb137eceb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} @@ -61,7 +61,7 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} * dependent on this configuration, don't forget you better explicitly set this configuration * within the test. */ -abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext { +abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSparkSession { protected def createParquetFilters( schema: MessageType, 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 6f2218ba82dc8..026ba5deffdfd 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 @@ -47,7 +47,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -74,7 +74,7 @@ private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteS /** * A test suite that tests basic Parquet I/O. */ -class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { +class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession { import testImplicits._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index edbc2493ac26d..1ded34f24e436 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -28,9 +28,9 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSQLContext { +class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSparkSession { test("parquet files with different physical schemas but share the same logical schema") { import ParquetCompatibilityTest._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 04ace0a236e6d..34f9c2e630e49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -51,7 +51,7 @@ case class ParquetData(intField: Int, stringField: String) case class ParquetDataWithKey(intField: Int, pi: Int, stringField: String, ps: String) abstract class ParquetPartitionDiscoverySuite - extends QueryTest with ParquetTest with SharedSQLContext { + extends QueryTest with ParquetTest with SharedSparkSession { import PartitioningUtils._ import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala index fa88019298a69..f24647bf41dd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql.Row -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { +class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession { test("unannotated array of primitive type") { checkAnswer(readResourceParquetFile("test-data/old-repeated-int.parquet"), Row(Seq(1, 2, 3))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index a6429bfc52924..3aa594ba44c31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -33,14 +33,14 @@ import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStru import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils /** * A test suite that tests various Parquet queries. */ -abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext { +abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSparkSession { import testImplicits._ test("simple select queries") { 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 f8e4822af11f1..1274995fd6779 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 @@ -28,10 +28,10 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { +abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession { /** * Checks whether the reflected Parquet message type for product type `T` conforms `messageType`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala index 4157a5b46dc42..c59c049612389 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql.Row -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { +class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession { import ParquetCompatibilityTest._ private val parquetFilePath = Thread.currentThread().getContextClassLoader.getResource( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index a86d5ee37f3db..62a779528cec1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -27,11 +27,11 @@ import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.TestUtils import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.Utils -class TextSuite extends QueryTest with SharedSQLContext { +class TextSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("reading text file") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala index a302d67b5cbf7..5e3b3441aa74f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala @@ -21,10 +21,10 @@ import java.io.File import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{StringType, StructType} -class WholeTextFileSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class WholeTextFileSuite extends QueryTest with SharedSparkSession { // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which // can cause Filesystem.get(Configuration) to return a cached instance created with a different diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala index ac1d5672af68c..ad0dfadacca15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -51,7 +51,7 @@ class DummyFileTable( override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[TextFileFormat] } -class FileTableSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class FileTableSuite extends QueryTest with SharedSparkSession { test("Data type validation should check data schema only") { withTempPath { dir => 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 4f14ecc28680d..64460d0338054 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 @@ -30,12 +30,12 @@ import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, TableCatalog, Tabl import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class V2SessionCatalogSuite - extends SparkFunSuite with SharedSQLContext with BeforeAndAfter with BeforeAndAfterAll { + extends SparkFunSuite with SharedSparkSession with BeforeAndAfter { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala index 8a0450fce76a1..e2d801104da7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala @@ -21,7 +21,6 @@ import java.util import scala.collection.JavaConverters._ -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical.Union import org.apache.spark.sql.execution.datasources.DataSource @@ -32,7 +31,7 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class V2StreamingScanSupportCheckSuite extends SparkFunSuite with SharedSparkSession { +class V2StreamingScanSupportCheckSuite extends SharedSparkSession { import TableCapability._ private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index e423420c2914a..aaf1fe4807fab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.debug import java.io.ByteArrayOutputStream -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData -class DebuggingSuite extends SparkFunSuite with SharedSQLContext { +class DebuggingSuite extends SharedSparkSession { test("DataFrame.debug()") { testData.debug() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 771a9730247af..5490246baceea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructType} -class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { +class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { private lazy val left = spark.createDataFrame( sparkContext.parallelize(Seq( 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 02e474ce83372..1a3d3e819213e 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 @@ -28,13 +28,13 @@ import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.collection.CompactBuffer -class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { +class HashedRelationSuite extends SharedSparkSession { val mm = new TaskMemoryManager( new UnifiedMemoryManager( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index f99a278bb2427..08898f80034e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} -class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { +class InnerJoinSuite extends SparkPlanTest with SharedSparkSession { import testImplicits.newProductEncoder import testImplicits.localSeqToDatasetHolder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 1f04fcf6ca451..a5ade0d8d7508 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} -class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { +class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { private lazy val left = spark.createDataFrame( sparkContext.parallelize(Seq( 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 dc4a2998a908f..0a254ab210bad 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 @@ -22,7 +22,6 @@ import java.io.File import scala.reflect.{classTag, ClassTag} import scala.util.Random -import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -30,10 +29,10 @@ import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeSt import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{AccumulatorContext, JsonProtocol} -class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with SharedSQLContext { +class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { import testImplicits._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index ac5752b41cf28..d26989b00a651 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -24,10 +24,10 @@ import org.apache.spark.api.python.{PythonEvalType, PythonFunction} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, DoubleType} -class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { +class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSparkSession { import testImplicits.newProductEncoder import testImplicits.localSeqToDatasetHolder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala index 311a8ef3257da..4f1c28d36e384 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSQLContext { +class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { import testImplicits._ val batchedPythonUDF = new MyDummyPythonUDF diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala index 1a971b0deb7f5..8cf1b7fc71079 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.python import org.apache.spark.sql.{IntegratedUDFTestUtils, QueryTest} import org.apache.spark.sql.functions.count -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class PythonUDFSuite extends QueryTest with SharedSQLContext { +class PythonUDFSuite extends QueryTest with SharedSparkSession { import testImplicits._ import IntegratedUDFTestUtils._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala index cbac1c13cdd33..c57b40c977e49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala @@ -90,7 +90,7 @@ abstract class CheckpointFileManagerTests extends SparkFunSuite with SQLHelper { } } -class CheckpointFileManagerSuite extends SparkFunSuite with SharedSparkSession { +class CheckpointFileManagerSuite extends SharedSparkSession { test("CheckpointFileManager.create() should pick up user-specified class from conf") { withSQLConf( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala index 71dc3776bcaf6..ead17d50b4e1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala @@ -20,11 +20,10 @@ package org.apache.spark.sql.execution.streaming import java.io._ import java.nio.charset.StandardCharsets._ -import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class CompactibleFileStreamLogSuite extends SparkFunSuite with SharedSQLContext { +class CompactibleFileStreamLogSuite extends SharedSparkSession { import CompactibleFileStreamLog._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index dd3a414659c23..f95daafdfe19b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -22,9 +22,9 @@ import java.nio.charset.StandardCharsets.UTF_8 import org.apache.spark.SparkFunSuite import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { +class FileStreamSinkLogSuite extends SparkFunSuite with SharedSparkSession { import CompactibleFileStreamLog._ import FileStreamSinkLog._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 04fe1e787909d..c09756cd1b248 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -25,11 +25,10 @@ import scala.language.implicitConversions import org.scalatest.concurrent.Waiters._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.UninterruptibleThread -class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { +class HDFSMetadataLogSuite extends SharedSparkSession { private implicit def toOption[A](a: A): Option[A] = Option(a) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala index e6cdc063c4e9f..e5dfa33164903 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.streaming import java.io.File -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext { +class OffsetSeqLogSuite extends SharedSparkSession { /** test string offset type */ case class StringOffset(override val json: String) extends Offset diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 1e7fa8e91cdff..0c17320acade9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { +class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeAndAfter { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 5ff250dd83867..e1284ea03267e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -27,8 +27,6 @@ import java.util.concurrent.TimeUnit._ import scala.collection.JavaConverters._ -import org.scalatest.BeforeAndAfterEach - import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.datasources.DataSource @@ -38,11 +36,11 @@ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap -class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach { +class TextSocketStreamSuite extends StreamTest with SharedSparkSession { override def afterEach() { sqlContext.streams.active.foreach(_.stop()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala index 95a6af3720d85..9e42056c19a0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala @@ -29,11 +29,11 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.scheduler.{JobFailed, SparkListenerJobEnd, SparkListenerJobStart} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.InMemoryStore -class AllExecutionsPageSuite extends SharedSQLContext with BeforeAndAfter { +class AllExecutionsPageSuite extends SharedSparkSession with BeforeAndAfter { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index bb3f6d8236fd7..90966d2efec23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -39,13 +39,13 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator} import org.apache.spark.util.kvstore.InMemoryStore -class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils +class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils with BeforeAndAfter { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index 6acac1a9aa317..d6a1fde2147b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -19,16 +19,13 @@ package org.apache.spark.sql.internal import java.io.File -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.{Column, Database, Function, Table} import org.apache.spark.sql.catalyst.{FunctionIdentifier, ScalaReflection, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.Range -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel @@ -36,10 +33,7 @@ import org.apache.spark.storage.StorageLevel /** * Tests for the user-facing [[org.apache.spark.sql.catalog.Catalog]]. */ -class CatalogSuite - extends SparkFunSuite - with BeforeAndAfterEach - with SharedSQLContext { +class CatalogSuite extends SharedSparkSession { import testImplicits._ private def sessionCatalog: SessionCatalog = spark.sessionState.catalog diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 829dea48a22be..1dfbca64f5778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} import org.apache.spark.util.Utils -class SQLConfSuite extends QueryTest with SharedSQLContext { +class SQLConfSuite extends QueryTest with SharedSparkSession { import testImplicits._ private val testKey = "test.key.0" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 158085c35351b..72a564506bd23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -36,12 +36,12 @@ import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartiti import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils class JDBCSuite extends QueryTest - with BeforeAndAfter with PrivateMethodTester with SharedSQLContext { + with BeforeAndAfter with PrivateMethodTester with SharedSparkSession { import testImplicits._ val url = "jdbc:h2:mem:testdb0" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index b751ec2de4825..b28c6531d42b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -29,11 +29,11 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { +class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { val url = "jdbc:h2:mem:testdb2" var conn: java.sql.Connection = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index c3edec39979ae..7043b6d396977 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -34,11 +34,11 @@ import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.util.Utils import org.apache.spark.util.collection.BitSet -class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedSQLContext { +class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedSparkSession { protected override def beforeAll(): Unit = { super.beforeAll() assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 75f68dea96bf0..9713de988e379 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.execution.datasources.BucketingUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} -class BucketedWriteWithoutHiveSupportSuite extends BucketedWriteSuite with SharedSQLContext { +class BucketedWriteWithoutHiveSupportSuite extends BucketedWriteSuite with SharedSparkSession { protected override def beforeAll(): Unit = { super.beforeAll() assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 5f9856656ac3b..08f0865c1e128 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -19,21 +19,16 @@ package org.apache.spark.sql.sources import java.io.File -import org.scalatest.BeforeAndAfterEach - import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.internal.SQLConf.BUCKETING_MAX_BUCKETS -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils -class CreateTableAsSelectSuite - extends DataSourceTest - with SharedSQLContext - with BeforeAndAfterEach { +class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession { import testImplicits._ protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala index f22d843bfabde..5d4ddeac66375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.{AnalysisException, SQLContext} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ // please note that the META-INF/services had to be modified for the test directory for this to work -class DDLSourceLoadSuite extends DataSourceTest with SharedSQLContext { +class DDLSourceLoadSuite extends DataSourceTest with SharedSparkSession { test("data sources with the same name - internal data sources") { val e = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 2b39bda97bd8d..657ef5ca13bd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -21,10 +21,9 @@ import java.util.Locale import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -133,7 +132,7 @@ object ColumnsRequired { var set: Set[String] = Set.empty } -class FilteredScanSuite extends DataSourceTest with SharedSQLContext with PredicateHelper { +class FilteredScanSuite extends DataSourceTest with SharedSparkSession { protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 4f1ae069d4b89..237f197bdec50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -50,7 +50,7 @@ case class SimpleInsert(userSpecifiedSchema: StructType)(@transient val sparkSes } } -class InsertSuite extends DataSourceTest with SharedSQLContext { +class InsertSuite extends DataSourceTest with SharedSparkSession { import testImplicits._ protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index 27c983f270bf6..ab1d1f80e7397 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String) @@ -43,7 +43,7 @@ private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String } } -class PartitionedWriteSuite extends QueryTest with SharedSQLContext { +class PartitionedWriteSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("write many partitions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala index 85da3f0e38468..87dce376a09dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, Metadata, MetadataBuilder, StructType} class TestOptionsSource extends SchemaRelationProvider with CreatableRelationProvider { @@ -65,7 +65,7 @@ class TestOptionsRelation(val options: Map[String, String])(@transient val sessi } } -class PathOptionSuite extends DataSourceTest with SharedSQLContext { +class PathOptionSuite extends DataSourceTest with SharedSparkSession { test("path option always exist") { withTable("src") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 309591dd90f0f..d99c605b2e478 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.sources import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ class PrunedScanSource extends RelationProvider { @@ -53,7 +53,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: Spa } } -class PrunedScanSuite extends DataSourceTest with SharedSQLContext { +class PrunedScanSuite extends DataSourceTest with SharedSparkSession { protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 0aa67bf1b0d48..36fb418b09cb6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.sources -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class ResolvedDataSourceSuite extends SparkFunSuite with SharedSQLContext { +class ResolvedDataSourceSuite extends SharedSparkSession { private def getProvidingClass(name: String): Class[_] = DataSource( sparkSession = spark, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 7680f61b8b6c7..ce3ec7f97a537 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -25,11 +25,11 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { +class SaveLoadSuite extends DataSourceTest with SharedSparkSession with BeforeAndAfter { import testImplicits._ protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 13a126ff963d5..d4e117953942e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ class DefaultSource extends SimpleScanSource @@ -108,7 +108,7 @@ case class AllDataTypesScan( } } -class TableScanSuite extends DataSourceTest with SharedSQLContext { +class TableScanSuite extends DataSourceTest with SharedSparkSession { protected override lazy val sql = spark.sql _ private lazy val tableWithSchemaExpected = (1 to 10).map { i => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala index 2ef2df3345e8f..2503c8c529889 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -31,13 +31,13 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class DataSourceV2DataFrameSessionCatalogSuite extends QueryTest - with SharedSQLContext + with SharedSparkSession with BeforeAndAfter { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala index 8909c41ddaa8f..af9e56a3b9816 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala @@ -21,9 +21,9 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataSourceV2DataFrameSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { +class DataSourceV2DataFrameSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { import testImplicits._ before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index cfa6506a95e76..26420e8437930 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -30,11 +30,11 @@ import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG} import org.apache.spark.sql.sources.v2.internal.UnresolvedTable -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { +class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index 379c9c4303cd6..8f7dbe8d13c39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -36,12 +36,12 @@ import org.apache.spark.sql.sources.{Filter, GreaterThan} import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch -class DataSourceV2Suite extends QueryTest with SharedSQLContext { +class DataSourceV2Suite extends QueryTest with SharedSparkSession { import testImplicits._ private def getBatch(query: DataFrame): AdvancedBatch = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala index 493aee6c1a9de..050292a4566bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} @@ -80,7 +80,7 @@ class DummyWriteOnlyFileTable extends Table with SupportsWrite { Set(TableCapability.BATCH_WRITE, TableCapability.ACCEPT_ANY_SCHEMA).asJava } -class FileDataSourceV2FallBackSuite extends QueryTest with SharedSQLContext { +class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].getName private val dummyParquetWriterV2 = classOf[DummyWriteOnlyFileDataSourceV2].getName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index f3f03715ee83a..f59f819c9c108 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -35,12 +35,12 @@ import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._ import org.apache.spark.sql.streaming.util.StreamManualClock -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils abstract class FileStreamSourceTest - extends StreamTest with SharedSQLContext with PrivateMethodTester { + extends StreamTest with SharedSparkSession with PrivateMethodTester { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 3a4414f6e6ecf..7914a713f0baa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.StreamingQueryListener._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{Clock, SystemClock, Utils} /** @@ -71,7 +71,7 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * avoid hanging forever in the case of failures. However, individual suites can change this * by overriding `streamingTimeout`. */ -trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with BeforeAndAfterAll { +trait StreamTest extends QueryTest with SharedSparkSession with TimeLimits with BeforeAndAfterAll { // Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x implicit val defaultSignaler: Signaler = ThreadSignaler diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 15a000b45a7b6..23efcf48844e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -142,7 +142,7 @@ class MessageCapturingCommitProtocol(jobId: String, path: String) } -class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { +class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { import testImplicits._ private val userSchema = new StructType().add("s", StringType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala index 14ac479e89754..2861b80190abe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset * The purpose of this suite is to make sure that generic FlatSpec-based scala * tests work with a shared spark session */ -class GenericFlatSpecSuite extends FlatSpec with SharedSparkSession { +class GenericFlatSpecSuite extends FlatSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala index e8971e36d112d..efdaac3ae1f0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset * The purpose of this suite is to make sure that generic FunSpec-based scala * tests work with a shared spark session */ -class GenericFunSpecSuite extends FunSpec with SharedSparkSession { +class GenericFunSpecSuite extends FunSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala index 44655a5345ca4..3a3540c1dbdbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset * The purpose of this suite is to make sure that generic WordSpec-based scala * tests work with a shared spark session */ -class GenericWordSpecSuite extends WordSpec with SharedSparkSession { +class GenericWordSpecSuite extends WordSpec with SharedSparkSessionBase { import testImplicits._ private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala deleted file mode 100644 index 0dd24d2d56b82..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ /dev/null @@ -1,44 +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.test - -trait SharedSQLContext extends SQLTestUtils with SharedSparkSession { - - /** - * Suites extending [[SharedSQLContext]] are sharing resources (eg. SparkSession) in their tests. - * That trait initializes the spark session in its [[beforeAll()]] implementation before the - * automatic thread snapshot is performed, so the audit code could fail to report threads leaked - * by that shared session. - * - * The behavior is overridden here to take the snapshot before the spark session is initialized. - */ - override protected val enableAutoThreadAudit = false - - protected override def beforeAll(): Unit = { - doThreadPreAudit() - super.beforeAll() - } - - protected override def afterAll(): Unit = { - try { - super.afterAll() - } finally { - doThreadPostAudit() - } - } -} 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 ff6211b95042f..ee29b4b8fb32b 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 @@ -28,10 +28,36 @@ import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +trait SharedSparkSession extends SQLTestUtils with SharedSparkSessionBase { + + /** + * Suites extending [[SharedSparkSession]] are sharing resources (eg. SparkSession) in their + * tests. That trait initializes the spark session in its [[beforeAll()]] implementation before + * the automatic thread snapshot is performed, so the audit code could fail to report threads + * leaked by that shared session. + * + * The behavior is overridden here to take the snapshot before the spark session is initialized. + */ + override protected val enableAutoThreadAudit = false + + protected override def beforeAll(): Unit = { + doThreadPreAudit() + super.beforeAll() + } + + protected override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + doThreadPostAudit() + } + } +} + /** * Helper trait for SQL test suites where all tests share a single [[TestSparkSession]]. */ -trait SharedSparkSession +trait SharedSparkSessionBase extends SQLTestUtilsBase with BeforeAndAfterEach with Eventually { self: Suite => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index a6f7f2250b586..a8e1a44f3d5d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -26,9 +26,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoTable, import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession -class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { +class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { import testImplicits._ import functions._ diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 535c32396b593..b1a907f9cba27 100644 --- a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** @@ -42,7 +42,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with SharedSQLContext { +class OrcFilterSuite extends OrcTest with SharedSparkSession { protected def checkFilterPredicate( df: DataFrame, diff --git a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 0f19c9e40b933..65b0537a0a8c1 100644 --- a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ /** @@ -42,7 +42,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with SharedSQLContext { +class OrcFilterSuite extends OrcTest with SharedSparkSession { protected def checkFilterPredicate( df: DataFrame, From ec14b6eb65ee129cbfa7baae67062306cd3548e5 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 19 Aug 2019 20:10:56 +0900 Subject: [PATCH 102/149] [SPARK-28393][SQL][PYTHON][TESTS] Convert and port 'pgSQL/join.sql' into UDF test base ## What changes were proposed in this pull request? This PR adds some tests converted from ```pgSQL/join.sql``` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921).
Diff comparing to 'join.sql'

```diff diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out index f75fe05196..ad2b5dd0db 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out -240,10 +240,10 struct<> -- !query 27 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) FROM J1_TBL AS tx -- !query 27 schema -struct +struct -- !query 27 output 0 NULL zero 1 4 one -259,10 +259,10 struct -- !query 28 -SELECT '' AS `xxx`, * +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) FROM J1_TBL tx -- !query 28 schema -struct +struct -- !query 28 output 0 NULL zero 1 4 one -278,10 +278,10 struct -- !query 29 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, a, udf(udf(b)), c FROM J1_TBL AS t1 (a, b, c) -- !query 29 schema -struct +struct -- !query 29 output 0 NULL zero 1 4 one -297,10 +297,10 struct -- !query 30 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) FROM J1_TBL t1 (a, b, c) -- !query 30 schema -struct +struct -- !query 30 output 0 NULL zero 1 4 one -316,10 +316,10 struct -- !query 31 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) -- !query 31 schema -struct +struct -- !query 31 output 0 NULL zero 0 NULL 0 NULL zero 1 -1 -423,7 +423,7 struct -- !query 32 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, * FROM J1_TBL CROSS JOIN J2_TBL -- !query 32 schema struct -530,20 +530,20 struct -- !query 33 -SELECT '' AS `xxx`, i, k, t +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t FROM J1_TBL CROSS JOIN J2_TBL -- !query 33 schema struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 20 +Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 29 -- !query 34 -SELECT '' AS `xxx`, t1.i, k, t +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) FROM J1_TBL t1 CROSS JOIN J2_TBL t2 -- !query 34 schema -struct +struct -- !query 34 output 0 -1 zero 0 -3 zero -647,11 +647,11 struct -- !query 35 -SELECT '' AS `xxx`, ii, tt, kk +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) FROM (J1_TBL CROSS JOIN J2_TBL) AS tx (ii, jj, tt, ii2, kk) -- !query 35 schema -struct +struct -- !query 35 output 0 zero -1 0 zero -3 -755,10 +755,10 struct -- !query 36 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b -- !query 36 schema -struct +struct -- !query 36 output 0 NULL zero 0 NULL 0 NULL 0 NULL zero 0 NULL 1 -1 -1654,10 +1654,10 struct -- !query 37 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) FROM J1_TBL INNER JOIN J2_TBL USING (i) -- !query 37 schema -struct +struct -- !query 37 output 0 NULL zero NULL 1 4 one -1 -1669,10 +1669,10 struct -- !query 38 -SELECT '' AS `xxx`, * +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k FROM J1_TBL JOIN J2_TBL USING (i) -- !query 38 schema -struct +struct -- !query 38 output 0 NULL zero NULL 1 4 one -1 -1684,9 +1684,9 struct -- !query 39 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, * FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) - ORDER BY a, d + ORDER BY udf(udf(a)), udf(d) -- !query 39 schema struct -- !query 39 output -1700,10 +1700,10 struct -- !query 40 -SELECT '' AS `xxx`, * +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) FROM J1_TBL NATURAL JOIN J2_TBL -- !query 40 schema -struct +struct -- !query 40 output 0 NULL zero NULL 1 4 one -1 -1715,10 +1715,10 struct -- !query 41 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) -- !query 41 schema -struct +struct -- !query 41 output 0 NULL zero NULL 1 4 one -1 -1730,10 +1730,10 struct -- !query 42 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) -- !query 42 schema -struct +struct -- !query 42 output 0 NULL zero NULL 2 3 two 2 -1741,10 +1741,10 struct -- !query 43 -SELECT '' AS `xxx`, * - FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i) +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i) -- !query 43 schema -struct +struct -- !query 43 output 0 NULL zero 0 NULL 1 4 one 1 -1 -1756,10 +1756,10 struct -- !query 44 -SELECT '' AS `xxx`, * - FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k) +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)) -- !query 44 schema -struct +struct -- !query 44 output 0 NULL zero NULL 0 2 3 two 2 2 -1767,10 +1767,10 struct -- !query 45 -SELECT '' AS `xxx`, * - FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k) +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))) -- !query 45 schema -struct +struct -- !query 45 output 0 NULL zero 2 2 0 NULL zero 2 4 -1784,11 +1784,11 struct -- !query 46 -SELECT '' AS `xxx`, * +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) - ORDER BY i, k, t + ORDER BY udf(udf(i)), udf(k), udf(t) -- !query 46 schema -struct +struct -- !query 46 output NULL NULL null NULL NULL 0 zero NULL -1806,11 +1806,11 struct -- !query 47 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) FROM J1_TBL LEFT JOIN J2_TBL USING (i) - ORDER BY i, k, t + ORDER BY udf(i), udf(udf(k)), udf(t) -- !query 47 schema -struct +struct -- !query 47 output NULL NULL null NULL NULL 0 zero NULL -1828,10 +1828,10 struct -- !query 48 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) -- !query 48 schema -struct +struct -- !query 48 output 0 NULL zero NULL 1 4 one -1 -1845,10 +1845,10 struct -- !query 49 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) FROM J1_TBL RIGHT JOIN J2_TBL USING (i) -- !query 49 schema -struct +struct -- !query 49 output 0 NULL zero NULL 1 4 one -1 -1862,11 +1862,11 struct -- !query 50 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) - ORDER BY i, k, t + ORDER BY udf(udf(i)), udf(k), udf(t) -- !query 50 schema -struct +struct -- !query 50 output NULL NULL NULL NULL NULL NULL null NULL -1886,11 +1886,11 struct -- !query 51 -SELECT '' AS `xxx`, * +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) FROM J1_TBL FULL JOIN J2_TBL USING (i) - ORDER BY i, k, t + ORDER BY udf(udf(i)), udf(k), udf(udf(t)) -- !query 51 schema -struct +struct -- !query 51 output NULL NULL NULL NULL NULL NULL null NULL -1910,19 +1910,19 struct -- !query 52 -SELECT '' AS `xxx`, * - FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1) +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1) -- !query 52 schema -struct +struct -- !query 52 output -- !query 53 -SELECT '' AS `xxx`, * - FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1) +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)) -- !query 53 schema -struct +struct -- !query 53 output 1 4 one -1 -2020,9 +2020,9 ee NULL 42 NULL -- !query 65 SELECT * FROM -(SELECT * FROM t2) as s2 +(SELECT udf(name) as name, t2.n FROM t2) as s2 INNER JOIN -(SELECT * FROM t3) s3 +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 USING (name) -- !query 65 schema struct -2033,9 +2033,9 cc 22 23 -- !query 66 SELECT * FROM -(SELECT * FROM t2) as s2 +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 LEFT JOIN -(SELECT * FROM t3) s3 +(SELECT udf(name) as name, t3.n FROM t3) s3 USING (name) -- !query 66 schema struct -2046,13 +2046,13 ee 42 NULL -- !query 67 -SELECT * FROM +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM (SELECT * FROM t2) as s2 FULL JOIN (SELECT * FROM t3) s3 USING (name) -- !query 67 schema -struct +struct -- !query 67 output bb 12 13 cc 22 23 -2062,9 +2062,9 ee 42 NULL -- !query 68 SELECT * FROM -(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 NATURAL INNER JOIN -(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query 68 schema struct -- !query 68 output -2074,9 +2074,9 cc 22 2 23 3 -- !query 69 SELECT * FROM -(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL LEFT JOIN -(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query 69 schema struct -- !query 69 output -2087,9 +2087,9 ee 42 2 NULL NULL -- !query 70 SELECT * FROM -(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL FULL JOIN -(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3 -- !query 70 schema struct -- !query 70 output -2101,11 +2101,11 ee 42 2 NULL NULL -- !query 71 SELECT * FROM -(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 NATURAL INNER JOIN -(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL INNER JOIN -(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query 71 schema struct -- !query 71 output -2114,11 +2114,11 bb 11 1 12 2 13 3 -- !query 72 SELECT * FROM -(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 NATURAL FULL JOIN -(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 NATURAL FULL JOIN -(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query 72 schema struct -- !query 72 output -2129,16 +2129,16 ee NULL NULL 42 2 NULL NULL -- !query 73 -SELECT * FROM -(SELECT name, n as s1_n FROM t1) as s1 +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 NATURAL FULL JOIN (SELECT * FROM - (SELECT name, n as s2_n FROM t2) as s2 + (SELECT name, udf(n) as s2_n FROM t2) as s2 NATURAL FULL JOIN - (SELECT name, n as s3_n FROM t3) as s3 + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 -- !query 73 schema -struct +struct -- !query 73 output bb 11 12 13 cc NULL 22 23 -2151,9 +2151,9 SELECT * FROM (SELECT name, n as s1_n FROM t1) as s1 NATURAL FULL JOIN (SELECT * FROM - (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL FULL JOIN - (SELECT name, n as s3_n FROM t3) as s3 + (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 -- !query 74 schema struct -2165,13 +2165,13 ee NULL 42 2 NULL -- !query 75 -SELECT * FROM - (SELECT name, n as s1_n FROM t1) as s1 +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 -ON (s1_n = s2_n) +ON (udf(udf(s1_n)) = udf(s2_n)) -- !query 75 schema -struct +struct -- !query 75 output NULL NULL bb 2 NULL NULL cc 2 -2200,9 +2200,9 struct<> -- !query 78 -select * from x +select udf(udf(x1)), udf(x2) from x -- !query 78 schema -struct +struct -- !query 78 output 1 11 2 22 -2212,9 +2212,9 struct -- !query 79 -select * from y +select udf(y1), udf(udf(y2)) from y -- !query 79 schema -struct +struct -- !query 79 output 1 111 2 222 -2223,7 +2223,7 struct -- !query 80 -select * from x left join y on (x1 = y1 and x2 is not null) +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null) -- !query 80 schema struct -- !query 80 output -2235,7 +2235,7 struct -- !query 81 -select * from x left join y on (x1 = y1 and y2 is not null) +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null) -- !query 81 schema struct -- !query 81 output -2247,8 +2247,8 struct -- !query 82 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1) +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)) -- !query 82 schema struct -- !query 82 output -2260,8 +2260,8 struct -- !query 83 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1 and x2 is not null) +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null) -- !query 83 schema struct -- !query 83 output -2273,8 +2273,8 struct -- !query 84 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1 and y2 is not null) +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null) -- !query 84 schema struct -- !query 84 output -2286,8 +2286,8 struct -- !query 85 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1 and xx2 is not null) +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null) -- !query 85 schema struct -- !query 85 output -2299,8 +2299,8 struct -- !query 86 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1) where (x2 is not null) +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null) -- !query 86 schema struct -- !query 86 output -2310,8 +2310,8 struct -- !query 87 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1) where (y2 is not null) +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null) -- !query 87 schema struct -- !query 87 output -2321,8 +2321,8 struct -- !query 88 -select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) -on (x1 = xx1) where (xx2 is not null) +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null) -- !query 88 schema struct -- !query 88 output -2332,75 +2332,75 struct -- !query 89 -select count(*) from tenk1 a where unique1 in - (select unique1 from tenk1 b join tenk1 c using (unique1) - where b.unique2 = 42) +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)) -- !query 89 schema -struct +struct -- !query 89 output 1 -- !query 90 -select count(*) from tenk1 x where - x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and - x.unique1 = 0 and - x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))) -- !query 90 schema -struct +struct -- !query 90 output 1 -- !query 91 -select count(*) from tenk1 x where - x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and - x.unique1 = 0 and - x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))) -- !query 91 schema -struct +struct -- !query 91 output 1 -- !query 92 select * from int8_tbl i1 left join (int8_tbl i2 join - (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 -order by 1, 2 + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2 -- !query 92 schema struct -- !query 92 output -123 456 123 456 123 -123 4567890123456789 123 4567890123456789 123 4567890123456789 -4567890123456789 NULL NULL NULL 4567890123456789 123 NULL NULL NULL +123 456 123 456 123 +123 4567890123456789 123 4567890123456789 123 4567890123456789 4567890123456789 123 4567890123456789 123 -- !query 93 -select count(*) +select udf(count(*)) from - (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2 + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 from tenk1 t1 - left join tenk1 t2 on t1.unique1 = t2.unique1 - join tenk1 t3 on t1.unique2 = t3.unique2) ss, + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, tenk1 t4, tenk1 t5 -where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)) -- !query 93 schema -struct +struct -- !query 93 output 1000 -- !query 94 -select a.f1, b.f1, t.thousand, t.tenthous from +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from tenk1 t, - (select sum(f1)+1 as f1 from int4_tbl i4a) a, - (select sum(f1) as f1 from int4_tbl i4b) b -where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)) -- !query 94 schema -struct +struct -- !query 94 output -2408,8 +2408,8 struct -- !query 95 select * from j1_tbl full join - (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl - on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k) -- !query 95 schema struct -- !query 95 output -2435,13 +2435,13 NULL NULL null NULL NULL -- !query 96 -select count(*) from - (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x left join - (select * from tenk1 y order by y.unique2) y - on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2 + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2 -- !query 96 schema -struct +struct -- !query 96 output 10000 -2507,7 +2507,7 struct<> -- !query 104 -select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol) -- !query 104 schema struct -- !query 104 output -2517,7 +2517,7 struct -- !query 105 -select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)) -- !query 105 schema struct -- !query 105 output -2527,10 +2527,10 struct -- !query 106 -select count(*) from tenk1 a, tenk1 b - where a.hundred = b.thousand and (b.fivethous % 10) < 10 +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10) -- !query 106 schema -struct +struct -- !query 106 output 100000 -2584,14 +2584,14 struct<> -- !query 113 -SELECT a.f1 +SELECT udf(udf(a.f1)) as f1 FROM tt4 a LEFT JOIN ( SELECT b.f1 - FROM tt3 b LEFT JOIN tt3 c ON (b.f1 = c.f1) - WHERE c.f1 IS NULL -) AS d ON (a.f1 = d.f1) -WHERE d.f1 IS NULL + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL -- !query 113 schema struct -- !query 113 output -2621,7 +2621,7 struct<> -- !query 116 -select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)) -- !query 116 schema struct -- !query 116 output -2649,12 +2649,12 struct<> -- !query 119 -select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, - xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx from yy - left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy - left join xx xxa on yya.pkxx = xxa.pkxx - left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx) -- !query 119 schema struct -- !query 119 output -2693,9 +2693,9 struct<> -- !query 123 select * from - zt2 left join zt3 on (f2 = f3) - left join zt1 on (f3 = f1) -where f2 = 53 + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53 -- !query 123 schema struct -- !query 123 output -2712,9 +2712,9 struct<> -- !query 125 select * from - zt2 left join zt3 on (f2 = f3) - left join zv1 on (f3 = f1) -where f2 = 53 + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53 -- !query 125 schema struct -- !query 125 output -2722,12 +2722,12 struct -- !query 126 -select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred -from tenk1 a left join tenk1 b on a.unique2 = b.tenthous -where a.unique1 = 42 and - ((b.unique2 is null and a.ten = 2) or b.hundred = 3) +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))) -- !query 126 schema -struct +struct -- !query 126 output -2749,7 +2749,7 struct<> -- !query 129 -select * from a left join b on i = x and i = y and x = i +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i) -- !query 129 schema struct -- !query 129 output -2757,11 +2757,11 struct -- !query 130 -select t1.q2, count(t2.*) -from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) -group by t1.q2 order by 1 +select udf(t1.q2), udf(count(t2.*)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1 -- !query 130 schema -struct +struct -- !query 130 output -4567890123456789 0 123 2 -2770,11 +2770,11 struct -- !query 131 -select t1.q2, count(t2.*) -from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) -group by t1.q2 order by 1 +select udf(udf(t1.q2)), udf(count(t2.*)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1 -- !query 131 schema -struct +struct -- !query 131 output -4567890123456789 0 123 2 -2783,13 +2783,13 struct -- !query 132 -select t1.q2, count(t2.*) +select udf(t1.q2) as q2, udf(udf(count(t2.*))) from int8_tbl t1 left join - (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 - on (t1.q2 = t2.q1) + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) group by t1.q2 order by 1 -- !query 132 schema -struct +struct -- !query 132 output -4567890123456789 0 123 2 -2828,17 +2828,17 struct<> -- !query 136 -select c.name, ss.code, ss.b_cnt, ss.const +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) from c left join (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const from a left join - (select count(1) as cnt, b.a from b group by b.a) as b_grp - on a.code = b_grp.a + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) ) as ss - on (c.a = ss.code) + on (udf(udf(c.a)) = udf(ss.code)) order by c.name -- !query 136 schema -struct +struct -- !query 136 output A p 2 -1 B q 0 -1 -2852,15 +2852,15 LEFT JOIN ( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM ( SELECT 1 as key3 ) sub3 LEFT JOIN - ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM ( SELECT 1 as key5 ) sub5 LEFT JOIN ( SELECT 2 as key6, 42 as value1 ) sub6 - ON sub5.key5 = sub6.key6 + ON sub5.key5 = udf(sub6.key6) ) sub4 - ON sub4.key5 = sub3.key3 + ON udf(sub4.key5) = sub3.key3 ) sub2 -ON sub1.key1 = sub2.key3 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)) -- !query 137 schema struct -- !query 137 output -2871,34 +2871,34 struct SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN -( SELECT sub3.key3, value2, COALESCE(value2, 66) as value3 FROM +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM ( SELECT 1 as key3 ) sub3 LEFT JOIN ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM ( SELECT 1 as key5 ) sub5 LEFT JOIN ( SELECT 2 as key6, 42 as value1 ) sub6 - ON sub5.key5 = sub6.key6 + ON udf(udf(sub5.key5)) = sub6.key6 ) sub4 ON sub4.key5 = sub3.key3 ) sub2 -ON sub1.key1 = sub2.key3 +ON sub1.key1 = udf(udf(sub2.key3)) -- !query 138 schema -struct +struct -- !query 138 output 1 1 1 1 -- !query 139 -SELECT qq, unique1 +SELECT udf(qq), udf(udf(unique1)) FROM - ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 FULL OUTER JOIN - ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 USING (qq) - INNER JOIN tenk1 c ON qq = unique2 + INNER JOIN tenk1 c ON udf(qq) = udf(unique2) -- !query 139 schema -struct +struct -- !query 139 output 123 4596 123 4596 -2936,19 +2936,19 struct<> -- !query 143 -select nt3.id +select udf(nt3.id) from nt3 as nt3 left join - (select nt2.*, (nt2.b1 and ss1.a3) AS b3 + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 from nt2 as nt2 left join - (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 - on ss1.id = nt2.nt1_id + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) ) as ss2 - on ss2.id = nt3.nt2_id -where nt3.id = 1 and ss2.b3 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3) -- !query 143 schema -struct +struct -- !query 143 output 1 -3003,73 +3003,73 NULL 2147483647 -- !query 146 -select count(*) from - tenk1 a join tenk1 b on a.unique1 = b.unique2 - left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand - join int4_tbl on b.thousand = f1 +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1 -- !query 146 schema -struct +struct -- !query 146 output 10 -- !query 147 -select b.unique1 from - tenk1 a join tenk1 b on a.unique1 = b.unique2 - left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand - join int4_tbl i1 on b.thousand = f1 - right join int4_tbl i2 on i2.f1 = b.tenthous - order by 1 +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1) -- !query 147 schema -struct +struct -- !query 147 output NULL NULL +0 NULL NULL -0 -- !query 148 select * from ( - select unique1, q1, coalesce(unique1, -1) + q1 as fault - from int8_tbl left join tenk1 on (q2 = unique2) + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) ) ss -where fault = 122 -order by fault +where udf(fault) = udf(122) +order by udf(fault) -- !query 148 schema -struct +struct -- !query 148 output NULL 123 122 -- !query 149 -select q1, unique2, thousand, hundred - from int8_tbl a left join tenk1 b on q1 = unique2 - where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))) -- !query 149 schema -struct +struct -- !query 149 output -- !query 150 -select f1, unique2, case when unique2 is null then f1 else 0 end - from int4_tbl a left join tenk1 b on f1 = unique2 - where (case when unique2 is null then f1 else 0 end) = 0 +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0 -- !query 150 schema -struct +struct -- !query 150 output 0 0 0 -- !query 151 -select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) - from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) - where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44) -- !query 151 schema -struct +struct -- !query 151 output -3078,11 +3078,11 struct -- !query 152 output -3092,10 +3092,10 doh! 123 456 hi de ho neighbor NULL -- !query 153 select * from - (select 1 as id) as xx + (select udf(udf(1)) as id) as xx left join - (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) - on (xx.id = coalesce(yy.id)) + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))) -- !query 153 schema struct -- !query 153 output -3103,11 +3103,11 struct 0 +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0) -- !query 154 schema -struct +struct -- !query 154 output -4567890123456789 NULL 123 123 -3142,7 +3142,7 struct<> -- !query 157 -select p.* from parent p left join child c on (p.k = c.k) +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)) -- !query 157 schema struct -- !query 157 output -3153,8 +3153,8 struct -- !query 158 select p.*, linked from parent p - left join (select c.*, true as linked from child c) as ss - on (p.k = ss.k) + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))) -- !query 158 schema struct -- !query 158 output -3165,8 +3165,8 struct -- !query 159 select p.* from - parent p left join child c on (p.k = c.k) - where p.k = 1 and p.k = 2 + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)) -- !query 159 schema struct -- !query 159 output -3175,8 +3175,8 struct -- !query 160 select p.* from - (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k - where p.k = 1 and p.k = 2 + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)) -- !query 160 schema struct -- !query 160 output -3204,7 +3204,7 struct<> -- !query 163 -SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0) -- !query 163 schema struct -- !query 163 output -3212,7 +3212,7 struct -- !query 164 -SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0) -- !query 164 schema struct -- !query 164 output -3231,13 +3231,13 struct<> -- !query 166 SELECT * FROM - (SELECT 1 AS x) ss1 + (SELECT udf(1) AS x) ss1 LEFT JOIN - (SELECT q1, q2, COALESCE(dat1, q1) AS y - FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 ON true -- !query 166 schema -struct +struct -- !query 166 output 1 123 456 123 1 123 4567890123456789 123 -3248,27 +3248,27 struct -- !query 167 select * from - int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1) -- !query 167 schema struct<> -- !query 167 output org.apache.spark.sql.AnalysisException -Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63 +Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 72 -- !query 168 select * from - int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) -- !query 168 schema struct<> -- !query 168 output org.apache.spark.sql.AnalysisException -cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 +cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 -- !query 169 select * from - int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)) -- !query 169 schema struct -- !query 169 output -3276,69 +3276,69 struct -- !query 170 -select t1.uunique1 from - tenk1 t1 join tenk2 t2 on t1.two = t2.two +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) -- !query 170 schema struct<> -- !query 170 output org.apache.spark.sql.AnalysisException -cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 +cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 -- !query 171 -select t2.uunique1 from - tenk1 t1 join tenk2 t2 on t1.two = t2.two +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two -- !query 171 schema struct<> -- !query 171 output org.apache.spark.sql.AnalysisException -cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 +cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 15 -- !query 172 -select uunique1 from - tenk1 t1 join tenk2 t2 on t1.two = t2.two +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) -- !query 172 schema struct<> -- !query 172 output org.apache.spark.sql.AnalysisException -cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 +cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 -- !query 173 -select f1,g from int4_tbl a, (select f1 as g) ss +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss -- !query 173 schema struct<> -- !query 173 output org.apache.spark.sql.AnalysisException -cannot resolve '`f1`' given input columns: []; line 1 pos 37 +cannot resolve '`f1`' given input columns: []; line 1 pos 55 -- !query 174 -select f1,g from int4_tbl a, (select a.f1 as g) ss +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss -- !query 174 schema struct<> -- !query 174 output org.apache.spark.sql.AnalysisException -cannot resolve '`a.f1`' given input columns: []; line 1 pos 37 +cannot resolve '`a.f1`' given input columns: []; line 1 pos 42 -- !query 175 -select f1,g from int4_tbl a cross join (select f1 as g) ss +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss -- !query 175 schema struct<> -- !query 175 output org.apache.spark.sql.AnalysisException -cannot resolve '`f1`' given input columns: []; line 1 pos 47 +cannot resolve '`f1`' given input columns: []; line 1 pos 61 -- !query 176 -select f1,g from int4_tbl a cross join (select a.f1 as g) ss +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss -- !query 176 schema struct<> -- !query 176 output org.apache.spark.sql.AnalysisException -cannot resolve '`a.f1`' given input columns: []; line 1 pos 47 +cannot resolve '`a.f1`' given input columns: []; line 1 pos 60 -- !query 177 -3383,8 +3383,8 struct<> -- !query 182 select * from j1 -inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 -where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1 -- !query 182 schema struct -- !query 182 output ```

## How was this patch tested? Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921). Closes #25371 from huaxingao/spark-28393. Authored-by: Huaxin Gao Signed-off-by: HyukjinKwon --- .../sql-tests/inputs/udf/pgSQL/udf-join.sql | 2081 ++++++++++ .../results/udf/pgSQL/udf-join.sql.out | 3408 +++++++++++++++++ 2 files changed, 5489 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql new file mode 100644 index 0000000000000..c05aa156a13bf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql @@ -0,0 +1,2081 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- JOIN +-- Test JOIN clauses +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql +-- +-- This test file was converted from pgSQL/join.sql. + +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2); +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet; + +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet; + + +INSERT INTO J1_TBL VALUES (1, 4, 'one'); +INSERT INTO J1_TBL VALUES (2, 3, 'two'); +INSERT INTO J1_TBL VALUES (3, 2, 'three'); +INSERT INTO J1_TBL VALUES (4, 1, 'four'); +INSERT INTO J1_TBL VALUES (5, 0, 'five'); +INSERT INTO J1_TBL VALUES (6, 6, 'six'); +INSERT INTO J1_TBL VALUES (7, 7, 'seven'); +INSERT INTO J1_TBL VALUES (8, 8, 'eight'); +INSERT INTO J1_TBL VALUES (0, NULL, 'zero'); +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null'); +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero'); + +INSERT INTO J2_TBL VALUES (1, -1); +INSERT INTO J2_TBL VALUES (2, 2); +INSERT INTO J2_TBL VALUES (3, -3); +INSERT INTO J2_TBL VALUES (2, 4); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (0, NULL); +INSERT INTO J2_TBL VALUES (NULL, NULL); +INSERT INTO J2_TBL VALUES (NULL, 0); + +-- [SPARK-20856] Do not need onerow because it only used for test statement using nested joins +-- useful in some tests below +-- create temp table onerow(); +-- insert into onerow default values; +-- analyze onerow; + + +-- +-- CORRELATION NAMES +-- Make sure that table/column aliases are supported +-- before diving into more complex join syntax. +-- + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) + FROM J1_TBL AS tx; + +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) + FROM J1_TBL tx; + +SELECT udf('') AS `xxx`, a, udf(udf(b)), c + FROM J1_TBL AS t1 (a, b, c); + +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) + FROM J1_TBL t1 (a, b, c); + +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS "xxx", t1.a, t2.e +-- FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- WHERE t1.a = t2.d; + + +-- +-- CROSS JOIN +-- Qualifications are not allowed on cross joins, +-- which degenerate into a standard unqualified inner join. +-- + +SELECT udf('') AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL; + +-- ambiguous column +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t + FROM J1_TBL CROSS JOIN J2_TBL; + +-- resolve previous ambiguity by specifying the table name +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) + FROM J1_TBL t1 CROSS JOIN J2_TBL t2; + +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, tx.ii, tx.jj, tx.kk +-- FROM (J1_TBL t1 (a, b, c) CROSS JOIN J2_TBL t2 (d, e)) +-- AS tx (ii, jj, tt, ii2, kk); + +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b; + + +-- +-- +-- Inner joins (equi-joins) +-- +-- + +-- +-- Inner joins (equi-joins) with USING clause +-- The USING syntax changes the shape of the resulting table +-- by including a column in the USING clause only once in the result. +-- + +-- Inner equi-join on specified column +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) + FROM J1_TBL INNER JOIN J2_TBL USING (i); + +-- Same as above, slightly different syntax +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k + FROM J1_TBL JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY udf(udf(a)), udf(d); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, b) USING (b) +-- ORDER BY b, t1.a; + + +-- +-- NATURAL JOIN +-- Inner equi-join on all columns with the same name +-- + +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL NATURAL JOIN J2_TBL; + +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d); + +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- mismatch number of columns +-- currently, Postgres will fill in with underlying names +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b) NATURAL JOIN J2_TBL t2 (a); + + +-- +-- Inner joins (equi-joins) +-- + +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i); + +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)); + + +-- +-- Non-equi-joins +-- + +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))); + + +-- +-- Outer joins +-- Note that OUTER is a noise word +-- + +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY udf(i), udf(udf(k)), udf(t); + +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) + FROM J1_TBL RIGHT JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t); + +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(udf(t)); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)); + +-- +-- semijoin selectivity for <> +-- +-- explain (costs off) +-- select * from int4_tbl i4, tenk1 a +-- where exists(select * from tenk1 b +-- where a.twothousand = b.twothousand and a.fivethous <> b.fivethous) +-- and i4.f1 = a.tenthous; + + +-- +-- More complicated constructs +-- + +-- +-- Multiway full join +-- + +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet; + +INSERT INTO t1 VALUES ( 'bb', 11 ); +INSERT INTO t2 VALUES ( 'bb', 12 ); +INSERT INTO t2 VALUES ( 'cc', 22 ); +INSERT INTO t2 VALUES ( 'ee', 42 ); +INSERT INTO t3 VALUES ( 'bb', 13 ); +INSERT INTO t3 VALUES ( 'cc', 23 ); +INSERT INTO t3 VALUES ( 'dd', 33 ); + +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name); + +-- +-- Test interactions of join syntax and subqueries +-- + +-- Basic cases (we expect planner to pull up the subquery here) +SELECT * FROM +(SELECT udf(name) as name, t2.n FROM t2) as s2 +INNER JOIN +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 +USING (name); + +SELECT * FROM +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 +LEFT JOIN +(SELECT udf(name) as name, t3.n FROM t3) s3 +USING (name); + +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name); + +-- Cases with non-nullable expressions in subquery results; +-- make sure these go to null as expected +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3; + +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(n) as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 + ) ss2; + +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(n) as s3_n FROM t3) as s3 + ) ss2; + +-- Constants as join keys can also be problematic +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (udf(udf(s1_n)) = udf(s2_n)); + + +-- Test for propagation of nullability constraints into sub-joins + +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2); + +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2); + +select udf(udf(x1)), udf(x2) from x; +select udf(y1), udf(udf(y2)) from y; + +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null); +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null); + +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null); +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null); +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null); +-- these should NOT give the same answers as above +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null); + +-- +-- regression test: check for bug with propagation of implied equality +-- to outside an IN +-- +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)); + +-- +-- regression test: check for failure to generate a plan with multiple +-- degenerate IN clauses +-- +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))); + +-- try that with GEQO too +-- begin; +-- set geqo = on; +-- set geqo_threshold = 2; +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))); +-- rollback; + +-- Skip this test because table b inherits from table a and we do not support this feature, see inherits.sql +-- +-- regression test: be sure we cope with proven-dummy append rels +-- +-- explain (costs off) +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- +-- regression test: check handling of empty-FROM subquery underneath outer join +-- +-- explain (costs off) +-- select * from int8_tbl i1 left join (int8_tbl i2 join +-- (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +-- order by 1, 2; + +select * from int8_tbl i1 left join (int8_tbl i2 join + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2; + +-- +-- regression test: check a case where join_clause_is_movable_into() gives +-- an imprecise result, causing an assertion failure +-- +select udf(count(*)) +from + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 + from tenk1 t1 + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, + tenk1 t4, + tenk1 t5 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)); + +-- +-- regression test: check a case where we formerly missed including an EC +-- enforcement clause because it was expected to be handled at scan level +-- +-- explain (costs off) +-- select a.f1, b.f1, t.thousand, t.tenthous from +-- tenk1 t, +-- (select sum(f1)+1 as f1 from int4_tbl i4a) a, +-- (select sum(f1) as f1 from int4_tbl i4b) b +-- where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous; + +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from + tenk1 t, + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)); + +-- +-- check a case where we formerly got confused by conflicting sort orders +-- in redundant merge join path keys +-- +-- explain (costs off) +-- select * from +-- j1_tbl full join +-- (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl +-- on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k; + +select * from + j1_tbl full join + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k); + +-- +-- a different check for handling of redundant sort keys in merge joins +-- +-- explain (costs off) +-- select count(*) from +-- (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x +-- left join +-- (select * from tenk1 y order by y.unique2) y +-- on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2; + +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x + left join + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2; + + +-- +-- Clean up +-- + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; + +DROP TABLE J1_TBL; +DROP TABLE J2_TBL; + +-- Both DELETE and UPDATE allow the specification of additional tables +-- to "join" against to determine which rows should be modified. + +-- CREATE TEMP TABLE t1 (a int, b int); +-- CREATE TEMP TABLE t2 (a int, b int); +-- CREATE TEMP TABLE t3 (x int, y int); + +-- INSERT INTO t1 VALUES (5, 10); +-- INSERT INTO t1 VALUES (15, 20); +-- INSERT INTO t1 VALUES (100, 100); +-- INSERT INTO t1 VALUES (200, 1000); +-- INSERT INTO t2 VALUES (200, 2000); +-- INSERT INTO t3 VALUES (5, 20); +-- INSERT INTO t3 VALUES (6, 7); +-- INSERT INTO t3 VALUES (7, 8); +-- INSERT INTO t3 VALUES (500, 100); + +-- DELETE FROM t3 USING t1 table1 WHERE t3.x = table1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t1 JOIN t2 USING (a) WHERE t3.x > t1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t3 t3_other WHERE t3.x = t3_other.x AND t3.y = t3_other.y; +-- SELECT * FROM t3; + +-- Test join against inheritance tree + +-- create temp table t2a () inherits (t2); + +-- insert into t2a values (200, 2001); + +-- select * from t1 left join t2 on (t1.a = t2.a); + +-- Test matching of column name with wrong alias + +-- select t1.x from t1 join t3 on (t1.a = t3.x); + +-- +-- regression test for 8.1 merge right join bug +-- + +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol); + +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol); + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- these should give the same results + +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol); + +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)); + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- +-- regression test for bug #13908 (hash join with skew tuples & nbatch increase) +-- + +-- set work_mem to '64kB'; +-- set enable_mergejoin to off; + +-- explain (costs off) +-- select count(*) from tenk1 a, tenk1 b +-- where a.hundred = b.thousand and (b.fivethous % 10) < 10; +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10); + +-- reset work_mem; +-- reset enable_mergejoin; + +-- +-- regression test for 8.2 bug with improper re-ordering of left joins +-- + +DROP TABLE IF EXISTS tt3; +CREATE TABLE tt3(f1 int, f2 string) USING parquet; +INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x; +-- create index tt3i on tt3(f1); +-- analyze tt3; + +DROP TABLE IF EXISTS tt4; +CREATE TABLE tt4(f1 int) USING parquet; +INSERT INTO tt4 VALUES (0),(1),(9999); +-- analyze tt4; + +SELECT udf(udf(a.f1)) as f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL; + +-- +-- regression test for proper handling of outer joins within antijoins +-- + +-- create temp table tt4x(c1 int, c2 int, c3 int); + +-- explain (costs off) +-- select * from tt4x t1 +-- where not exists ( +-- select 1 from tt4x t2 +-- left join tt4x t3 on t2.c3 = t3.c1 +-- left join ( select t5.c1 as c1 +-- from tt4x t4 left join tt4x t5 on t4.c2 = t5.c1 +-- ) a1 on t3.c2 = a1.c1 +-- where t1.c1 = t2.c2 +-- ); + +-- +-- regression test for problems of the sort depicted in bug #3494 +-- + +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2); +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2); + +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)); + +-- +-- regression test for problems of the sort depicted in bug #3588 +-- + +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx); +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx); + +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx); + +-- +-- regression test for improper pushing of constants across outer-join clauses +-- (as seen in early 8.2.x releases) +-- + +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1); +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2); +create or replace temporary view zt3(f3 int) using parquet; + +select * from + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53; + +create temp view zv1 as select *,'dummy' AS junk from zt1; + +select * from + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53; + +-- +-- regression test for improper extraction of OR indexqual conditions +-- (as seen in early 8.3.x releases) +-- + +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))); + +-- +-- test proper positioning of one-time quals in EXISTS (8.4devel bug) +-- +-- prepare foo(bool) as +-- select count(*) from tenk1 a left join tenk1 b +-- on (a.unique2 = b.unique1 and exists +-- (select 1 from tenk1 c where c.thousand = b.unique2 and $1)); +-- execute foo(true); +-- execute foo(false); + +-- +-- test for sane behavior with noncanonical merge clauses, per bug #4926 +-- + +-- begin; + +-- set enable_mergejoin = 1; +-- set enable_hashjoin = 0; +-- set enable_nestloop = 0; + +create or replace temporary view a (i integer) using parquet; +create or replace temporary view b (x integer, y integer) using parquet; + +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i); + +-- rollback; + +-- +-- test handling of merge clauses using record_ops +-- +-- begin; + +-- create type mycomptype as (id int, v bigint); + +-- create temp table tidv (idv mycomptype); +-- create index on tidv (idv); + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- set enable_mergejoin = 0; + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- rollback; + +-- +-- test NULL behavior of whole-row Vars, per bug #5025 +-- +select udf(t1.q2), udf(count(t2.*)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1; + +select udf(udf(t1.q2)), udf(count(t2.*)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1; + +-- [SPARK-28330] Enhance query limit +-- select t1.q2, count(t2.*) +-- from int8_tbl t1 left join (select * from int8_tbl offset 0) t2 on (t1.q2 = t2.q1) +-- group by t1.q2 order by 1; + +select udf(t1.q2) as q2, udf(udf(count(t2.*))) +from int8_tbl t1 left join + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) +group by t1.q2 order by 1; + +-- +-- test incorrect failure to NULL pulled-up subexpressions +-- +-- begin; +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code); +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num); +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a); + +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) + ) as ss + on (udf(udf(c.a)) = udf(ss.code)) +order by c.name; + +-- rollback; + +-- +-- test incorrect handling of placeholders that only appear in targetlists, +-- per bug #6154 +-- +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = udf(sub6.key6) + ) sub4 + ON udf(sub4.key5) = sub3.key3 +) sub2 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)); + +-- test the path using join aliases, too +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON udf(udf(sub5.key5)) = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = udf(udf(sub2.key3)); + +-- +-- test case where a PlaceHolderVar is used as a nestloop parameter +-- + +-- EXPLAIN (COSTS OFF) +-- SELECT qq, unique1 +-- FROM +-- ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 +-- FULL OUTER JOIN +-- ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 +-- USING (qq) +-- INNER JOIN tenk1 c ON qq = unique2; + +SELECT udf(qq), udf(udf(unique1)) + FROM + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON udf(qq) = udf(unique2); + +-- +-- nested nestloops can require nested PlaceHolderVars +-- + +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2); +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2); +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1); +-- explain (costs off) +-- select nt3.id +-- from nt3 as nt3 +-- left join +-- (select nt2.*, (nt2.b1 and ss1.a3) AS b3 +-- from nt2 as nt2 +-- left join +-- (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 +-- on ss1.id = nt2.nt1_id +-- ) as ss2 +-- on ss2.id = nt3.nt2_id +-- where nt3.id = 1 and ss2.b3; + +select udf(nt3.id) +from nt3 as nt3 + left join + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) + ) as ss2 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3); + +-- [SPARK-28379] Correlated scalar subqueries must be aggregated +-- +-- test case where a PlaceHolderVar is propagated into a subquery +-- + +-- explain (costs off) +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- +-- test the corner cases FULL JOIN ON TRUE and FULL JOIN ON FALSE +-- +select * from int4_tbl a full join int4_tbl b on true; +select * from int4_tbl a full join int4_tbl b on false; + +-- +-- test for ability to use a cartesian join when necessary +-- + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where q1 = thousand or q2 = thousand; + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where thousand = (q1 + q2); + +-- +-- test ability to generate a suitable plan for a star-schema query +-- + +-- explain (costs off) +-- select * from +-- tenk1, int8_tbl a, int8_tbl b +-- where thousand = a.q1 and tenthous = b.q1 and a.q2 = 1 and b.q2 = 2; + +-- +-- test a corner case in which we shouldn't apply the star-schema optimization +-- + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- variant that isn't quite a star-schema case + +-- [SPARK-16452] basic INFORMATION_SCHEMA support +-- select ss1.d1 from +-- tenk1 as t1 +-- inner join tenk1 as t2 +-- on t1.tenthous = t2.ten +-- inner join +-- int8_tbl as i8 +-- left join int4_tbl as i4 +-- inner join (select 64::information_schema.cardinal_number as d1 +-- from tenk1 t3, +-- lateral (select abs(t3.unique1) + random()) ss0(x) +-- where t3.fivethous < 0) as ss1 +-- on i4.f1 = ss1.d1 +-- on i8.q1 = i4.f1 +-- on t1.tenthous = ss1.d1 +-- where t1.unique1 < i4.f1; + +-- this variant is foldable by the remove-useless-RESULT-RTEs code + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- +-- test extraction of restriction OR clauses from join OR clause +-- (we used to only do this for indexable clauses) +-- + +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.hundred = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.ten = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or +-- ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); + +-- +-- test placement of movable quals in a parameterized join tree +-- + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten + t2.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select count(*) from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand +-- join int4_tbl on b.thousand = f1; + +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1; + +-- explain (costs off) +-- select b.unique1 from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand +-- join int4_tbl i1 on b.thousand = f1 +-- right join int4_tbl i2 on i2.f1 = b.tenthous +-- order by 1; + +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1); + +-- explain (costs off) +-- select * from +-- ( +-- select unique1, q1, coalesce(unique1, -1) + q1 as fault +-- from int8_tbl left join tenk1 on (q2 = unique2) +-- ) ss +-- where fault = 122 +-- order by fault; + +select * from +( + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) +) ss +where udf(fault) = udf(122) +order by udf(fault); + +-- explain (costs off) +-- select * from +-- (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- [SPARK-28382] Array Functions: unnest +-- select * from +-- (values (1, array(10,20)), (2, array(20,30))) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- +-- test handling of potential equivalence clauses above outer joins +-- + +-- explain (costs off) +-- select q1, unique2, thousand, hundred +-- from int8_tbl a left join tenk1 b on q1 = unique2 +-- where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123); + +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))); + +-- explain (costs off) +-- select f1, unique2, case when unique2 is null then f1 else 0 end +-- from int4_tbl a left join tenk1 b on f1 = unique2 +-- where (case when unique2 is null then f1 else 0 end) = 0; + +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0; + +-- +-- another case with equivalence clauses above outer joins (bug #8591) +-- + +-- explain (costs off) +-- select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) +-- from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) +-- where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44; + +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44); + +-- +-- check handling of join aliases when flattening multiple levels of subquery +-- + +-- explain (verbose, costs off) +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + +-- [SPARK-20856] Support statement using nested joins +-- +-- test successful handling of nested outer joins with degenerate join quals +-- + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- inner join int8_tbl i8 +-- on i8.q2 = 456 +-- right join text_tbl t2 +-- on t1.f1 = 'doh!' +-- left join int4_tbl i4 +-- on i8.q1 = i4.f1; + +select * from + text_tbl t1 + inner join int8_tbl i8 + on udf(i8.q2) = udf(udf(456)) + right join text_tbl t2 + on udf(t1.f1) = udf(udf('doh!')) + left join int4_tbl i4 + on udf(udf(i8.q1)) = i4.f1; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- test for appropriate join order in the presence of lateral references +-- + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +-- where t1.f1 = ss.f1; + +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +-- where t1.f1 = ss.f1; + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, +-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +-- where t1.f1 = ss2.f1; + +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, +-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +-- where t1.f1 = ss2.f1; + +-- explain (verbose, costs off) +-- select 1 from +-- text_tbl as tt1 +-- inner join text_tbl as tt2 on (tt1.f1 = 'foo') +-- left join text_tbl as tt3 on (tt3.f1 = 'foo') +-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1), +-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +-- where tt1.f1 = ss1.c0; + +-- select 1 from +-- text_tbl as tt1 +-- inner join text_tbl as tt2 on (tt1.f1 = 'foo') +-- left join text_tbl as tt3 on (tt3.f1 = 'foo') +-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1), +-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +-- where tt1.f1 = ss1.c0; + +-- +-- check a case in which a PlaceHolderVar forces join order +-- + +-- explain (verbose, costs off) +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- +-- test successful handling of full join underneath left join (bug #14105) +-- + +-- explain (costs off) +-- select * from +-- (select 1 as id) as xx +-- left join +-- (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) +-- on (xx.id = coalesce(yy.id)); + +select * from + (select udf(udf(1)) as id) as xx + left join + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))); + +-- +-- test ability to push constants through outer join clauses +-- + +-- explain (costs off) +-- select * from int4_tbl a left join tenk1 b on f1 = unique2 where f1 = 0; + +-- explain (costs off) +-- select * from tenk1 a full join tenk1 b using(unique2) where unique2 = 42; + +-- +-- test that quals attached to an outer join have correct semantics, +-- specifically that they don't re-use expressions computed below the join; +-- we force a mergejoin so that coalesce(b.q1, 1) appears as a join input +-- + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- explain (verbose, costs off) +-- select a.q2, b.q1 +-- from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) +-- where coalesce(b.q1, 1) > 0; +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0); + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- Skip these test because it only test explain +-- +-- test join removal +-- + +-- begin; + +-- CREATE TEMP TABLE a (id int PRIMARY KEY, b_id int); +-- CREATE TEMP TABLE b (id int PRIMARY KEY, c_id int); +-- CREATE TEMP TABLE c (id int PRIMARY KEY); +-- CREATE TEMP TABLE d (a int, b int); +-- INSERT INTO a VALUES (0, 0), (1, NULL); +-- INSERT INTO b VALUES (0, 0), (1, NULL); +-- INSERT INTO c VALUES (0), (1); +-- INSERT INTO d VALUES (1,3), (2,2), (3,1); + +-- all three cases should be optimizable into a simple seqscan +-- explain (costs off) SELECT a.* FROM a LEFT JOIN b ON a.b_id = b.id; +-- explain (costs off) SELECT b.* FROM b LEFT JOIN c ON b.c_id = c.id; +-- explain (costs off) +-- SELECT a.* FROM a LEFT JOIN (b left join c on b.c_id = c.id) +-- ON (a.b_id = b.id); + +-- check optimization of outer join within another special join +-- explain (costs off) +-- select id from a where id in ( +-- select b.id from b left join c on b.id = c.id +-- ); + +-- check that join removal works for a left join when joining a subquery +-- that is guaranteed to be unique by its GROUP BY clause +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id and d.b = s.c_id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id and d.b = s.c_id; + +-- join removal is not possible when the GROUP BY contains a column that is +-- not in the join condition. (Note: as of 9.6, we notice that b.id is a +-- primary key and so drop b.c_id from the GROUP BY of the resulting plan; +-- but this happens too late for join removal in the outer plan level.) +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id; + +-- check join removal works when uniqueness of the join condition is enforced +-- by a UNION +-- explain (costs off) +-- select d.* from d left join (select id from a union select id from b) s +-- on d.a = s.id; + +-- check join removal with a cross-type comparison operator +-- explain (costs off) +-- select i8.* from int8_tbl i8 left join (select f1 from int4_tbl group by f1) i4 +-- on i8.q1 = i4.f1; + +-- check join removal with lateral references +-- explain (costs off) +-- select 1 from (select a.id FROM a left join b on a.b_id = b.id) q, +-- lateral generate_series(1, q.id) gs(i) where q.id = gs.i; + +-- rollback; + +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd); +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd); + +-- this case is optimizable +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)); +-- explain (costs off) +-- select p.* from parent p left join child c on (p.k = c.k); + +-- this case is not +select p.*, linked from parent p + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))); +-- explain (costs off) +-- select p.*, linked from parent p +-- left join (select c.*, true as linked from child c) as ss +-- on (p.k = ss.k); + +-- check for a 9.0rc1 bug: join removal breaks pseudoconstant qual handling +select p.* from + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)); +-- explain (costs off) +-- select p.* from +-- parent p left join child c on (p.k = c.k) +-- where p.k = 1 and p.k = 2; + +select p.* from + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)); +-- explain (costs off) +-- select p.* from +-- (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k +-- where p.k = 1 and p.k = 2; + +-- bug 5255: this is not optimizable by join removal +-- begin; + +create or replace temporary view a as select * from + (values (0), (1)) + as v(id); +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id); + +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0); +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0); + +-- rollback; + +-- another join removal bug: this is not optimizable, either +-- begin; + +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1); + +SELECT * FROM + (SELECT udf(1) AS x) ss1 + LEFT JOIN + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 + ON true; + +-- rollback; + +-- another join removal bug: we must clean up correctly when removing a PHV +-- begin; + +-- create temp table uniquetbl (f1 text unique); + +-- explain (costs off) +-- select t1.* from +-- uniquetbl as t1 +-- left join (select *, '***'::text as d1 from uniquetbl) t2 +-- on t1.f1 = t2.f1 +-- left join uniquetbl t3 +-- on t2.d1 = t3.f1; + +-- explain (costs off) +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- [SPARK-27987] Support POSIX Regular Expressions +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- rollback; + +-- bug #8444: we've historically allowed duplicate aliases within aliased JOINs + +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1); -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1); -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)); -- ok + +-- +-- Test hints given on incorrect column references are useful +-- + +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two); -- error, prefer "t1" suggestion +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two; -- error, prefer "t2" suggestion +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two); -- error, suggest both at once + +-- Skip this test because it is a PostgreSQL specific case +-- +-- Take care to reference the correct RTE +-- + +-- -- select atts.relid::regclass, s.* from pg_stats s join +-- -- pg_attribute a on s.attname = a.attname and s.tablename = +-- -- a.attrelid::regclass::text join (select unnest(indkey) attnum, +-- -- indexrelid from pg_index i) atts on atts.attnum = a.attnum where +-- schemaname != 'pg_catalog'; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- Test LATERAL +-- + +-- select unique2, x.* +-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +-- explain (costs off) +-- select unique2, x.* +-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +-- select unique2, x.* +-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x cross join lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- select unique2, x.* +-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- check scoping of lateral versus parent references +-- the first of these should return int8_tbl.q2, the second int8_tbl.q1 +-- select *, (select r from (select q1 as q2) x, (select q2 as r) y) from int8_tbl; +-- select *, (select r from (select q1 as q2) x, lateral (select q2 as r) y) from int8_tbl; + +-- lateral with function in FROM +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a cross join lateral generate_series(1,two) g; +-- don't need the explicit LATERAL keyword for functions +-- explain (costs off) +-- select count(*) from tenk1 a, generate_series(1,two) g; + +-- lateral with UNION ALL subselect +-- explain (costs off) +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; + +-- lateral with VALUES +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; + +-- lateral with VALUES, no flattening possible +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; + +-- lateral injecting a strange outer join condition +-- explain (costs off) +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; + +-- lateral reference to a join alias variable +-- select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, +-- lateral (select x) ss2(y); +-- select * from (select f1 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, +-- lateral (values(x)) ss2(y); +-- select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, +-- lateral (select x) ss2(y); + +-- lateral references requiring pullup +-- select * from (values(1)) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (select f1/1000000000 from int4_tbl) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (values(1)) x(lb), +-- lateral (values(lb)) y(lbcopy); +-- select * from (values(1)) x(lb), +-- lateral (select lb from int4_tbl) y(lbcopy); +-- select * from +-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, +-- lateral (values(x.q1,y.q1,y.q2)) v(xq1,yq1,yq2); +-- select * from +-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, +-- lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +-- select x.* from +-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, +-- lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +-- select v.* from +-- (int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 from onerow union all select x.q2,y.q2 from onerow) v(vx,vy); + +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +-- select * from +-- int8_tbl a left join +-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; +-- select * from +-- int8_tbl a left join +-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; + +-- lateral can result in join conditions appearing below their +-- real semantic level +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +-- select * from int4_tbl i left join +-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; + +-- lateral reference in a PlaceHolderVar evaluated at join level +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join lateral +-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from +-- int8_tbl b cross join int8_tbl c) ss +-- on a.q2 = ss.bq1; +-- select * from +-- int8_tbl a left join lateral +-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from +-- int8_tbl b cross join int8_tbl c) ss +-- on a.q2 = ss.bq1; + +-- case requiring nested PlaceHolderVars +-- explain (verbose, costs off) +-- select * from +-- int8_tbl c left join ( +-- int8_tbl a left join (select q1, coalesce(q2,42) as x from int8_tbl b) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select ss2.y offset 0) ss3; + +-- case that breaks the old ph_may_need optimization +-- explain (verbose, costs off) +-- select c.*,a.*,ss1.q1,ss2.q1,ss3.* from +-- int8_tbl c left join ( +-- int8_tbl a left join +-- (select q1, coalesce(q2,f1) as x from int8_tbl b, int4_tbl b2 +-- where q1 < f1) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select * from int4_tbl i where ss2.y > f1) ss3; + +-- check processing of postponed quals (bug #9041) +-- explain (verbose, costs off) +-- select * from +-- (select 1 as x offset 0) x cross join (select 2 as y offset 0) y +-- left join lateral ( +-- select * from (select 3 as z offset 0) z where z.z = x.x +-- ) zz on zz.z = y.y; + +-- check dummy rels with lateral references (bug #15694) +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl where false) ss on true; +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl i1, int4_tbl i2 where false) ss on true; + +-- check handling of nested appendrels inside LATERAL +-- select * from +-- ((select 2 as v) union all (select 3 as v)) as q1 +-- cross join lateral +-- ((select * from +-- ((select 4 as v) union all (select 5 as v)) as q3) +-- union all +-- (select q1.v) +-- ) as q2; + +-- check we don't try to do a unique-ified semijoin with LATERAL +-- explain (verbose, costs off) +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; + +-- check proper extParam/allParam handling (this isn't exactly a LATERAL issue, +-- but we can make the test case much more compact with LATERAL) +-- explain (verbose, costs off) +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- test some error cases where LATERAL should have been used but wasn't +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss; +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss; +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss; +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss; +-- SQL:2008 says the left table is in scope but illegal to access here +-- select f1,g from int4_tbl a right join lateral generate_series(0, a.f1) g on true; +-- select f1,g from int4_tbl a full join lateral generate_series(0, a.f1) g on true; +-- check we complain about ambiguous table references +-- select * from +-- int8_tbl x cross join (int4_tbl x cross join lateral (select x.f1) ss); +-- LATERAL can be used to put an aggregate into the FROM clause of its query +-- select 1 from tenk1 a, lateral (select max(a.unique1) from int4_tbl b) ss; + +-- check behavior of LATERAL in UPDATE/DELETE + +-- create temp table xx1 as select f1 as x1, -f1 as x2 from int4_tbl; + +-- error, can't do this: +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = x1) ss; +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = xx1.x1) ss; +-- can't do it even with LATERAL: +-- update xx1 set x2 = f1 from lateral (select * from int4_tbl where f1 = x1) ss; +-- we might in future allow something like this, but for now it's an error: +-- update xx1 set x2 = f1 from xx1, lateral (select * from int4_tbl where f1 = x1) ss; + +-- also errors: +-- delete from xx1 using (select * from int4_tbl where f1 = x1) ss; +-- delete from xx1 using (select * from int4_tbl where f1 = xx1.x1) ss; +-- delete from xx1 using lateral (select * from int4_tbl where f1 = x1) ss; + +-- [SPARK-25411] Implement range partition in Spark +-- +-- test LATERAL reference propagation down a multi-level inheritance hierarchy +-- produced for a multi-level partitioned table hierarchy. +-- +-- create table join_pt1 (a int, b int, c varchar) partition by range(a); +-- create table join_pt1p1 partition of join_pt1 for values from (0) to (100) partition by range(b); +-- create table join_pt1p2 partition of join_pt1 for values from (100) to (200); +-- create table join_pt1p1p1 partition of join_pt1p1 for values from (0) to (100); +-- insert into join_pt1 values (1, 1, 'x'), (101, 101, 'y'); +-- create table join_ut1 (a int, b int, c varchar); +-- insert into join_ut1 values (101, 101, 'y'), (2, 2, 'z'); +-- explain (verbose, costs off) +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- +-- drop table join_pt1; +-- drop table join_ut1; +-- +-- test that foreign key join estimation performs sanely for outer joins +-- + +-- begin; + +-- create table fkest (a int, b int, c int unique, primary key(a,b)); +-- create table fkest1 (a int, b int, primary key(a,b)); + +-- insert into fkest select x/10, x%10, x from generate_series(1,1000) x; +-- insert into fkest1 select x/10, x%10 from generate_series(1,1000) x; + +-- alter table fkest1 +-- add constraint fkest1_a_b_fkey foreign key (a,b) references fkest; + +-- analyze fkest; +-- analyze fkest1; + +-- explain (costs off) +-- select * +-- from fkest f +-- left join fkest1 f1 on f.a = f1.a and f.b = f1.b +-- left join fkest1 f2 on f.a = f2.a and f.b = f2.b +-- left join fkest1 f3 on f.a = f3.a and f.b = f3.b +-- where f.c = 1; + +-- rollback; + +-- Skip these test because it only test explain +-- +-- test planner's ability to mark joins as unique +-- + +-- create table j1 (id int primary key); +-- create table j2 (id int primary key); +-- create table j3 (id int); + +-- insert into j1 values(1),(2),(3); +-- insert into j2 values(1),(2),(3); +-- insert into j3 values(1),(1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure join is properly marked as unique +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id = j2.id; + +-- ensure join is not unique when not an equi-join +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id > j2.id; + +-- ensure non-unique rel is not chosen as inner +-- explain (verbose, costs off) +-- select * from j1 inner join j3 on j1.id = j3.id; + +-- ensure left join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 left join j2 on j1.id = j2.id; + +-- ensure right join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 right join j2 on j1.id = j2.id; + +-- ensure full join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 full join j2 on j1.id = j2.id; + +-- a clauseless (cross) join can't be unique +-- explain (verbose, costs off) +-- select * from j1 cross join j2; + +-- ensure a natural join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 natural join j2; + +-- ensure a distinct clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select distinct id from j3) j3 on j1.id = j3.id; + +-- ensure group by clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select id from j3 group by id) j3 on j1.id = j3.id; + +-- drop table if exists j1; +-- drop table if exists j2; +-- drop table if exists j3; + +-- test more complex permutations of unique joins + +CREATE TABLE j1 (id1 int, id2 int) USING parquet; +CREATE TABLE j2 (id1 int, id2 int) USING parquet; +-- create table j3 (id1 int, id2 int) using parquet; + +INSERT INTO j1 values(1,1),(1,2); +INSERT INTO j2 values(1,1); +-- insert into j3 values(1,1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure there's no unique join when not all columns which are part of the +-- unique index are seen in the join clause +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1; + +-- ensure proper unique detection with multiple join quals +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2; + +-- ensure we don't detect the join to be unique when quals are not part of the +-- join condition +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- as above, but for left joins. +-- explain (verbose, costs off) +-- select * from j1 +-- left join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- validate logic in merge joins which skips mark and restore. +-- it should only do this if all quals which were used to detect the unique +-- are present as join quals, and not plain quals. +-- set enable_nestloop to 0; +-- set enable_hashjoin to 0; +-- set enable_sort to 0; + +-- create indexes that will be preferred over the PKs to perform the join +-- create index j1_id1_idx on j1 (id1) where id1 % 1000 = 1; +-- create index j2_id1_idx on j2 (id1) where id1 % 1000 = 1; + +-- need an additional row in j2, if we want j2_id1_idx to be preferred +INSERT INTO j2 values(1,2); +-- analyze j2; + +-- explain (costs off) select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +-- where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1; + +select * from j1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1; + +-- reset enable_nestloop; +-- reset enable_hashjoin; +-- reset enable_sort; + +drop table j1; +drop table j2; +-- drop table j3; + +-- Skip these tests because it only test explain +-- check that semijoin inner is not seen as unique for a portion of the outerrel +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from tenk1 t3 +-- where t3.thousand = t1.unique1 and t3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- ... unless it actually is unique +-- create table j3 as select unique1, tenthous from onek; +-- vacuum analyze j3; +-- create unique index on j3(unique1, tenthous); + +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from j3 +-- where j3.unique1 = t1.unique1 and j3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- drop table j3; diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out new file mode 100644 index 0000000000000..6fcff129d7568 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out @@ -0,0 +1,3408 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 185 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO J1_TBL VALUES (1, 4, 'one') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO J1_TBL VALUES (2, 3, 'two') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO J1_TBL VALUES (3, 2, 'three') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO J1_TBL VALUES (4, 1, 'four') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO J1_TBL VALUES (5, 0, 'five') +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO J1_TBL VALUES (6, 6, 'six') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO J1_TBL VALUES (7, 7, 'seven') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO J1_TBL VALUES (8, 8, 'eight') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO J1_TBL VALUES (0, NULL, 'zero') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +INSERT INTO J2_TBL VALUES (1, -1) +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO J2_TBL VALUES (2, 2) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +INSERT INTO J2_TBL VALUES (3, -3) +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +INSERT INTO J2_TBL VALUES (2, 4) +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +INSERT INTO J2_TBL VALUES (5, -5) +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +INSERT INTO J2_TBL VALUES (5, -5) +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +INSERT INTO J2_TBL VALUES (0, NULL) +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +INSERT INTO J2_TBL VALUES (NULL, NULL) +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +INSERT INTO J2_TBL VALUES (NULL, 0) +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) + FROM J1_TBL AS tx +-- !query 27 schema +struct +-- !query 27 output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query 28 +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) + FROM J1_TBL tx +-- !query 28 schema +struct +-- !query 28 output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query 29 +SELECT udf('') AS `xxx`, a, udf(udf(b)), c + FROM J1_TBL AS t1 (a, b, c) +-- !query 29 schema +struct +-- !query 29 output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query 30 +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) + FROM J1_TBL t1 (a, b, c) +-- !query 30 schema +struct +-- !query 30 output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query 31 +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- !query 31 schema +struct +-- !query 31 output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query 32 +SELECT udf('') AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL +-- !query 32 schema +struct +-- !query 32 output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query 33 +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t + FROM J1_TBL CROSS JOIN J2_TBL +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.AnalysisException +Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 29 + + +-- !query 34 +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) + FROM J1_TBL t1 CROSS JOIN J2_TBL t2 +-- !query 34 schema +struct +-- !query 34 output + 0 -1 zero + 0 -3 zero + 0 -5 zero + 0 -5 zero + 0 0 zero + 0 2 zero + 0 4 zero + 0 NULL zero + 0 NULL zero + 1 -1 one + 1 -3 one + 1 -5 one + 1 -5 one + 1 0 one + 1 2 one + 1 4 one + 1 NULL one + 1 NULL one + 2 -1 two + 2 -3 two + 2 -5 two + 2 -5 two + 2 0 two + 2 2 two + 2 4 two + 2 NULL two + 2 NULL two + 3 -1 three + 3 -3 three + 3 -5 three + 3 -5 three + 3 0 three + 3 2 three + 3 4 three + 3 NULL three + 3 NULL three + 4 -1 four + 4 -3 four + 4 -5 four + 4 -5 four + 4 0 four + 4 2 four + 4 4 four + 4 NULL four + 4 NULL four + 5 -1 five + 5 -3 five + 5 -5 five + 5 -5 five + 5 0 five + 5 2 five + 5 4 five + 5 NULL five + 5 NULL five + 6 -1 six + 6 -3 six + 6 -5 six + 6 -5 six + 6 0 six + 6 2 six + 6 4 six + 6 NULL six + 6 NULL six + 7 -1 seven + 7 -3 seven + 7 -5 seven + 7 -5 seven + 7 0 seven + 7 2 seven + 7 4 seven + 7 NULL seven + 7 NULL seven + 8 -1 eight + 8 -3 eight + 8 -5 eight + 8 -5 eight + 8 0 eight + 8 2 eight + 8 4 eight + 8 NULL eight + 8 NULL eight + NULL -1 null + NULL -1 zero + NULL -3 null + NULL -3 zero + NULL -5 null + NULL -5 null + NULL -5 zero + NULL -5 zero + NULL 0 null + NULL 0 zero + NULL 2 null + NULL 2 zero + NULL 4 null + NULL 4 zero + NULL NULL null + NULL NULL null + NULL NULL zero + NULL NULL zero + + +-- !query 35 +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk) +-- !query 35 schema +struct +-- !query 35 output + 0 zero -1 + 0 zero -3 + 0 zero -5 + 0 zero -5 + 0 zero 0 + 0 zero 2 + 0 zero 4 + 0 zero NULL + 0 zero NULL + 1 one -1 + 1 one -3 + 1 one -5 + 1 one -5 + 1 one 0 + 1 one 2 + 1 one 4 + 1 one NULL + 1 one NULL + 2 two -1 + 2 two -3 + 2 two -5 + 2 two -5 + 2 two 0 + 2 two 2 + 2 two 4 + 2 two NULL + 2 two NULL + 3 three -1 + 3 three -3 + 3 three -5 + 3 three -5 + 3 three 0 + 3 three 2 + 3 three 4 + 3 three NULL + 3 three NULL + 4 four -1 + 4 four -3 + 4 four -5 + 4 four -5 + 4 four 0 + 4 four 2 + 4 four 4 + 4 four NULL + 4 four NULL + 5 five -1 + 5 five -3 + 5 five -5 + 5 five -5 + 5 five 0 + 5 five 2 + 5 five 4 + 5 five NULL + 5 five NULL + 6 six -1 + 6 six -3 + 6 six -5 + 6 six -5 + 6 six 0 + 6 six 2 + 6 six 4 + 6 six NULL + 6 six NULL + 7 seven -1 + 7 seven -3 + 7 seven -5 + 7 seven -5 + 7 seven 0 + 7 seven 2 + 7 seven 4 + 7 seven NULL + 7 seven NULL + 8 eight -1 + 8 eight -3 + 8 eight -5 + 8 eight -5 + 8 eight 0 + 8 eight 2 + 8 eight 4 + 8 eight NULL + 8 eight NULL + NULL null -1 + NULL null -3 + NULL null -5 + NULL null -5 + NULL null 0 + NULL null 2 + NULL null 4 + NULL null NULL + NULL null NULL + NULL zero -1 + NULL zero -3 + NULL zero -5 + NULL zero -5 + NULL zero 0 + NULL zero 2 + NULL zero 4 + NULL zero NULL + NULL zero NULL + + +-- !query 36 +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b +-- !query 36 schema +struct +-- !query 36 output + 0 NULL zero 0 NULL 0 NULL + 0 NULL zero 0 NULL 1 -1 + 0 NULL zero 0 NULL 2 2 + 0 NULL zero 0 NULL 2 4 + 0 NULL zero 0 NULL 3 -3 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL NULL 0 + 0 NULL zero 0 NULL NULL NULL + 0 NULL zero 1 -1 0 NULL + 0 NULL zero 1 -1 1 -1 + 0 NULL zero 1 -1 2 2 + 0 NULL zero 1 -1 2 4 + 0 NULL zero 1 -1 3 -3 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 NULL 0 + 0 NULL zero 1 -1 NULL NULL + 0 NULL zero 2 2 0 NULL + 0 NULL zero 2 2 1 -1 + 0 NULL zero 2 2 2 2 + 0 NULL zero 2 2 2 4 + 0 NULL zero 2 2 3 -3 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 NULL 0 + 0 NULL zero 2 2 NULL NULL + 0 NULL zero 2 4 0 NULL + 0 NULL zero 2 4 1 -1 + 0 NULL zero 2 4 2 2 + 0 NULL zero 2 4 2 4 + 0 NULL zero 2 4 3 -3 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 NULL 0 + 0 NULL zero 2 4 NULL NULL + 0 NULL zero 3 -3 0 NULL + 0 NULL zero 3 -3 1 -1 + 0 NULL zero 3 -3 2 2 + 0 NULL zero 3 -3 2 4 + 0 NULL zero 3 -3 3 -3 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 NULL 0 + 0 NULL zero 3 -3 NULL NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero NULL 0 0 NULL + 0 NULL zero NULL 0 1 -1 + 0 NULL zero NULL 0 2 2 + 0 NULL zero NULL 0 2 4 + 0 NULL zero NULL 0 3 -3 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 NULL 0 + 0 NULL zero NULL 0 NULL NULL + 0 NULL zero NULL NULL 0 NULL + 0 NULL zero NULL NULL 1 -1 + 0 NULL zero NULL NULL 2 2 + 0 NULL zero NULL NULL 2 4 + 0 NULL zero NULL NULL 3 -3 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL NULL 0 + 0 NULL zero NULL NULL NULL NULL + 1 4 one 0 NULL 0 NULL + 1 4 one 0 NULL 1 -1 + 1 4 one 0 NULL 2 2 + 1 4 one 0 NULL 2 4 + 1 4 one 0 NULL 3 -3 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL NULL 0 + 1 4 one 0 NULL NULL NULL + 1 4 one 1 -1 0 NULL + 1 4 one 1 -1 1 -1 + 1 4 one 1 -1 2 2 + 1 4 one 1 -1 2 4 + 1 4 one 1 -1 3 -3 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 NULL 0 + 1 4 one 1 -1 NULL NULL + 1 4 one 2 2 0 NULL + 1 4 one 2 2 1 -1 + 1 4 one 2 2 2 2 + 1 4 one 2 2 2 4 + 1 4 one 2 2 3 -3 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 NULL 0 + 1 4 one 2 2 NULL NULL + 1 4 one 2 4 0 NULL + 1 4 one 2 4 1 -1 + 1 4 one 2 4 2 2 + 1 4 one 2 4 2 4 + 1 4 one 2 4 3 -3 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 NULL 0 + 1 4 one 2 4 NULL NULL + 1 4 one 3 -3 0 NULL + 1 4 one 3 -3 1 -1 + 1 4 one 3 -3 2 2 + 1 4 one 3 -3 2 4 + 1 4 one 3 -3 3 -3 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 NULL 0 + 1 4 one 3 -3 NULL NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL NULL + 1 4 one 5 -5 NULL NULL + 1 4 one NULL 0 0 NULL + 1 4 one NULL 0 1 -1 + 1 4 one NULL 0 2 2 + 1 4 one NULL 0 2 4 + 1 4 one NULL 0 3 -3 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 NULL 0 + 1 4 one NULL 0 NULL NULL + 1 4 one NULL NULL 0 NULL + 1 4 one NULL NULL 1 -1 + 1 4 one NULL NULL 2 2 + 1 4 one NULL NULL 2 4 + 1 4 one NULL NULL 3 -3 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL NULL 0 + 1 4 one NULL NULL NULL NULL + 2 3 two 0 NULL 0 NULL + 2 3 two 0 NULL 1 -1 + 2 3 two 0 NULL 2 2 + 2 3 two 0 NULL 2 4 + 2 3 two 0 NULL 3 -3 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL NULL 0 + 2 3 two 0 NULL NULL NULL + 2 3 two 1 -1 0 NULL + 2 3 two 1 -1 1 -1 + 2 3 two 1 -1 2 2 + 2 3 two 1 -1 2 4 + 2 3 two 1 -1 3 -3 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 NULL 0 + 2 3 two 1 -1 NULL NULL + 2 3 two 2 2 0 NULL + 2 3 two 2 2 1 -1 + 2 3 two 2 2 2 2 + 2 3 two 2 2 2 4 + 2 3 two 2 2 3 -3 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 NULL 0 + 2 3 two 2 2 NULL NULL + 2 3 two 2 4 0 NULL + 2 3 two 2 4 1 -1 + 2 3 two 2 4 2 2 + 2 3 two 2 4 2 4 + 2 3 two 2 4 3 -3 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 NULL 0 + 2 3 two 2 4 NULL NULL + 2 3 two 3 -3 0 NULL + 2 3 two 3 -3 1 -1 + 2 3 two 3 -3 2 2 + 2 3 two 3 -3 2 4 + 2 3 two 3 -3 3 -3 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 NULL 0 + 2 3 two 3 -3 NULL NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL NULL + 2 3 two 5 -5 NULL NULL + 2 3 two NULL 0 0 NULL + 2 3 two NULL 0 1 -1 + 2 3 two NULL 0 2 2 + 2 3 two NULL 0 2 4 + 2 3 two NULL 0 3 -3 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 NULL 0 + 2 3 two NULL 0 NULL NULL + 2 3 two NULL NULL 0 NULL + 2 3 two NULL NULL 1 -1 + 2 3 two NULL NULL 2 2 + 2 3 two NULL NULL 2 4 + 2 3 two NULL NULL 3 -3 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL NULL 0 + 2 3 two NULL NULL NULL NULL + 3 2 three 0 NULL 0 NULL + 3 2 three 0 NULL 1 -1 + 3 2 three 0 NULL 2 2 + 3 2 three 0 NULL 2 4 + 3 2 three 0 NULL 3 -3 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL NULL 0 + 3 2 three 0 NULL NULL NULL + 3 2 three 1 -1 0 NULL + 3 2 three 1 -1 1 -1 + 3 2 three 1 -1 2 2 + 3 2 three 1 -1 2 4 + 3 2 three 1 -1 3 -3 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 NULL 0 + 3 2 three 1 -1 NULL NULL + 3 2 three 2 2 0 NULL + 3 2 three 2 2 1 -1 + 3 2 three 2 2 2 2 + 3 2 three 2 2 2 4 + 3 2 three 2 2 3 -3 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 NULL 0 + 3 2 three 2 2 NULL NULL + 3 2 three 2 4 0 NULL + 3 2 three 2 4 1 -1 + 3 2 three 2 4 2 2 + 3 2 three 2 4 2 4 + 3 2 three 2 4 3 -3 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 NULL 0 + 3 2 three 2 4 NULL NULL + 3 2 three 3 -3 0 NULL + 3 2 three 3 -3 1 -1 + 3 2 three 3 -3 2 2 + 3 2 three 3 -3 2 4 + 3 2 three 3 -3 3 -3 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 NULL 0 + 3 2 three 3 -3 NULL NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL NULL + 3 2 three 5 -5 NULL NULL + 3 2 three NULL 0 0 NULL + 3 2 three NULL 0 1 -1 + 3 2 three NULL 0 2 2 + 3 2 three NULL 0 2 4 + 3 2 three NULL 0 3 -3 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 NULL 0 + 3 2 three NULL 0 NULL NULL + 3 2 three NULL NULL 0 NULL + 3 2 three NULL NULL 1 -1 + 3 2 three NULL NULL 2 2 + 3 2 three NULL NULL 2 4 + 3 2 three NULL NULL 3 -3 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL NULL 0 + 3 2 three NULL NULL NULL NULL + 4 1 four 0 NULL 0 NULL + 4 1 four 0 NULL 1 -1 + 4 1 four 0 NULL 2 2 + 4 1 four 0 NULL 2 4 + 4 1 four 0 NULL 3 -3 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL NULL 0 + 4 1 four 0 NULL NULL NULL + 4 1 four 1 -1 0 NULL + 4 1 four 1 -1 1 -1 + 4 1 four 1 -1 2 2 + 4 1 four 1 -1 2 4 + 4 1 four 1 -1 3 -3 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 NULL 0 + 4 1 four 1 -1 NULL NULL + 4 1 four 2 2 0 NULL + 4 1 four 2 2 1 -1 + 4 1 four 2 2 2 2 + 4 1 four 2 2 2 4 + 4 1 four 2 2 3 -3 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 NULL 0 + 4 1 four 2 2 NULL NULL + 4 1 four 2 4 0 NULL + 4 1 four 2 4 1 -1 + 4 1 four 2 4 2 2 + 4 1 four 2 4 2 4 + 4 1 four 2 4 3 -3 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 NULL 0 + 4 1 four 2 4 NULL NULL + 4 1 four 3 -3 0 NULL + 4 1 four 3 -3 1 -1 + 4 1 four 3 -3 2 2 + 4 1 four 3 -3 2 4 + 4 1 four 3 -3 3 -3 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 NULL 0 + 4 1 four 3 -3 NULL NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL NULL + 4 1 four 5 -5 NULL NULL + 4 1 four NULL 0 0 NULL + 4 1 four NULL 0 1 -1 + 4 1 four NULL 0 2 2 + 4 1 four NULL 0 2 4 + 4 1 four NULL 0 3 -3 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 NULL 0 + 4 1 four NULL 0 NULL NULL + 4 1 four NULL NULL 0 NULL + 4 1 four NULL NULL 1 -1 + 4 1 four NULL NULL 2 2 + 4 1 four NULL NULL 2 4 + 4 1 four NULL NULL 3 -3 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL NULL 0 + 4 1 four NULL NULL NULL NULL + 5 0 five 0 NULL 0 NULL + 5 0 five 0 NULL 1 -1 + 5 0 five 0 NULL 2 2 + 5 0 five 0 NULL 2 4 + 5 0 five 0 NULL 3 -3 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL NULL 0 + 5 0 five 0 NULL NULL NULL + 5 0 five 1 -1 0 NULL + 5 0 five 1 -1 1 -1 + 5 0 five 1 -1 2 2 + 5 0 five 1 -1 2 4 + 5 0 five 1 -1 3 -3 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 NULL 0 + 5 0 five 1 -1 NULL NULL + 5 0 five 2 2 0 NULL + 5 0 five 2 2 1 -1 + 5 0 five 2 2 2 2 + 5 0 five 2 2 2 4 + 5 0 five 2 2 3 -3 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 NULL 0 + 5 0 five 2 2 NULL NULL + 5 0 five 2 4 0 NULL + 5 0 five 2 4 1 -1 + 5 0 five 2 4 2 2 + 5 0 five 2 4 2 4 + 5 0 five 2 4 3 -3 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 NULL 0 + 5 0 five 2 4 NULL NULL + 5 0 five 3 -3 0 NULL + 5 0 five 3 -3 1 -1 + 5 0 five 3 -3 2 2 + 5 0 five 3 -3 2 4 + 5 0 five 3 -3 3 -3 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 NULL 0 + 5 0 five 3 -3 NULL NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL NULL + 5 0 five 5 -5 NULL NULL + 5 0 five NULL 0 0 NULL + 5 0 five NULL 0 1 -1 + 5 0 five NULL 0 2 2 + 5 0 five NULL 0 2 4 + 5 0 five NULL 0 3 -3 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 NULL 0 + 5 0 five NULL 0 NULL NULL + 5 0 five NULL NULL 0 NULL + 5 0 five NULL NULL 1 -1 + 5 0 five NULL NULL 2 2 + 5 0 five NULL NULL 2 4 + 5 0 five NULL NULL 3 -3 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL NULL 0 + 5 0 five NULL NULL NULL NULL + 6 6 six 0 NULL 0 NULL + 6 6 six 0 NULL 1 -1 + 6 6 six 0 NULL 2 2 + 6 6 six 0 NULL 2 4 + 6 6 six 0 NULL 3 -3 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL NULL 0 + 6 6 six 0 NULL NULL NULL + 6 6 six 1 -1 0 NULL + 6 6 six 1 -1 1 -1 + 6 6 six 1 -1 2 2 + 6 6 six 1 -1 2 4 + 6 6 six 1 -1 3 -3 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 NULL 0 + 6 6 six 1 -1 NULL NULL + 6 6 six 2 2 0 NULL + 6 6 six 2 2 1 -1 + 6 6 six 2 2 2 2 + 6 6 six 2 2 2 4 + 6 6 six 2 2 3 -3 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 NULL 0 + 6 6 six 2 2 NULL NULL + 6 6 six 2 4 0 NULL + 6 6 six 2 4 1 -1 + 6 6 six 2 4 2 2 + 6 6 six 2 4 2 4 + 6 6 six 2 4 3 -3 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 NULL 0 + 6 6 six 2 4 NULL NULL + 6 6 six 3 -3 0 NULL + 6 6 six 3 -3 1 -1 + 6 6 six 3 -3 2 2 + 6 6 six 3 -3 2 4 + 6 6 six 3 -3 3 -3 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 NULL 0 + 6 6 six 3 -3 NULL NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL NULL + 6 6 six 5 -5 NULL NULL + 6 6 six NULL 0 0 NULL + 6 6 six NULL 0 1 -1 + 6 6 six NULL 0 2 2 + 6 6 six NULL 0 2 4 + 6 6 six NULL 0 3 -3 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 NULL 0 + 6 6 six NULL 0 NULL NULL + 6 6 six NULL NULL 0 NULL + 6 6 six NULL NULL 1 -1 + 6 6 six NULL NULL 2 2 + 6 6 six NULL NULL 2 4 + 6 6 six NULL NULL 3 -3 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL NULL 0 + 6 6 six NULL NULL NULL NULL + 7 7 seven 0 NULL 0 NULL + 7 7 seven 0 NULL 1 -1 + 7 7 seven 0 NULL 2 2 + 7 7 seven 0 NULL 2 4 + 7 7 seven 0 NULL 3 -3 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL NULL 0 + 7 7 seven 0 NULL NULL NULL + 7 7 seven 1 -1 0 NULL + 7 7 seven 1 -1 1 -1 + 7 7 seven 1 -1 2 2 + 7 7 seven 1 -1 2 4 + 7 7 seven 1 -1 3 -3 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 NULL 0 + 7 7 seven 1 -1 NULL NULL + 7 7 seven 2 2 0 NULL + 7 7 seven 2 2 1 -1 + 7 7 seven 2 2 2 2 + 7 7 seven 2 2 2 4 + 7 7 seven 2 2 3 -3 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 NULL 0 + 7 7 seven 2 2 NULL NULL + 7 7 seven 2 4 0 NULL + 7 7 seven 2 4 1 -1 + 7 7 seven 2 4 2 2 + 7 7 seven 2 4 2 4 + 7 7 seven 2 4 3 -3 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 NULL 0 + 7 7 seven 2 4 NULL NULL + 7 7 seven 3 -3 0 NULL + 7 7 seven 3 -3 1 -1 + 7 7 seven 3 -3 2 2 + 7 7 seven 3 -3 2 4 + 7 7 seven 3 -3 3 -3 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 NULL 0 + 7 7 seven 3 -3 NULL NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL NULL + 7 7 seven 5 -5 NULL NULL + 7 7 seven NULL 0 0 NULL + 7 7 seven NULL 0 1 -1 + 7 7 seven NULL 0 2 2 + 7 7 seven NULL 0 2 4 + 7 7 seven NULL 0 3 -3 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 NULL 0 + 7 7 seven NULL 0 NULL NULL + 7 7 seven NULL NULL 0 NULL + 7 7 seven NULL NULL 1 -1 + 7 7 seven NULL NULL 2 2 + 7 7 seven NULL NULL 2 4 + 7 7 seven NULL NULL 3 -3 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL NULL 0 + 7 7 seven NULL NULL NULL NULL + 8 8 eight 0 NULL 0 NULL + 8 8 eight 0 NULL 1 -1 + 8 8 eight 0 NULL 2 2 + 8 8 eight 0 NULL 2 4 + 8 8 eight 0 NULL 3 -3 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL NULL 0 + 8 8 eight 0 NULL NULL NULL + 8 8 eight 1 -1 0 NULL + 8 8 eight 1 -1 1 -1 + 8 8 eight 1 -1 2 2 + 8 8 eight 1 -1 2 4 + 8 8 eight 1 -1 3 -3 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 NULL 0 + 8 8 eight 1 -1 NULL NULL + 8 8 eight 2 2 0 NULL + 8 8 eight 2 2 1 -1 + 8 8 eight 2 2 2 2 + 8 8 eight 2 2 2 4 + 8 8 eight 2 2 3 -3 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 NULL 0 + 8 8 eight 2 2 NULL NULL + 8 8 eight 2 4 0 NULL + 8 8 eight 2 4 1 -1 + 8 8 eight 2 4 2 2 + 8 8 eight 2 4 2 4 + 8 8 eight 2 4 3 -3 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 NULL 0 + 8 8 eight 2 4 NULL NULL + 8 8 eight 3 -3 0 NULL + 8 8 eight 3 -3 1 -1 + 8 8 eight 3 -3 2 2 + 8 8 eight 3 -3 2 4 + 8 8 eight 3 -3 3 -3 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 NULL 0 + 8 8 eight 3 -3 NULL NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL NULL + 8 8 eight 5 -5 NULL NULL + 8 8 eight NULL 0 0 NULL + 8 8 eight NULL 0 1 -1 + 8 8 eight NULL 0 2 2 + 8 8 eight NULL 0 2 4 + 8 8 eight NULL 0 3 -3 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 NULL 0 + 8 8 eight NULL 0 NULL NULL + 8 8 eight NULL NULL 0 NULL + 8 8 eight NULL NULL 1 -1 + 8 8 eight NULL NULL 2 2 + 8 8 eight NULL NULL 2 4 + 8 8 eight NULL NULL 3 -3 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL NULL 0 + 8 8 eight NULL NULL NULL NULL + NULL 0 zero 0 NULL 0 NULL + NULL 0 zero 0 NULL 1 -1 + NULL 0 zero 0 NULL 2 2 + NULL 0 zero 0 NULL 2 4 + NULL 0 zero 0 NULL 3 -3 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL NULL 0 + NULL 0 zero 0 NULL NULL NULL + NULL 0 zero 1 -1 0 NULL + NULL 0 zero 1 -1 1 -1 + NULL 0 zero 1 -1 2 2 + NULL 0 zero 1 -1 2 4 + NULL 0 zero 1 -1 3 -3 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 NULL 0 + NULL 0 zero 1 -1 NULL NULL + NULL 0 zero 2 2 0 NULL + NULL 0 zero 2 2 1 -1 + NULL 0 zero 2 2 2 2 + NULL 0 zero 2 2 2 4 + NULL 0 zero 2 2 3 -3 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 NULL 0 + NULL 0 zero 2 2 NULL NULL + NULL 0 zero 2 4 0 NULL + NULL 0 zero 2 4 1 -1 + NULL 0 zero 2 4 2 2 + NULL 0 zero 2 4 2 4 + NULL 0 zero 2 4 3 -3 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 NULL 0 + NULL 0 zero 2 4 NULL NULL + NULL 0 zero 3 -3 0 NULL + NULL 0 zero 3 -3 1 -1 + NULL 0 zero 3 -3 2 2 + NULL 0 zero 3 -3 2 4 + NULL 0 zero 3 -3 3 -3 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 NULL 0 + NULL 0 zero 3 -3 NULL NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero NULL 0 0 NULL + NULL 0 zero NULL 0 1 -1 + NULL 0 zero NULL 0 2 2 + NULL 0 zero NULL 0 2 4 + NULL 0 zero NULL 0 3 -3 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 NULL 0 + NULL 0 zero NULL 0 NULL NULL + NULL 0 zero NULL NULL 0 NULL + NULL 0 zero NULL NULL 1 -1 + NULL 0 zero NULL NULL 2 2 + NULL 0 zero NULL NULL 2 4 + NULL 0 zero NULL NULL 3 -3 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL NULL 0 + NULL 0 zero NULL NULL NULL NULL + NULL NULL null 0 NULL 0 NULL + NULL NULL null 0 NULL 1 -1 + NULL NULL null 0 NULL 2 2 + NULL NULL null 0 NULL 2 4 + NULL NULL null 0 NULL 3 -3 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL NULL 0 + NULL NULL null 0 NULL NULL NULL + NULL NULL null 1 -1 0 NULL + NULL NULL null 1 -1 1 -1 + NULL NULL null 1 -1 2 2 + NULL NULL null 1 -1 2 4 + NULL NULL null 1 -1 3 -3 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 NULL 0 + NULL NULL null 1 -1 NULL NULL + NULL NULL null 2 2 0 NULL + NULL NULL null 2 2 1 -1 + NULL NULL null 2 2 2 2 + NULL NULL null 2 2 2 4 + NULL NULL null 2 2 3 -3 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 NULL 0 + NULL NULL null 2 2 NULL NULL + NULL NULL null 2 4 0 NULL + NULL NULL null 2 4 1 -1 + NULL NULL null 2 4 2 2 + NULL NULL null 2 4 2 4 + NULL NULL null 2 4 3 -3 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 NULL 0 + NULL NULL null 2 4 NULL NULL + NULL NULL null 3 -3 0 NULL + NULL NULL null 3 -3 1 -1 + NULL NULL null 3 -3 2 2 + NULL NULL null 3 -3 2 4 + NULL NULL null 3 -3 3 -3 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 NULL 0 + NULL NULL null 3 -3 NULL NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL NULL + NULL NULL null 5 -5 NULL NULL + NULL NULL null NULL 0 0 NULL + NULL NULL null NULL 0 1 -1 + NULL NULL null NULL 0 2 2 + NULL NULL null NULL 0 2 4 + NULL NULL null NULL 0 3 -3 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 NULL 0 + NULL NULL null NULL 0 NULL NULL + NULL NULL null NULL NULL 0 NULL + NULL NULL null NULL NULL 1 -1 + NULL NULL null NULL NULL 2 2 + NULL NULL null NULL NULL 2 4 + NULL NULL null NULL NULL 3 -3 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL NULL 0 + NULL NULL null NULL NULL NULL NULL + + +-- !query 37 +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) + FROM J1_TBL INNER JOIN J2_TBL USING (i) +-- !query 37 schema +struct +-- !query 37 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query 38 +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k + FROM J1_TBL JOIN J2_TBL USING (i) +-- !query 38 schema +struct +-- !query 38 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query 39 +SELECT udf('') AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY udf(udf(a)), udf(d) +-- !query 39 schema +struct +-- !query 39 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query 40 +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL NATURAL JOIN J2_TBL +-- !query 40 schema +struct +-- !query 40 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query 41 +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) +-- !query 41 schema +struct +-- !query 41 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query 42 +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) +-- !query 42 schema +struct +-- !query 42 output + 0 NULL zero NULL + 2 3 two 2 + 4 1 four 2 + + +-- !query 43 +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i) +-- !query 43 schema +struct +-- !query 43 output + 0 NULL zero 0 NULL + 1 4 one 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + + +-- !query 44 +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)) +-- !query 44 schema +struct +-- !query 44 output + 0 NULL zero NULL 0 + 2 3 two 2 2 + 4 1 four 2 4 + + +-- !query 45 +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))) +-- !query 45 schema +struct +-- !query 45 output + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero NULL 0 + 1 4 one 2 2 + 1 4 one 2 4 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 2 4 + 4 1 four 2 4 + + +-- !query 46 +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query 46 schema +struct +-- !query 46 output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query 47 +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY udf(i), udf(udf(k)), udf(t) +-- !query 47 schema +struct +-- !query 47 output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query 48 +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) +-- !query 48 schema +struct +-- !query 48 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query 49 +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) + FROM J1_TBL RIGHT JOIN J2_TBL USING (i) +-- !query 49 schema +struct +-- !query 49 output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query 50 +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query 50 schema +struct +-- !query 50 output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query 51 +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(udf(t)) +-- !query 51 schema +struct +-- !query 51 output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query 52 +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1) +-- !query 52 schema +struct +-- !query 52 output + + + +-- !query 53 +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)) +-- !query 53 schema +struct +-- !query 53 output + 1 4 one -1 + + +-- !query 54 +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet +-- !query 56 schema +struct<> +-- !query 56 output + + + +-- !query 57 +INSERT INTO t1 VALUES ( 'bb', 11 ) +-- !query 57 schema +struct<> +-- !query 57 output + + + +-- !query 58 +INSERT INTO t2 VALUES ( 'bb', 12 ) +-- !query 58 schema +struct<> +-- !query 58 output + + + +-- !query 59 +INSERT INTO t2 VALUES ( 'cc', 22 ) +-- !query 59 schema +struct<> +-- !query 59 output + + + +-- !query 60 +INSERT INTO t2 VALUES ( 'ee', 42 ) +-- !query 60 schema +struct<> +-- !query 60 output + + + +-- !query 61 +INSERT INTO t3 VALUES ( 'bb', 13 ) +-- !query 61 schema +struct<> +-- !query 61 output + + + +-- !query 62 +INSERT INTO t3 VALUES ( 'cc', 23 ) +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +INSERT INTO t3 VALUES ( 'dd', 33 ) +-- !query 63 schema +struct<> +-- !query 63 output + + + +-- !query 64 +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) +-- !query 64 schema +struct +-- !query 64 output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query 65 +SELECT * FROM +(SELECT udf(name) as name, t2.n FROM t2) as s2 +INNER JOIN +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 +USING (name) +-- !query 65 schema +struct +-- !query 65 output +bb 12 13 +cc 22 23 + + +-- !query 66 +SELECT * FROM +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 +LEFT JOIN +(SELECT udf(name) as name, t3.n FROM t3) s3 +USING (name) +-- !query 66 schema +struct +-- !query 66 output +bb 12 13 +cc 22 23 +ee 42 NULL + + +-- !query 67 +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query 67 schema +struct +-- !query 67 output +bb 12 13 +cc 22 23 +dd NULL 33 +ee 42 NULL + + +-- !query 68 +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query 68 schema +struct +-- !query 68 output +bb 12 2 13 3 +cc 22 2 23 3 + + +-- !query 69 +SELECT * FROM +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query 69 schema +struct +-- !query 69 output +bb 12 2 13 3 +cc 22 2 23 3 +ee 42 2 NULL NULL + + +-- !query 70 +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query 70 schema +struct +-- !query 70 output +bb 12 2 13 3 +cc 22 2 23 3 +dd NULL NULL 33 3 +ee 42 2 NULL NULL + + +-- !query 71 +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query 71 schema +struct +-- !query 71 output +bb 11 1 12 2 13 3 + + +-- !query 72 +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query 72 schema +struct +-- !query 72 output +bb 11 1 12 2 13 3 +cc NULL NULL 22 2 23 3 +dd NULL NULL NULL NULL 33 3 +ee NULL NULL 42 2 NULL NULL + + +-- !query 73 +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(n) as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 + ) ss2 +-- !query 73 schema +struct +-- !query 73 output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query 74 +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(n) as s3_n FROM t3) as s3 + ) ss2 +-- !query 74 schema +struct +-- !query 74 output +bb 11 12 2 13 +cc NULL 22 2 23 +dd NULL NULL NULL 33 +ee NULL 42 2 NULL + + +-- !query 75 +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (udf(udf(s1_n)) = udf(s2_n)) +-- !query 75 schema +struct +-- !query 75 output +NULL NULL bb 2 +NULL NULL cc 2 +NULL NULL ee 2 +bb 11 NULL NULL + + +-- !query 76 +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2) +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2) +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +select udf(udf(x1)), udf(x2) from x +-- !query 78 schema +struct +-- !query 78 output +1 11 +2 22 +3 NULL +4 44 +5 NULL + + +-- !query 79 +select udf(y1), udf(udf(y2)) from y +-- !query 79 schema +struct +-- !query 79 output +1 111 +2 222 +3 333 +4 NULL + + +-- !query 80 +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null) +-- !query 80 schema +struct +-- !query 80 output +1 11 1 111 +2 22 2 222 +3 NULL NULL NULL +4 44 4 NULL +5 NULL NULL NULL + + +-- !query 81 +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null) +-- !query 81 schema +struct +-- !query 81 output +1 11 1 111 +2 22 2 222 +3 NULL 3 333 +4 44 NULL NULL +5 NULL NULL NULL + + +-- !query 82 +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)) +-- !query 82 schema +struct +-- !query 82 output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL 5 NULL + + +-- !query 83 +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null) +-- !query 83 schema +struct +-- !query 83 output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query 84 +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null) +-- !query 84 schema +struct +-- !query 84 output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL NULL NULL +5 NULL NULL NULL NULL NULL + + +-- !query 85 +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null) +-- !query 85 schema +struct +-- !query 85 output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query 86 +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null) +-- !query 86 schema +struct +-- !query 86 output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query 87 +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null) +-- !query 87 schema +struct +-- !query 87 output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL + + +-- !query 88 +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null) +-- !query 88 schema +struct +-- !query 88 output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query 89 +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)) +-- !query 89 schema +struct +-- !query 89 output +1 + + +-- !query 90 +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))) +-- !query 90 schema +struct +-- !query 90 output +1 + + +-- !query 91 +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))) +-- !query 91 schema +struct +-- !query 91 output +1 + + +-- !query 92 +select * from int8_tbl i1 left join (int8_tbl i2 join + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2 +-- !query 92 schema +struct +-- !query 92 output +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 NULL NULL NULL +123 456 123 456 123 +123 4567890123456789 123 4567890123456789 123 +4567890123456789 4567890123456789 123 4567890123456789 123 + + +-- !query 93 +select udf(count(*)) +from + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 + from tenk1 t1 + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, + tenk1 t4, + tenk1 t5 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)) +-- !query 93 schema +struct +-- !query 93 output +1000 + + +-- !query 94 +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from + tenk1 t, + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)) +-- !query 94 schema +struct +-- !query 94 output + + + +-- !query 95 +select * from + j1_tbl full join + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k) +-- !query 95 schema +struct +-- !query 95 output +0 NULL zero NULL NULL +1 4 one NULL NULL +2 3 two 2 2 +3 2 three NULL NULL +4 1 four NULL NULL +5 0 five NULL NULL +6 6 six NULL NULL +7 7 seven NULL NULL +8 8 eight NULL NULL +NULL 0 zero NULL NULL +NULL NULL NULL 0 NULL +NULL NULL NULL 1 -1 +NULL NULL NULL 2 4 +NULL NULL NULL 3 -3 +NULL NULL NULL 5 -5 +NULL NULL NULL 5 -5 +NULL NULL NULL NULL 0 +NULL NULL NULL NULL NULL +NULL NULL null NULL NULL + + +-- !query 96 +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x + left join + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2 +-- !query 96 schema +struct +-- !query 96 output +10000 + + +-- !query 97 +DROP TABLE t1 +-- !query 97 schema +struct<> +-- !query 97 output + + + +-- !query 98 +DROP TABLE t2 +-- !query 98 schema +struct<> +-- !query 98 output + + + +-- !query 99 +DROP TABLE t3 +-- !query 99 schema +struct<> +-- !query 99 output + + + +-- !query 100 +DROP TABLE J1_TBL +-- !query 100 schema +struct<> +-- !query 100 output + + + +-- !query 101 +DROP TABLE J2_TBL +-- !query 101 schema +struct<> +-- !query 101 output + + + +-- !query 102 +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol) +-- !query 102 schema +struct<> +-- !query 102 output + + + +-- !query 103 +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol) +-- !query 103 schema +struct<> +-- !query 103 output + + + +-- !query 104 +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol) +-- !query 104 schema +struct +-- !query 104 output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query 105 +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)) +-- !query 105 schema +struct +-- !query 105 output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query 106 +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10) +-- !query 106 schema +struct +-- !query 106 output +100000 + + +-- !query 107 +DROP TABLE IF EXISTS tt3 +-- !query 107 schema +struct<> +-- !query 107 output + + + +-- !query 108 +CREATE TABLE tt3(f1 int, f2 string) USING parquet +-- !query 108 schema +struct<> +-- !query 108 output + + + +-- !query 109 +INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x +-- !query 109 schema +struct<> +-- !query 109 output + + + +-- !query 110 +DROP TABLE IF EXISTS tt4 +-- !query 110 schema +struct<> +-- !query 110 output + + + +-- !query 111 +CREATE TABLE tt4(f1 int) USING parquet +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +INSERT INTO tt4 VALUES (0),(1),(9999) +-- !query 112 schema +struct<> +-- !query 112 output + + + +-- !query 113 +SELECT udf(udf(a.f1)) as f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL +-- !query 113 schema +struct +-- !query 113 output +0 +1 +9999 + + +-- !query 114 +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2) +-- !query 114 schema +struct<> +-- !query 114 output + + + +-- !query 115 +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2) +-- !query 115 schema +struct<> +-- !query 115 output + + + +-- !query 116 +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)) +-- !query 116 schema +struct +-- !query 116 output +1 10 1 9 + + +-- !query 117 +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx) +-- !query 117 schema +struct<> +-- !query 117 output + + + +-- !query 118 +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx) +-- !query 118 schema +struct<> +-- !query 118 output + + + +-- !query 119 +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx) +-- !query 119 schema +struct +-- !query 119 output +101 1 101 1 1 +201 2 NULL NULL 1 +301 NULL NULL NULL 1 + + +-- !query 120 +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1) +-- !query 120 schema +struct<> +-- !query 120 output + + + +-- !query 121 +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2) +-- !query 121 schema +struct<> +-- !query 121 output + + + +-- !query 122 +create or replace temporary view zt3(f3 int) using parquet +-- !query 122 schema +struct<> +-- !query 122 output + + + +-- !query 123 +select * from + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53 +-- !query 123 schema +struct +-- !query 123 output +53 NULL NULL + + +-- !query 124 +create temp view zv1 as select *,'dummy' AS junk from zt1 +-- !query 124 schema +struct<> +-- !query 124 output + + + +-- !query 125 +select * from + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53 +-- !query 125 schema +struct +-- !query 125 output +53 NULL NULL NULL + + +-- !query 126 +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))) +-- !query 126 schema +struct +-- !query 126 output + + + +-- !query 127 +create or replace temporary view a (i integer) using parquet +-- !query 127 schema +struct<> +-- !query 127 output + + + +-- !query 128 +create or replace temporary view b (x integer, y integer) using parquet +-- !query 128 schema +struct<> +-- !query 128 output + + + +-- !query 129 +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i) +-- !query 129 schema +struct +-- !query 129 output + + + +-- !query 130 +select udf(t1.q2), udf(count(t2.*)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1 +-- !query 130 schema +struct +-- !query 130 output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query 131 +select udf(udf(t1.q2)), udf(count(t2.*)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1 +-- !query 131 schema +struct +-- !query 131 output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query 132 +select udf(t1.q2) as q2, udf(udf(count(t2.*))) +from int8_tbl t1 left join + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) +group by t1.q2 order by 1 +-- !query 132 schema +struct +-- !query 132 output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query 133 +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code) +-- !query 133 schema +struct<> +-- !query 133 output + + + +-- !query 134 +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num) +-- !query 134 schema +struct<> +-- !query 134 output + + + +-- !query 135 +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a) +-- !query 135 schema +struct<> +-- !query 135 output + + + +-- !query 136 +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) + ) as ss + on (udf(udf(c.a)) = udf(ss.code)) +order by c.name +-- !query 136 schema +struct +-- !query 136 output +A p 2 -1 +B q 0 -1 +C NULL NULL NULL + + +-- !query 137 +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = udf(sub6.key6) + ) sub4 + ON udf(sub4.key5) = sub3.key3 +) sub2 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)) +-- !query 137 schema +struct +-- !query 137 output +1 1 1 1 + + +-- !query 138 +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON udf(udf(sub5.key5)) = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = udf(udf(sub2.key3)) +-- !query 138 schema +struct +-- !query 138 output +1 1 1 1 + + +-- !query 139 +SELECT udf(qq), udf(udf(unique1)) + FROM + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON udf(qq) = udf(unique2) +-- !query 139 schema +struct +-- !query 139 output +123 4596 +123 4596 +456 7318 + + +-- !query 140 +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2) +-- !query 140 schema +struct<> +-- !query 140 output + + + +-- !query 141 +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2) +-- !query 141 schema +struct<> +-- !query 141 output + + + +-- !query 142 +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1) +-- !query 142 schema +struct<> +-- !query 142 output + + + +-- !query 143 +select udf(nt3.id) +from nt3 as nt3 + left join + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) + ) as ss2 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3) +-- !query 143 schema +struct +-- !query 143 output +1 + + +-- !query 144 +select * from int4_tbl a full join int4_tbl b on true +-- !query 144 schema +struct +-- !query 144 output +-123456 -123456 +-123456 -2147483647 +-123456 0 +-123456 123456 +-123456 2147483647 +-2147483647 -123456 +-2147483647 -2147483647 +-2147483647 0 +-2147483647 123456 +-2147483647 2147483647 +0 -123456 +0 -2147483647 +0 0 +0 123456 +0 2147483647 +123456 -123456 +123456 -2147483647 +123456 0 +123456 123456 +123456 2147483647 +2147483647 -123456 +2147483647 -2147483647 +2147483647 0 +2147483647 123456 +2147483647 2147483647 + + +-- !query 145 +select * from int4_tbl a full join int4_tbl b on false +-- !query 145 schema +struct +-- !query 145 output +-123456 NULL +-2147483647 NULL +0 NULL +123456 NULL +2147483647 NULL +NULL -123456 +NULL -2147483647 +NULL 0 +NULL 123456 +NULL 2147483647 + + +-- !query 146 +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1 +-- !query 146 schema +struct +-- !query 146 output +10 + + +-- !query 147 +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1) +-- !query 147 schema +struct +-- !query 147 output +NULL +NULL +0 +NULL +NULL + + +-- !query 148 +select * from +( + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) +) ss +where udf(fault) = udf(122) +order by udf(fault) +-- !query 148 schema +struct +-- !query 148 output +NULL 123 122 + + +-- !query 149 +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))) +-- !query 149 schema +struct +-- !query 149 output + + + +-- !query 150 +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0 +-- !query 150 schema +struct +-- !query 150 output +0 0 0 + + +-- !query 151 +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44) +-- !query 151 schema +struct +-- !query 151 output + + + +-- !query 152 +select * from + text_tbl t1 + inner join int8_tbl i8 + on udf(i8.q2) = udf(udf(456)) + right join text_tbl t2 + on udf(t1.f1) = udf(udf('doh!')) + left join int4_tbl i4 + on udf(udf(i8.q1)) = i4.f1 +-- !query 152 schema +struct +-- !query 152 output +doh! 123 456 doh! NULL +doh! 123 456 hi de ho neighbor NULL + + +-- !query 153 +select * from + (select udf(udf(1)) as id) as xx + left join + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))) +-- !query 153 schema +struct +-- !query 153 output +1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 + + +-- !query 154 +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0) +-- !query 154 schema +struct +-- !query 154 output +-4567890123456789 NULL +123 123 +123 123 +456 NULL +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 155 +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd) +-- !query 155 schema +struct<> +-- !query 155 output + + + +-- !query 156 +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd) +-- !query 156 schema +struct<> +-- !query 156 output + + + +-- !query 157 +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)) +-- !query 157 schema +struct +-- !query 157 output +1 10 +2 20 +3 30 + + +-- !query 158 +select p.*, linked from parent p + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))) +-- !query 158 schema +struct +-- !query 158 output +1 10 true +2 20 NULL +3 30 NULL + + +-- !query 159 +select p.* from + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query 159 schema +struct +-- !query 159 output + + + +-- !query 160 +select p.* from + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query 160 schema +struct +-- !query 160 output + + + +-- !query 161 +create or replace temporary view a as select * from + (values (0), (1)) + as v(id) +-- !query 161 schema +struct<> +-- !query 161 output + + + +-- !query 162 +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id) +-- !query 162 schema +struct<> +-- !query 162 output + + + +-- !query 163 +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0) +-- !query 163 schema +struct +-- !query 163 output +1 NULL NULL + + +-- !query 164 +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0) +-- !query 164 schema +struct +-- !query 164 output +1 NULL + + +-- !query 165 +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1) +-- !query 165 schema +struct<> +-- !query 165 output + + + +-- !query 166 +SELECT * FROM + (SELECT udf(1) AS x) ss1 + LEFT JOIN + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 + ON true +-- !query 166 schema +struct +-- !query 166 output +1 123 456 123 +1 123 4567890123456789 123 +1 4567890123456789 -4567890123456789 4567890123456789 +1 4567890123456789 123 42 +1 4567890123456789 4567890123456789 4567890123456789 + + +-- !query 167 +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1) +-- !query 167 schema +struct<> +-- !query 167 output +org.apache.spark.sql.AnalysisException +Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 72 + + +-- !query 168 +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) +-- !query 168 schema +struct<> +-- !query 168 output +org.apache.spark.sql.AnalysisException +cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 + + +-- !query 169 +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)) +-- !query 169 schema +struct +-- !query 169 output + + + +-- !query 170 +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) +-- !query 170 schema +struct<> +-- !query 170 output +org.apache.spark.sql.AnalysisException +cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 + + +-- !query 171 +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two +-- !query 171 schema +struct<> +-- !query 171 output +org.apache.spark.sql.AnalysisException +cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 15 + + +-- !query 172 +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) +-- !query 172 schema +struct<> +-- !query 172 output +org.apache.spark.sql.AnalysisException +cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 + + +-- !query 173 +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss +-- !query 173 schema +struct<> +-- !query 173 output +org.apache.spark.sql.AnalysisException +cannot resolve '`f1`' given input columns: []; line 1 pos 55 + + +-- !query 174 +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss +-- !query 174 schema +struct<> +-- !query 174 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a.f1`' given input columns: []; line 1 pos 42 + + +-- !query 175 +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss +-- !query 175 schema +struct<> +-- !query 175 output +org.apache.spark.sql.AnalysisException +cannot resolve '`f1`' given input columns: []; line 1 pos 61 + + +-- !query 176 +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss +-- !query 176 schema +struct<> +-- !query 176 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a.f1`' given input columns: []; line 1 pos 60 + + +-- !query 177 +CREATE TABLE j1 (id1 int, id2 int) USING parquet +-- !query 177 schema +struct<> +-- !query 177 output + + + +-- !query 178 +CREATE TABLE j2 (id1 int, id2 int) USING parquet +-- !query 178 schema +struct<> +-- !query 178 output + + + +-- !query 179 +INSERT INTO j1 values(1,1),(1,2) +-- !query 179 schema +struct<> +-- !query 179 output + + + +-- !query 180 +INSERT INTO j2 values(1,1) +-- !query 180 schema +struct<> +-- !query 180 output + + + +-- !query 181 +INSERT INTO j2 values(1,2) +-- !query 181 schema +struct<> +-- !query 181 output + + + +-- !query 182 +select * from j1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1 +-- !query 182 schema +struct +-- !query 182 output +1 1 1 1 +1 2 1 2 + + +-- !query 183 +drop table j1 +-- !query 183 schema +struct<> +-- !query 183 output + + + +-- !query 184 +drop table j2 +-- !query 184 schema +struct<> +-- !query 184 output + From 2fd83c28203ef9c300a3feaaecc8edb5546814cf Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 19 Aug 2019 20:15:17 +0900 Subject: [PATCH 103/149] [SPARK-28756][R][FOLLOW-UP] Specify minimum and maximum Java versions ### What changes were proposed in this pull request? This PR proposes to set minimum and maximum Java version specification. (see https://cran.r-project.org/doc/manuals/r-release/R-exts.html#Writing-portable-packages). Seems there is not the standard way to specify both given the documentation and other packages (see https://gist.github.com/glin/bd36cf1eb0c7f8b1f511e70e2fb20f8d). I found two ways from existing packages on CRAN. ``` Package (<= 1 & > 2) Package (<= 1, > 2) ``` The latter seems closer to other standard notations such as `R (>= 2.14.0), R (>= r56550)`. So I have chosen the latter way. ### Why are the changes needed? Seems the package might be rejected by CRAN. See https://github.com/apache/spark/pull/25472#issuecomment-522405742 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? JDK 8 ```bash ./build/mvn -DskipTests -Psparkr clean package ./R/run-tests.sh ... basic tests for CRAN: ............. ... ``` JDK 11 ```bash ./build/mvn -DskipTests -Psparkr -Phadoop-3.2 clean package ./R/run-tests.sh ... basic tests for CRAN: ............. ... ``` Closes #25490 from HyukjinKwon/SPARK-28756. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/DESCRIPTION | 2 +- R/pkg/R/client.R | 13 ++++++++----- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 6a83e00dff79f..f4780862099d3 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -13,7 +13,7 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), License: Apache License (== 2.0) URL: https://www.apache.org/ https://spark.apache.org/ BugReports: https://spark.apache.org/contributing.html -SystemRequirements: Java (>= 8) +SystemRequirements: Java (>= 8, < 12) Depends: R (>= 3.1), methods diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R index 3299346bce007..2ff68ab7b9d77 100644 --- a/R/pkg/R/client.R +++ b/R/pkg/R/client.R @@ -64,7 +64,9 @@ checkJavaVersion <- function() { javaBin <- "java" javaHome <- Sys.getenv("JAVA_HOME") javaReqs <- utils::packageDescription(utils::packageName(), fields = c("SystemRequirements")) - sparkJavaVersion <- as.numeric(tail(strsplit(javaReqs, "[(=)]")[[1]], n = 1L)) + sparkJavaVersions <- strsplit(javaReqs, "[(,)]")[[1]] + minJavaVersion <- as.numeric(strsplit(sparkJavaVersions[[2]], ">= ")[[1]][[2]]) + maxJavaVersion <- as.numeric(strsplit(sparkJavaVersions[[3]], "< ")[[1]][[2]]) if (javaHome != "") { javaBin <- file.path(javaHome, "bin", javaBin) } @@ -99,10 +101,11 @@ checkJavaVersion <- function() { } else { javaVersionNum <- as.integer(versions[1]) } - if (javaVersionNum < sparkJavaVersion) { - stop(paste("Java version", sparkJavaVersion, - ", or greater, is required for this package; found version:", - javaVersionStr)) + if (javaVersionNum < minJavaVersion || javaVersionNum >= maxJavaVersion) { + stop(paste0("Java version, greater than or equal to ", minJavaVersion, + " and less than ", maxJavaVersion, + ", is required for this package; found version: ", + javaVersionStr)) } return(javaVersionNum) } From 1de4a22c52779bbdf68e40167a91e8606225f6b7 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 19 Aug 2019 20:31:39 +0900 Subject: [PATCH 104/149] Revert "[SPARK-28759][BUILD] Upgrade scala-maven-plugin to 4.1.1" This reverts commit 1819a6f22eee5314197aab4c169c74bd6ff6c17c. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3b0383375c56b..140d19e62672d 100644 --- a/pom.xml +++ b/pom.xml @@ -2280,7 +2280,7 @@ net.alchim31.maven scala-maven-plugin - 4.1.1 + 3.4.4 eclipse-add-source From 5f6eb5d20dee57ea7ba9d47b21c712dee06fa7ef Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 19 Aug 2019 11:06:02 -0700 Subject: [PATCH 105/149] [SPARK-28634][YARN] Ignore kerberos login config in client mode AM This change makes the client mode AM ignore any login configuration, which is now always handled by the driver. The previous code tried to achieve that by modifying the configuration visible to the AM, but that missed the case where old configuration names were being used. Tested in real cluster with reproduction provided in the bug. Closes #25467 from vanzin/SPARK-28634. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../apache/spark/deploy/yarn/ApplicationMaster.scala | 4 +++- .../scala/org/apache/spark/deploy/yarn/Client.scala | 11 ++--------- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5a67caf8e2a60..ae9486d7e44cf 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -851,7 +851,9 @@ object ApplicationMaster extends Logging { master = new ApplicationMaster(amArgs, sparkConf, yarnConf) val ugi = sparkConf.get(PRINCIPAL) match { - case Some(principal) => + // We only need to log in with the keytab in cluster mode. In client mode, the driver + // handles the user keytab. + case Some(principal) if amArgs.userClass != null => val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() SparkHadoopUtil.get.loginUserFromKeytab(principal, sparkConf.get(KEYTAB).orNull) val newUGI = UserGroupInformation.getCurrentUser() diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 651e706021fcb..5fc6894f89098 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -767,15 +767,8 @@ private[spark] class Client( val props = confToProperties(sparkConf) // If propagating the keytab to the AM, override the keytab name with the name of the - // distributed file. Otherwise remove princpal/keytab from the conf, so they're not seen - // by the AM at all. - amKeytabFileName match { - case Some(kt) => - props.setProperty(KEYTAB.key, kt) - case None => - props.remove(PRINCIPAL.key) - props.remove(KEYTAB.key) - } + // distributed file. + amKeytabFileName.foreach { kt => props.setProperty(KEYTAB.key, kt) } writePropertiesToArchive(props, SPARK_CONF_FILE, confStream) From fa7fd8f2a4f0100e225e97b5fb6d6518dd73fa4a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 19 Aug 2019 17:01:14 -0500 Subject: [PATCH 106/149] [SPARK-28434][TESTS][ML] Fix values in dummy tree in DecisionTreeSuite ### What changes were proposed in this pull request? Fix dummy tree created in decision tree tests to have actually consistent stats, so that it can be compared in tests more completely. The current one has values for, say, impurity that don't even match internally. With this, the tests can assert more about stats staying correct after load. ### Why are the changes needed? Fixes a TODO and improves the test slightly. ### Does this PR introduce any user-facing change? None ### How was this patch tested? Existing tests. Closes #25485 from srowen/SPARK-28434. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../spark/mllib/tree/DecisionTreeSuite.scala | 38 ++++++++----------- 1 file changed, 16 insertions(+), 22 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8378a599362a2..88b9d4c039ba9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -561,34 +561,29 @@ object DecisionTreeSuite extends SparkFunSuite { * Create an internal node with the given node ID and feature type. * Note: This does NOT set the child nodes. */ - private def createInternalNode(id: Int, featureType: FeatureType): Node = { + private def createInternalNode( + id: Int, featureType: FeatureType, left: Node, right: Node): Node = { val node = Node(nodeIndex = id, new Predict(0.0, 1.0), impurity = 0.5, isLeaf = false) - featureType match { - case Continuous => - node.split = Some(new Split(feature = 0, threshold = 0.5, Continuous, - categories = List.empty[Double])) - case Categorical => - node.split = Some(new Split(feature = 1, threshold = 0.0, Categorical, - categories = List(0.0, 1.0))) - } - // TODO: The information gain stats should be consistent with info in children: SPARK-7131 - node.stats = Some(new InformationGainStats(gain = 0.1, impurity = 0.2, - leftImpurity = 0.3, rightImpurity = 0.4, new Predict(1.0, 0.4), new Predict(0.0, 0.6))) + node.split = Some(featureType match { + case Continuous => Split(feature = 0, threshold = 0.5, featureType, List.empty[Double]) + case Categorical => Split(feature = 1, threshold = 0.0, featureType, List(0.0, 1.0)) + }) + node.stats = Some(new InformationGainStats(gain = 0.1, impurity = node.impurity, + leftImpurity = left.impurity, rightImpurity = right.impurity, + left.predict, right.predict)) + node.leftNode = Some(left) + node.rightNode = Some(right) node } /** * Create a tree model. This is deterministic and contains a variety of node and feature types. - * TODO: Update to be a correct tree (with matching probabilities, impurities, etc.): SPARK-7131 */ private[spark] def createModel(algo: Algo): DecisionTreeModel = { - val topNode = createInternalNode(id = 1, Continuous) - val (node2, node3) = (createLeafNode(id = 2), createInternalNode(id = 3, Categorical)) - val (node6, node7) = (createLeafNode(id = 6), createLeafNode(id = 7)) - topNode.leftNode = Some(node2) - topNode.rightNode = Some(node3) - node3.leftNode = Some(node6) - node3.rightNode = Some(node7) + val (node6, node7) = (createLeafNode(6), createLeafNode(7)) + val node3 = createInternalNode(3, Categorical, node6, node7) + val node2 = createLeafNode(2) + val topNode = createInternalNode(1, Continuous, node2, node3) new DecisionTreeModel(topNode, algo) } @@ -625,8 +620,7 @@ object DecisionTreeSuite extends SparkFunSuite { assert(a.isLeaf === b.isLeaf) assert(a.split === b.split) (a.stats, b.stats) match { - // TODO: Check other fields besides the information gain. - case (Some(aStats), Some(bStats)) => assert(aStats.gain === bStats.gain) + case (Some(aStats), Some(bStats)) => assert(aStats === bStats) case (None, None) => case _ => fail(s"Only one instance has stats defined. (a.stats: ${a.stats}, " + s"b.stats: ${b.stats})") From 26f344354b5e845b50ca2104e662e901212ddba9 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 19 Aug 2019 15:15:50 -0700 Subject: [PATCH 107/149] [SPARK-27905][SQL][FOLLOW-UP] Add prettyNames ### What changes were proposed in this pull request? This is a follow-up of #24761 which added a higher-order function `ArrayForAll`. The PR mistakenly removed the `prettyName` from `ArrayExists` and forgot to add it to `ArrayForAll`. ### Why are the changes needed? This reverts the `prettyName` back to `ArrayExists` not to affect explained plans, and adds it to `ArrayForAll` to clarify the `prettyName` as the same as the expressions around. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #25501 from ueshin/issues/SPARK-27905/pretty_names. Authored-by: Takuya UESHIN Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/higherOrderFunctions.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 87b35b10d29fa..ed26bb375de25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -445,6 +445,8 @@ case class ArrayExists( false } } + + override def prettyName: String = "exists" } /** @@ -512,6 +514,8 @@ case class ArrayForAll( forall } } + + override def prettyName: String = "forall" } /** From b79cf0d14351c741efe4f27523919a0e24b8b2ed Mon Sep 17 00:00:00 2001 From: Mick Jermsurawong Date: Tue, 20 Aug 2019 09:47:04 +0900 Subject: [PATCH 108/149] [SPARK-28224][SQL] Check overflow in decimal Sum aggregate ## What changes were proposed in this pull request? - Currently `sum` in aggregates for decimal type can overflow and return null. - `Sum` expression codegens arithmetic on `sql.Decimal` and the output which preserves scale and precision goes into `UnsafeRowWriter`. Here overflowing will be converted to null when writing out. - It also does not go through this branch in `DecimalAggregates` because it's expecting precision of the sum (not the elements to be summed) to be less than 5. https://github.com/apache/spark/blob/4ebff5b6d68f26cc1ff9265a5489e0d7c2e05449/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L1400-L1403 - This PR adds the check at the final result of the sum operator itself. https://github.com/apache/spark/blob/4ebff5b6d68f26cc1ff9265a5489e0d7c2e05449/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala#L372-L376 https://issues.apache.org/jira/browse/SPARK-28224 ## How was this patch tested? - Added an integration test on dataframe suite cc mgaido91 JoshRosen Closes #25033 from mickjermsurawong-stripe/SPARK-28224. Authored-by: Mick Jermsurawong Signed-off-by: Takeshi Yamamuro --- .../catalyst/expressions/aggregate/Sum.scala | 7 +++++- .../org/apache/spark/sql/DataFrameSuite.scala | 23 ++++++++++++++++++- 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index ef204ec82c527..d04fe9249d064 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @ExpressionDescription( @@ -89,5 +90,9 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast ) } - override lazy val evaluateExpression: Expression = sum + override lazy val evaluateExpression: Expression = resultType match { + case d: DecimalType => CheckOverflow(sum, d, SQLConf.get.decimalOperationsNullOnOverflow) + case _ => sum + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ba8fced983c61..c6daff1479fb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExc import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} -import org.apache.spark.sql.test.SQLTestData.{NullStrings, TestData2} +import org.apache.spark.sql.test.SQLTestData.{DecimalData, NullStrings, TestData2} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom @@ -156,6 +156,27 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { structDf.select(xxhash64($"a", $"record.*"))) } + test("SPARK-28224: Aggregate sum big decimal overflow") { + val largeDecimals = spark.sparkContext.parallelize( + DecimalData(BigDecimal("1"* 20 + ".123"), BigDecimal("1"* 20 + ".123")) :: + DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() + + Seq(true, false).foreach { nullOnOverflow => + withSQLConf((SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key, nullOnOverflow.toString)) { + val structDf = largeDecimals.select("a").agg(sum("a")) + if (nullOnOverflow) { + checkAnswer(structDf, Row(null)) + } else { + val e = intercept[SparkException] { + structDf.collect + } + assert(e.getCause.getClass.equals(classOf[ArithmeticException])) + assert(e.getCause.getMessage.contains("cannot be represented as Decimal")) + } + } + } + } + test("Star Expansion - explode should fail with a meaningful message if it takes a star") { val df = Seq(("1,2"), ("4"), ("7,8,9")).toDF("csv") val e = intercept[AnalysisException] { From 3b4e345fa1afa0d4004988f8800b63150c305fd4 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 19 Aug 2019 17:54:25 -0700 Subject: [PATCH 109/149] [SPARK-28775][CORE][TESTS] Skip date 8633 in Kwajalein due to changes in tzdata2018i that only some JDK 8s use ### What changes were proposed in this pull request? Some newer JDKs use the tzdata2018i database, which changes how certain (obscure) historical dates and timezones are handled. As previously, we can pretty much safely ignore these in tests, as the value may vary by JDK. ### Why are the changes needed? Test otherwise fails using, for example, JDK 1.8.0_222. https://bugs.openjdk.java.net/browse/JDK-8215982 has a full list of JDKs which has this. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests Closes #25504 from srowen/SPARK-28775. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 195690f88cdf2..056337205ae7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -542,12 +542,12 @@ class DateTimeUtilsSuite extends SparkFunSuite { // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( - "Kwajalein" -> Set(8632), + "Kwajalein" -> Set(8632, 8633), "Pacific/Apia" -> Set(15338), "Pacific/Enderbury" -> Set(9130, 9131), "Pacific/Fakaofo" -> Set(15338), "Pacific/Kiritimati" -> Set(9130, 9131), - "Pacific/Kwajalein" -> Set(8632), + "Pacific/Kwajalein" -> Set(8632, 8633), "MIT" -> Set(15338)) for (tz <- ALL_TIMEZONES) { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) From a787bc28840eafae53a08137a53ea56500bfd675 Mon Sep 17 00:00:00 2001 From: darrentirto Date: Mon, 19 Aug 2019 20:44:46 -0700 Subject: [PATCH 110/149] [SPARK-28777][PYTHON][DOCS] Fix format_string doc string with the correct parameters ### What changes were proposed in this pull request? The parameters doc string of the function format_string was changed from _col_, _d_ to _format_, _cols_ which is what the actual function declaration states ### Why are the changes needed? The parameters stated by the documentation was inaccurate ### Does this PR introduce any user-facing change? Yes. **BEFORE** ![before](https://user-images.githubusercontent.com/9700541/63310013-e21a0e80-c2ad-11e9-806b-1d272c5cde12.png) **AFTER** ![after](https://user-images.githubusercontent.com/9700541/63315812-6b870c00-c2c1-11e9-8165-82782628cd1a.png) ### How was this patch tested? N/A: documentation only Closes #25506 from darrentirto/SPARK-28777. Authored-by: darrentirto Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/functions.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 09d20fc7673a2..9e5b61f9e00b4 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1569,8 +1569,9 @@ def format_string(format, *cols): """ Formats the arguments in printf-style and returns the result as a string column. - :param col: the column name of the numeric value to be formatted - :param d: the N decimal places + :param format: string that can contain embedded format tags and used as result column's value + :param cols: list of column names (string) or list of :class:`Column` expressions to + be used in formatting >>> df = spark.createDataFrame([(5, "hello")], ['a', 'b']) >>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect() From 0d3a783cc57ed09650ee31851a19728d8f16cd0c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 20 Aug 2019 13:47:59 +0800 Subject: [PATCH 111/149] [SPARK-28699][CORE] Fix a corner case for aborting indeterminate stage ### What changes were proposed in this pull request? Change the logic of collecting the indeterminate stage, we should look at stages from mapStage, not failedStage during handle FetchFailed. ### Why are the changes needed? In the fetch failed error handle logic, the original logic of collecting indeterminate stage from the fetch failed stage. And in the scenario of the fetch failed happened in the first task of this stage, this logic will cause the indeterminate stage to resubmit partially. Eventually, we are capable of getting correctness bug. ### Does this PR introduce any user-facing change? It makes the corner case of indeterminate stage abort as expected. ### How was this patch tested? New UT in DAGSchedulerSuite. Run below integrated test with `local-cluster[5, 2, 5120]`, and set `spark.sql.execution.sortBeforeRepartition`=false, it will abort the indeterminate stage as expected: ``` import scala.sys.process._ import org.apache.spark.TaskContext val res = spark.range(0, 10000 * 10000, 1).map{ x => (x % 1000, x)} // kill an executor in the stage that performs repartition(239) val df = res.repartition(113).map{ x => (x._1 + 1, x._2)}.repartition(239).map { x => if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 1 && TaskContext.get.stageAttemptNumber == 0) { throw new Exception("pkill -f -n java".!!) } x } val r2 = df.distinct.count() ``` Closes #25498 from xuanyuanking/SPARK-28699-followup. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../apache/spark/scheduler/DAGScheduler.scala | 6 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 53 +++++++++++-------- 2 files changed, 35 insertions(+), 24 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 482691c94f87e..c03e3e0bbaf59 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1571,13 +1571,13 @@ 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.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) { - // It's a little tricky to find all the succeeding stages of `failedStage`, because + // It's a little tricky to find all the succeeding stages of `mapStage`, because // each stage only know its parents not children. Here we traverse the stages from // the leaf nodes (the result stages of active jobs), and rollback all the stages - // in the stage chains that connect to the `failedStage`. To speed up the stage + // in the stage chains that connect to the `mapStage`. To speed up the stage // traversing, we collect the stages to rollback first. If a stage needs to // rollback, all its succeeding stages need to rollback to. - val stagesToRollback = HashSet(failedStage) + val stagesToRollback = HashSet[Stage](mapStage) def collectStagesToRollback(stageChain: List[Stage]): Unit = { if (stagesToRollback.contains(stageChain.head)) { 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 cff3ebf2fb7e0..2b3423f9a4d40 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2741,27 +2741,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), null)) - val failedStages = scheduler.failedStages.toSeq - assert(failedStages.length == 2) - // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. - assert(failedStages.collect { - case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage - }.head.findMissingPartitions() == Seq(0)) - // The result stage is still waiting for its 2 tasks to complete - assert(failedStages.collect { - case stage: ResultStage => stage - }.head.findMissingPartitions() == Seq(0, 1)) - - scheduler.resubmitFailedStages() - - // The first task of the `shuffleMapRdd2` failed with fetch failure - runEvent(makeCompletionEvent( - taskSets(3).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), - null)) - - // The job should fail because Spark can't rollback the shuffle map stage. - assert(failure != null && failure.getMessage.contains("Spark cannot rollback")) + // The second shuffle map stage need to rerun, the job will abort for the indeterminate + // stage rerun. + // TODO: After we support re-generate shuffle file(SPARK-25341), this test will be extended. + assert(failure != null && failure.getMessage + .contains("Spark cannot rollback the ShuffleMapStage 1")) } private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { @@ -2872,6 +2856,33 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(latch.await(10, TimeUnit.SECONDS)) } + test("SPARK-28699: abort stage if parent stage is indeterminate stage") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) + + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleId = shuffleDep.shuffleId + val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) + + submit(finalRdd, Array(0, 1)) + + // Finish the first shuffle map stage. + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + null)) + + // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd` needs to retry. + // The result stage is still waiting for its 2 tasks to complete. + // Because of shuffleMapRdd is indeterminate, this job will be abort. + assert(failure != null && failure.getMessage + .contains("Spark cannot rollback the ShuffleMapStage 0")) + } + test("Completions in zombie tasksets update status of non-zombie taskset") { val parts = 4 val shuffleMapRdd = new MyRDD(sc, parts, Nil) From bc75ed675b0d0bbd9d94d4b33a70e8db74ac95a0 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 20 Aug 2019 14:21:47 +0800 Subject: [PATCH 112/149] [SPARK-28483][CORE] Fix canceling a spark job using barrier mode but barrier tasks blocking on BarrierTaskContext.barrier() ## What changes were proposed in this pull request? Fix canceling a spark job using barrier mode but barrier tasks do not exit. Currently, when spark tasks are killed, `BarrierTaskContext.barrier()` cannot be killed (it will blocking on RPC request), cause the task blocking and cannot exit. In my PR I implement an interface for RPC which support `abort` in class `RpcEndpointRef` ``` def askAbortable[T: ClassTag]( message: Any, timeout: RpcTimeout): AbortableRpcFuture[T] ``` The returned `AbortableRpcFuture` instance include an `abort` method so that we can abort the RPC before it timeout. ## How was this patch tested? Unit test added. Manually test: ### Test code launch spark-shell via `spark-shell --master local[4]` and run following code: ``` sc.setLogLevel("INFO") import org.apache.spark.BarrierTaskContext val n = 4 def taskf(iter: Iterator[Int]): Iterator[Int] = { val context = BarrierTaskContext.get() val x = iter.next() if (x % 2 == 0) { // sleep 6000000 seconds with task killed checking for (i <- 0 until 6000000) { Thread.sleep(1000) if (context.isInterrupted()) { throw new org.apache.spark.TaskKilledException() } } } context.barrier() return Iterator.empty } // launch spark job, including 4 tasks, tasks 1/3 will enter `barrier()`, and tasks 0/2 will enter `sleep` sc.parallelize((0 to n), n).barrier().mapPartitions(taskf).collect() ``` And then press Ctrl+C to exit the running job. ### Before press Ctrl+C to exit the running job, then open spark UI we can see 2 tasks (task 1/3) are not killed. They are blocking. ### After press Ctrl+C to exit the running job, we can see in spark UI all tasks killed successfully. Please review https://spark.apache.org/contributing.html before opening a pull request. Closes #25235 from WeichenXu123/sc_14848. Authored-by: WeichenXu Signed-off-by: Wenchen Fan --- .../org/apache/spark/BarrierTaskContext.scala | 21 +++++++- .../org/apache/spark/rpc/RpcEndpointRef.scala | 29 +++++++++++ .../apache/spark/rpc/netty/NettyRpcEnv.scala | 24 +++++++-- .../org/apache/spark/rpc/netty/Outbox.scala | 12 ++++- .../org/apache/spark/util/ThreadUtils.scala | 7 ++- .../org/apache/spark/rpc/RpcEnvSuite.scala | 42 +++++++++++++++ .../scheduler/BarrierTaskContextSuite.scala | 51 +++++++++++++++++++ 7 files changed, 178 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index c393df8f02226..a841508578a98 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.util.{Properties, Timer, TimerTask} +import scala.concurrent.TimeoutException import scala.concurrent.duration._ import org.apache.spark.annotation.{Experimental, Since} @@ -117,12 +118,30 @@ class BarrierTaskContext private[spark] ( timer.schedule(timerTask, 60000, 60000) try { - barrierCoordinator.askSync[Unit]( + val abortableRpcFuture = barrierCoordinator.askAbortable[Unit]( message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, barrierEpoch), // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework. timeout = new RpcTimeout(365.days, "barrierTimeout")) + + // Wait the RPC future to be completed, but every 1 second it will jump out waiting + // and check whether current spark task is killed. If killed, then throw + // a `TaskKilledException`, otherwise continue wait RPC until it completes. + while(!abortableRpcFuture.toFuture.isCompleted) { + if (taskContext.isInterrupted()) { + val reason = taskContext.getKillReason().get + abortableRpcFuture.abort(reason) + throw new TaskKilledException(reason) + } + // wait RPC future for at most 1 second + try { + ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) + } catch { + case _: TimeoutException => Unit // await future time reach 1 second. + } + } + barrierEpoch += 1 logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) finished " + "global sync successfully, waited for " + diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala index 6c4c0383b3c7f..49d58929a97a4 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -46,6 +46,17 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf) */ def send(message: Any): Unit + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a + * [[AbortableRpcFuture]] to receive the reply within the specified timeout. + * The [[AbortableRpcFuture]] instance wraps [[Future]] with additional `abort` method. + * + * This method only sends the message once and never retries. + */ + def askAbortable[T: ClassTag](message: Any, timeout: RpcTimeout): AbortableRpcFuture[T] = { + throw new UnsupportedOperationException() + } + /** * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to * receive the reply within the specified timeout. @@ -93,3 +104,21 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf) } } + +/** + * An exception thrown if the RPC is aborted. + */ +class RpcAbortException(message: String) extends Exception(message) + +/** + * A wrapper for [[Future]] but add abort method. + * This is used in long run RPC and provide an approach to abort the RPC. + */ +private[spark] class AbortableRpcFuture[T: ClassTag]( + future: Future[T], + onAbort: String => Unit) { + + def abort(reason: String): Unit = onAbort(reason) + + def toFuture: Future[T] = future +} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 5dce43b7523d9..265e158d7c5e3 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -204,7 +204,8 @@ private[netty] class NettyRpcEnv( clientFactory.createClient(address.host, address.port) } - private[netty] def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = { + private[netty] def askAbortable[T: ClassTag]( + message: RequestMessage, timeout: RpcTimeout): AbortableRpcFuture[T] = { val promise = Promise[Any]() val remoteAddr = message.receiver.address @@ -225,6 +226,10 @@ private[netty] class NettyRpcEnv( } } + def onAbort(reason: String): Unit = { + onFailure(new RpcAbortException(reason)) + } + try { if (remoteAddr == address) { val p = Promise[Any]() @@ -240,6 +245,7 @@ private[netty] class NettyRpcEnv( postToOutbox(message.receiver, rpcMessage) promise.future.failed.foreach { case _: TimeoutException => rpcMessage.onTimeout() + case _: RpcAbortException => rpcMessage.onAbort() case _ => }(ThreadUtils.sameThread) } @@ -257,7 +263,14 @@ private[netty] class NettyRpcEnv( case NonFatal(e) => onFailure(e) } - promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread) + + new AbortableRpcFuture[T]( + promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread), + onAbort) + } + + private[netty] def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = { + askAbortable(message, timeout).toFuture } private[netty] def serialize(content: Any): ByteBuffer = { @@ -528,8 +541,13 @@ private[netty] class NettyRpcEndpointRef( override def name: String = endpointAddress.name + override def askAbortable[T: ClassTag]( + message: Any, timeout: RpcTimeout): AbortableRpcFuture[T] = { + nettyEnv.askAbortable(new RequestMessage(nettyEnv.address, this, message), timeout) + } + override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { - nettyEnv.ask(new RequestMessage(nettyEnv.address, this, message), timeout) + askAbortable(message, timeout).toFuture } override def send(message: Any): Unit = { diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala index 3db63934813af..205e6e966866f 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala @@ -66,14 +66,22 @@ private[netty] case class RpcOutboxMessage( this.requestId = client.sendRpc(content, this) } - def onTimeout(): Unit = { + private[netty] def removeRpcRequest(): Unit = { if (client != null) { client.removeRpcRequest(requestId) } else { - logError("Ask timeout before connecting successfully") + logError("Ask terminated before connecting successfully") } } + def onTimeout(): Unit = { + removeRpcRequest() + } + + def onAbort(): Unit = { + removeRpcRequest() + } + override def onFailure(e: Throwable): Unit = { _onFailure(e) } diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 04b0b4c37df9e..8df331251c749 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -29,6 +29,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import scala.util.control.NonFatal import org.apache.spark.SparkException +import org.apache.spark.rpc.RpcAbortException private[spark] object ThreadUtils { @@ -220,8 +221,10 @@ private[spark] object ThreadUtils { } catch { case e: SparkFatalException => throw e.throwable - // TimeoutException is thrown in the current thread, so not need to warp the exception. - case NonFatal(t) if !t.isInstanceOf[TimeoutException] => + // TimeoutException and RpcAbortException is thrown in the current thread, so not need to warp + // the exception. + case NonFatal(t) + if !t.isInstanceOf[TimeoutException] && !t.isInstanceOf[RpcAbortException] => throw new SparkException("Exception thrown in awaitResult: ", t) } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 99b4e8fe8280c..5bdf71be35b3b 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -191,6 +191,48 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } + test("ask a message abort") { + env.setupEndpoint("ask-abort", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + Thread.sleep(10000) + context.reply(msg) + } + }) + + val conf = new SparkConf() + val shortProp = "spark.rpc.short.timeout" + conf.set(Network.RPC_RETRY_WAIT, 0L) + conf.set(Network.RPC_NUM_RETRIES, 1) + val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-abort") + try { + val e = intercept[RpcAbortException] { + val timeout = new RpcTimeout(10.seconds, shortProp) + val abortableRpcFuture = rpcEndpointRef.askAbortable[String]( + "hello", timeout) + + new Thread { + override def run: Unit = { + Thread.sleep(100) + abortableRpcFuture.abort("TestAbort") + } + }.start() + + timeout.awaitResult(abortableRpcFuture.toFuture) + } + // The SparkException cause should be a RpcAbortException with "TestAbort" message + assert(e.isInstanceOf[RpcAbortException]) + assert(e.getMessage.contains("TestAbort")) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + test("onStart and onStop") { val stopLatch = new CountDownLatch(1) val calledMethods = mutable.ArrayBuffer[String]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 112fd31a060e6..101d83314855b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.io.File + import scala.util.Random import org.apache.spark._ @@ -153,4 +155,53 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(error.contains("The coordinator didn't get all barrier sync requests")) assert(error.contains("within 1 second(s)")) } + + test("barrier task killed") { + val conf = new SparkConf() + .set("spark.barrier.sync.timeout", "1") + .set(TEST_NO_STAGE_RETRY, true) + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + + withTempDir { dir => + val killedFlagFile = "barrier.task.killed" + val rdd = sc.makeRDD(Seq(0, 1), 2) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + if (context.partitionId() == 0) { + try { + context.barrier() + } catch { + case _: TaskKilledException => + new File(dir, killedFlagFile).createNewFile() + } + } else { + Thread.sleep(5000) + context.barrier() + } + it + } + + val listener = new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + new Thread { + override def run: Unit = { + Thread.sleep(1000) + sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = false) + } + }.start() + } + } + sc.addSparkListener(listener) + + intercept[SparkException] { + rdd2.collect() + } + + sc.removeSparkListener(listener) + + assert(new File(dir, killedFlagFile).exists(), "Expect barrier task being killed.") + } + } } From 79464bed2f06665cbf5241337a96f37c3cd6c393 Mon Sep 17 00:00:00 2001 From: lihao Date: Tue, 20 Aug 2019 14:37:04 +0800 Subject: [PATCH 113/149] [SPARK-28662][SQL] Create Hive Partitioned Table DDL should fail when partition column type missed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Create Hive Partitioned Table without specifying data type for partition column will success unexpectedly. ```HiveQL // create a hive table partition by b, but the data type of b isn't specified. CREATE TABLE tbl(a int) PARTITIONED BY (b) STORED AS parquet ``` In https://issues.apache.org/jira/browse/SPARK-26435 , PARTITIONED BY clause are extended to support Hive CTAS as following: ```ANTLR // Before (PARTITIONED BY '(' partitionColumns=colTypeList ')' // After (PARTITIONED BY '(' partitionColumns=colTypeList ')'| PARTITIONED BY partitionColumnNames=identifierList) | ``` Create Table Statement like above case will pass the syntax check, and recognized as (PARTITIONED BY partitionColumnNames=identifierList) 。 This PR will check this case in visitCreateHiveTable and throw a exception which contains explicit error message to user. ## How was this patch tested? Added tests. Closes #25390 from lidinghao/hive-ddl-fix. Authored-by: lihao Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/SparkSqlParser.scala | 10 +++++++++- .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 8 ++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) 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 c4edadba278fe..4514675c56f39 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 @@ -985,7 +985,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } else { CreateTable(tableDescWithPartitionColNames, mode, Some(q)) } - case None => CreateTable(tableDesc, mode, None) + case None => + // When creating partitioned table, we must specify data type for the partition columns. + if (Option(ctx.partitionColumnNames).isDefined) { + val errorMessage = "Must specify a data type for each partition column while creating " + + "Hive partitioned table." + operationNotAllowed(errorMessage, ctx) + } + + CreateTable(tableDesc, mode, None) } } 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 a907fcae526c0..cd8e2eaa2b4dc 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 @@ -548,6 +548,14 @@ class HiveDDLSuite assert(e.message == "Found duplicate column(s) in the table definition of `default`.`tbl`: `a`") } + test("create partitioned table without specifying data type for the partition columns") { + val e = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int) PARTITIONED BY (b) STORED AS parquet") + } + assert(e.message.contains("Must specify a data type for each partition column while creating " + + "Hive partitioned table.")) + } + test("add/drop partition with location - managed table") { val tab = "tab_with_partitions" withTempDir { tmpDir => From b37c8d5cea2e31e7821d848e42277f8fb7b68f30 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 20 Aug 2019 00:56:53 -0700 Subject: [PATCH 114/149] [SPARK-28650][SS][DOC] Correct explanation of guarantee for ForeachWriter # What changes were proposed in this pull request? This patch modifies the explanation of guarantee for ForeachWriter as it doesn't guarantee same output for `(partitionId, epochId)`. Refer the description of [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for more details. Spark itself still guarantees same output for same epochId (batch) if the preconditions are met, 1) source is always providing the same input records for same offset request. 2) the query is idempotent in overall (indeterministic calculation like now(), random() can break this). Assuming breaking preconditions as an exceptional case (the preconditions are implicitly required even before), we still can describe the guarantee with `epochId`, though it will be harder to leverage the guarantee: 1) ForeachWriter should implement a feature to track whether all the partitions are written successfully for given `epochId` 2) There's pretty less chance to leverage the fact, as the chance for Spark to successfully write all partitions and fail to checkpoint the batch is small. Credit to zsxwing on discovering the broken guarantee. ## How was this patch tested? This is just a documentation change, both on javadoc and guide doc. Closes #25407 from HeartSaVioR/SPARK-28650. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Shixiong Zhu --- docs/structured-streaming-programming-guide.md | 14 ++++++-------- .../scala/org/apache/spark/sql/ForeachWriter.scala | 13 +++++-------- 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index e07a0e5f33e57..b0d3e162c28c7 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1843,7 +1843,7 @@ Here are the details of all the sinks in Spark. Foreach Sink Append, Update, Complete None - Depends on ForeachWriter implementation + Yes (at-least-once) More details in the
next section @@ -2251,13 +2251,11 @@ When the streaming query is started, Spark calls the function or the object’s - The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM or Python process crashes in the middle. -- **Note:** The partitionId and epochId in the open() method can be used to deduplicate generated data - when failures cause reprocessing of some input data. This depends on the execution mode of the query. - If the streaming query is being executed in the micro-batch mode, then every partition represented - by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data. - Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit - data and achieve exactly-once guarantees. However, if the streaming query is being executed - in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. +- **Note:** Spark does not guarantee same output for (partitionId, epochId), so deduplication + cannot be achieved with (partitionId, epochId). e.g. source provides different number of + partitions for some reasons, Spark optimization changes number of partitions, etc. + See [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for more details. + If you need deduplication on output, try out `foreachBatch` instead. #### Triggers The trigger settings of a streaming query define the timing of streaming data processing, whether diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index 5c0fe798b1044..a0b0a34a01073 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -50,14 +50,11 @@ import org.apache.spark.annotation.Evolving * * Important points to note: *