From 0c68578fa9d03002a2d1708762c3010fd28430c5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 15 Nov 2019 13:33:30 +0800 Subject: [PATCH 01/83] [SPARK-29888][SQL] new interval string parser shall handle numeric with only fractional part ### What changes were proposed in this pull request? Current string to interval cast logic does not support i.e. cast('.111 second' as interval) which will fail in SIGN state and return null, actually, it is 00:00:00.111. ```scala -- !query 63 select interval '.111 seconds' -- !query 63 schema struct<0.111 seconds:interval> -- !query 63 output 0.111 seconds -- !query 64 select cast('.111 seconds' as interval) -- !query 64 schema struct -- !query 64 output NULL ```` ### Why are the changes needed? bug fix. ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26514 from yaooqinn/SPARK-29888. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/IntervalUtils.scala | 22 ++++++++++++++----- .../catalyst/util/IntervalUtilsSuite.scala | 1 + 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 882c1d85267e4..991312bff30aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -496,6 +496,17 @@ object IntervalUtils { state = TRIM_BEFORE_SIGN case TRIM_BEFORE_SIGN => trimToNextState(b, SIGN) case SIGN => + currentValue = 0 + fraction = 0 + // We preset next state from SIGN to TRIM_BEFORE_VALUE. If we meet '.' in the SIGN state, + // it means that the interval value we deal with here is a numeric with only fractional + // part, such as '.11 second', which can be parsed to 0.11 seconds. In this case, we need + // to reset next state to `VALUE_FRACTIONAL_PART` to go parse the fraction part of the + // interval value. + state = TRIM_BEFORE_VALUE + // We preset the scale to an invalid value to track fraction presence in the UNIT_BEGIN + // state. If we meet '.', the scale become valid for the VALUE_FRACTIONAL_PART state. + fractionScale = -1 b match { case '-' => isNegative = true @@ -505,14 +516,13 @@ object IntervalUtils { i += 1 case _ if '0' <= b && b <= '9' => isNegative = false + case '.' => + isNegative = false + fractionScale = (NANOS_PER_SECOND / 10).toInt + i += 1 + state = VALUE_FRACTIONAL_PART case _ => return null } - currentValue = 0 - fraction = 0 - // Sets the scale to an invalid value to track fraction presence - // in the BEGIN_UNIT_NAME state - fractionScale = -1 - state = TRIM_BEFORE_VALUE case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) case VALUE => b match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 8c84eb107cd30..f919bd1644871 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -106,6 +106,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromString("-1.5 seconds", new CalendarInterval(0, 0, -1500000)) // truncate nanoseconds to microseconds checkFromString("0.999999999 seconds", new CalendarInterval(0, 0, 999999)) + checkFromString(".999999999 seconds", new CalendarInterval(0, 0, 999999)) checkFromInvalidString("0.123456789123 seconds", "Error parsing interval string") } From 4f10e54ba385daa37598efa49dbfb536a7726dbc Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 15 Nov 2019 15:49:24 +0800 Subject: [PATCH 02/83] [SPARK-29655][SQL] Read bucketed tables obeys spark.sql.shuffle.partitions ### What changes were proposed in this pull request? In order to avoid frequently changing the value of `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions`, we usually set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` much larger than `spark.sql.shuffle.partitions` after enabling adaptive execution, which causes some bucket map join lose efficacy and add more `ShuffleExchange`. How to reproduce: ```scala val bucketedTableName = "bucketed_table" spark.range(10000).write.bucketBy(500, "id").sortBy("id").mode(org.apache.spark.sql.SaveMode.Overwrite).saveAsTable(bucketedTableName) val bucketedTable = spark.table(bucketedTableName) val df = spark.range(8) spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1) // Spark 2.4. spark.sql.adaptive.enabled=false // We set spark.sql.shuffle.partitions <= 500 every time based on our data in this case. spark.conf.set("spark.sql.shuffle.partitions", 500) bucketedTable.join(df, "id").explain() // Since 3.0. We enabled adaptive execution and set spark.sql.adaptive.shuffle.maxNumPostShufflePartitions to a larger values to fit more cases. spark.conf.set("spark.sql.adaptive.enabled", true) spark.conf.set("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions", 1000) bucketedTable.join(df, "id").explain() ``` ``` scala> bucketedTable.join(df, "id").explain() == Physical Plan == *(4) Project [id#5L] +- *(4) SortMergeJoin [id#5L], [id#7L], Inner :- *(1) Sort [id#5L ASC NULLS FIRST], false, 0 : +- *(1) Project [id#5L] : +- *(1) Filter isnotnull(id#5L) : +- *(1) ColumnarToRow : +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct, SelectedBucketsCount: 500 out of 500 +- *(3) Sort [id#7L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#7L, 500), true, [id=#49] +- *(2) Range (0, 8, step=1, splits=16) ``` vs ``` scala> bucketedTable.join(df, "id").explain() == Physical Plan == AdaptiveSparkPlan(isFinalPlan=false) +- Project [id#5L] +- SortMergeJoin [id#5L], [id#7L], Inner :- Sort [id#5L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#5L, 1000), true, [id=#93] : +- Project [id#5L] : +- Filter isnotnull(id#5L) : +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct, SelectedBucketsCount: 500 out of 500 +- Sort [id#7L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#7L, 1000), true, [id=#92] +- Range (0, 8, step=1, splits=16) ``` This PR makes read bucketed tables always obeys `spark.sql.shuffle.partitions` even enabling adaptive execution and set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` to avoid add more `ShuffleExchange`. ### Why are the changes needed? Do not degrade performance after enabling adaptive execution. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #26409 from wangyum/SPARK-29655. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../exchange/EnsureRequirements.scala | 19 +++++++++++- .../ReduceNumShufflePartitionsSuite.scala | 3 +- .../spark/sql/sources/BucketedReadSuite.scala | 31 +++++++++++++++++-- 3 files changed, 49 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index c56a5c015f32d..866b382a1d808 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -83,7 +83,24 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { numPartitionsSet.headOption } - val targetNumPartitions = requiredNumPartitions.getOrElse(childrenNumPartitions.max) + // If there are non-shuffle children that satisfy the required distribution, we have + // some tradeoffs when picking the expected number of shuffle partitions: + // 1. We should avoid shuffling these children. + // 2. We should have a reasonable parallelism. + val nonShuffleChildrenNumPartitions = + childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) + .map(_.outputPartitioning.numPartitions) + val expectedChildrenNumPartitions = if (nonShuffleChildrenNumPartitions.nonEmpty) { + // Here we pick the max number of partitions among these non-shuffle children as the + // expected number of shuffle partitions. However, if it's smaller than + // `conf.numShufflePartitions`, we pick `conf.numShufflePartitions` as the + // expected number of shuffle partitions. + math.max(nonShuffleChildrenNumPartitions.max, conf.numShufflePartitions) + } else { + childrenNumPartitions.max + } + + val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, distribution), index) if childrenIndexes.contains(index) => 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 4d408cd8ebd70..21ec1ac9bda08 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 @@ -274,6 +274,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA .setMaster("local[*]") .setAppName("test") .set(UI_ENABLED, false) + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") @@ -507,7 +508,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA join, expectedAnswer.collect()) - // Then, let's make sure we do not reduce number of ppst shuffle partitions. + // Then, let's make sure we do not reduce number of post shuffle partitions. val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { 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 7043b6d396977..a585f215ad681 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.{DataSourceScanExec, SortExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.BucketingUtils import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -382,8 +383,16 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { joined.sort("bucketed_table1.k", "bucketed_table2.k"), df1.join(df2, joinCondition(df1, df2), joinType).sort("df1.k", "df2.k")) - assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) - val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] + val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { + val executedPlan = + joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert(executedPlan.isInstanceOf[SortMergeJoinExec]) + executedPlan.asInstanceOf[SortMergeJoinExec] + } else { + val executedPlan = joined.queryExecution.executedPlan + assert(executedPlan.isInstanceOf[SortMergeJoinExec]) + executedPlan.asInstanceOf[SortMergeJoinExec] + } // check existence of shuffle assert( @@ -795,4 +804,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) + Seq(false, true).foreach { enableAdaptive => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> s"$enableAdaptive") { + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(None, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + } + } + } } From ee4784bf26421b830f02c6d65faae4e546c622a8 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 15 Nov 2019 08:12:41 -0600 Subject: [PATCH 03/83] [SPARK-26499][SQL][FOLLOW-UP] Replace `update` with `setByte` for ByteType in JdbcUtils.makeGetter ### What changes were proposed in this pull request? This is a follow-up pr to fix the code coming from #23400; it replaces `update` with `setByte` for ByteType in `JdbcUtils.makeGetter`. ### Why are the changes needed? For better code. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26532 from maropu/SPARK-26499-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index d4c7f005a16df..e3baa2a028da2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -445,7 +445,7 @@ object JdbcUtils extends Logging { case ByteType => (rs: ResultSet, row: InternalRow, pos: Int) => - row.update(pos, rs.getByte(pos + 1)) + row.setByte(pos, rs.getByte(pos + 1)) case StringType => (rs: ResultSet, row: InternalRow, pos: Int) => From 15218898cdc540420d6a6c957e5040f78e75cc61 Mon Sep 17 00:00:00 2001 From: shahid Date: Fri, 15 Nov 2019 08:20:10 -0600 Subject: [PATCH 04/83] [SPARK-29902][DOC][MINOR] Add listener event queue capacity configuration to documentation ### What changes were proposed in this pull request? Add listener event queue capacity configuration to documentation ### Why are the changes needed? We some time see many event drops happening in eventLog listener queue. So, instead of increasing all the queues size, using this config we just need to increase eventLog queue capacity. ``` scala> sc.parallelize(1 to 100000, 100000).count() [Stage 0:=================================================>(98299 + 4) / 100000]19/11/14 20:56:35 ERROR AsyncEventQueue: Dropping event from queue eventLog. This likely means one of the listeners is too slow and cannot keep up with the rate at which tasks are being started by the scheduler. 19/11/14 20:56:35 WARN AsyncEventQueue: Dropped 1 events from eventLog since the application started. ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #26529 from shahidki31/master1. Authored-by: shahid Signed-off-by: Sean Owen --- docs/configuration.md | 45 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 97ea1fb4ba041..0c7cc6022eb09 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1857,6 +1857,51 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. + + spark.scheduler.listenerbus.eventqueue.shared.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for shared event queue in Spark listener bus, which hold events for external listener(s) + that register to the listener bus. Consider increasing value, if the listener events corresponding + to shared queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.appStatus.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for appStatus event queue, which hold events for internal application status listeners. + Consider increasing value, if the listener events corresponding to appStatus queue are dropped. + Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.executorManagement.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for executorManagement event queue in Spark listener bus, which hold events for internal + executor management listeners. Consider increasing value if the listener events corresponding to + executorManagement queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.eventLog.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for eventLog queue in Spark listener bus, which hold events for Event logging listeners + that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue + are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.streams.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for streams queue in Spark listener bus, which hold events for internal streaming listener. + Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing + this value may result in the driver using more memory. + + spark.scheduler.blacklist.unschedulableTaskSetTimeout 120s From 848bdfa218c5da55d7fbc0cf82866e6b1c4a09e6 Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Fri, 15 Nov 2019 14:25:33 -0800 Subject: [PATCH 05/83] [SPARK-29829][SQL] SHOW TABLE EXTENDED should do multi-catalog resolution MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add ShowTableStatement and make SHOW TABLE EXTENDED go through the same catalog/table resolution framework of v2 commands. We don’t have this methods in the catalog to implement an V2 command - catalog.getPartition - catalog.getTempViewOrPermanentTableMetadata ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing ```sql USE my_catalog DESC t // success and describe the table t from my_catalog SHOW TABLE EXTENDED FROM LIKE 't' // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running SHOW TABLE EXTENDED Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26540 from planga82/feature/SPARK-29481_ShowTableExtended. Authored-by: Pablo Langa Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 10 +++++ .../catalyst/plans/logical/statements.scala | 9 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 25 +++++++++++ .../analysis/ResolveSessionCatalog.scala | 9 ++++ .../spark/sql/execution/SparkSqlParser.scala | 17 ------- .../sql/connector/DataSourceV2SQLSuite.scala | 44 +++++++++++++++++++ 7 files changed, 98 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 49fba6b7f35df..7ca33a8ecc0c4 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -187,7 +187,7 @@ statement statement #explain | SHOW TABLES ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showTables - | SHOW TABLE EXTENDED ((FROM | IN) db=errorCapturingIdentifier)? + | SHOW TABLE EXTENDED ((FROM | IN) namespace=multipartIdentifier)? LIKE pattern=STRING partitionSpec? #showTable | SHOW TBLPROPERTIES table=multipartIdentifier ('(' key=tablePropertyKey ')')? #showTblProperties diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c623b5c4d36a5..56025ccbf0cd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2719,6 +2719,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.pattern).map(string)) } + /** + * Create a [[ShowTableStatement]] command. + */ + override def visitShowTable(ctx: ShowTableContext): LogicalPlan = withOrigin(ctx) { + ShowTableStatement( + Option(ctx.namespace).map(visitMultipartIdentifier), + string(ctx.pattern), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + /** * Parse new column info from ADD COLUMN into a QualifiedColType. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index ec373d95fad88..03d02a2b13cd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -313,6 +313,15 @@ case class InsertIntoStatement( case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) extends ParsedStatement +/** + * A SHOW TABLE EXTENDED statement, as parsed from SQL. + */ +case class ShowTableStatement( + namespace: Option[Seq[String]], + pattern: String, + partitionSpec: Option[TablePartitionSpec]) + extends ParsedStatement + /** * A CREATE NAMESPACE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 94171feba2ac7..5fa25300793a4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1022,6 +1022,31 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("show table extended") { + comparePlans( + parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"), + ShowTableStatement(None, "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED FROM testcat.ns1.ns2 LIKE '*test*'"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED IN testcat.ns1.ns2 LIKE '*test*'"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED LIKE '*test*' PARTITION(ds='2008-04-09', hr=11)"), + ShowTableStatement(None, "*test*", Some(Map("ds" -> "2008-04-09", "hr" -> "11")))) + comparePlans( + parsePlan("SHOW TABLE EXTENDED FROM testcat.ns1.ns2 LIKE '*test*' " + + "PARTITION(ds='2008-04-09')"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", + Some(Map("ds" -> "2008-04-09")))) + comparePlans( + parsePlan("SHOW TABLE EXTENDED IN testcat.ns1.ns2 LIKE '*test*' " + + "PARTITION(ds='2008-04-09')"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", + Some(Map("ds" -> "2008-04-09")))) + } + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { val expected = CreateNamespaceStatement( Seq("a", "b", "c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 340e09ae66adb..7adcd7836e952 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -301,6 +301,15 @@ class ResolveSessionCatalog( case ShowTablesStatement(None, pattern) if isSessionCatalog(currentCatalog) => ShowTablesCommand(None, pattern) + case ShowTableStatement(namespace, pattern, partitionsSpec) => + val db = namespace match { + case Some(namespace) if namespace.length != 1 => + throw new AnalysisException( + s"The database name is not valid: ${namespace.quoted}") + case _ => namespace.map(_.head) + } + ShowTablesCommand(db, Some(pattern), true, partitionsSpec) + case AnalyzeTableStatement(tableName, partitionSpec, noScan) => val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") if (partitionSpec.isEmpty) { 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 b1271ad870565..ab5b1126dc536 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 @@ -89,23 +89,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ResetCommand } - /** - * Create a [[ShowTablesCommand]] logical plan. - * Example SQL : - * {{{ - * SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' - * [PARTITION(partition_spec)]; - * }}} - */ - override def visitShowTable(ctx: ShowTableContext): LogicalPlan = withOrigin(ctx) { - val partitionSpec = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ShowTablesCommand( - Option(ctx.db).map(_.getText), - Option(ctx.pattern).map(string), - isExtended = true, - partitionSpec = partitionSpec) - } - /** * Create a [[RefreshResource]] logical plan. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 17f6e51f8454c..19b2ad601328c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -766,6 +766,50 @@ class DataSourceV2SQLSuite assert(expected === df.collect()) } + test("SHOW TABLE EXTENDED not valid v1 database") { + def testV1CommandNamespace(sqlCommand: String, namespace: String): Unit = { + val e = intercept[AnalysisException] { + sql(sqlCommand) + } + assert(e.message.contains(s"The database name is not valid: ${namespace}")) + } + + val namespace = "testcat.ns1.ns2" + val table = "tbl" + withTable(s"$namespace.$table") { + sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " + + s"USING foo PARTITIONED BY (id)") + + testV1CommandNamespace(s"SHOW TABLE EXTENDED FROM $namespace LIKE 'tb*'", + namespace) + testV1CommandNamespace(s"SHOW TABLE EXTENDED IN $namespace LIKE 'tb*'", + namespace) + testV1CommandNamespace("SHOW TABLE EXTENDED " + + s"FROM $namespace LIKE 'tb*' PARTITION(id=1)", + namespace) + testV1CommandNamespace("SHOW TABLE EXTENDED " + + s"IN $namespace LIKE 'tb*' PARTITION(id=1)", + namespace) + } + } + + test("SHOW TABLE EXTENDED valid v1") { + val expected = Seq(Row("", "source", true), Row("", "source2", true)) + val schema = new StructType() + .add("database", StringType, nullable = false) + .add("tableName", StringType, nullable = false) + .add("isTemporary", BooleanType, nullable = false) + .add("information", StringType, nullable = false) + + val df = sql("SHOW TABLE EXTENDED FROM default LIKE '*source*'") + val result = df.collect() + val resultWithoutInfo = result.map{ case Row(db, table, temp, _) => Row(db, table, temp)} + + assert(df.schema === schema) + assert(resultWithoutInfo === expected) + result.foreach{ case Row(_, _, _, info: String) => assert(info.nonEmpty)} + } + test("CreateNameSpace: basic tests") { // Session catalog is used. withNamespace("ns") { From c0507e0f7530032b26db6ac1611a6a53b9802d5a Mon Sep 17 00:00:00 2001 From: ulysses Date: Fri, 15 Nov 2019 16:17:24 -0800 Subject: [PATCH 06/83] [SPARK-29833][YARN] Add FileNotFoundException check for spark.yarn.jars ### What changes were proposed in this pull request? When set `spark.yarn.jars=/xxx/xxx` which is just a no schema path, spark will throw a NullPointerException. The reason is hdfs will return null if pathFs.globStatus(path) is not exist, and spark just use `pathFs.globStatus(path).filter(_.isFile())` without check it. ### Why are the changes needed? Avoid NullPointerException. ### Does this PR introduce any user-facing change? Yes. User will get a FileNotFoundException instead NullPointerException when `spark.yarn.jars` does not have schema and not exists. ### How was this patch tested? Add UT. Closes #26462 from ulysses-you/check-yarn-jars-path-exist. Authored-by: ulysses Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/yarn/Client.scala | 6 +++++- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 14 +++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) 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 a1b5d53b91416..696afaacb0e79 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 @@ -538,7 +538,11 @@ private[spark] class Client( if (!Utils.isLocalUri(jar)) { val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf) val pathFs = FileSystem.get(path.toUri(), hadoopConf) - pathFs.globStatus(path).filter(_.isFile()).foreach { entry => + val fss = pathFs.globStatus(path) + if (fss == null) { + throw new FileNotFoundException(s"Path ${path.toString} does not exist") + } + fss.filter(_.isFile()).foreach { entry => val uri = entry.getPath().toUri() statCache.update(uri, entry) distribute(uri.toString(), targetDir = Some(LOCALIZED_LIB_DIR)) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index ba7c2dd8a1cdf..7cce908cd5fb7 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{File, FileInputStream, FileNotFoundException, FileOutputStream} import java.net.URI import java.util.Properties @@ -473,6 +473,18 @@ class ClientSuite extends SparkFunSuite with Matchers { assert(allResourceInfo.get(yarnMadeupResource).get === 5) } + test("test yarn jars path not exists") { + withTempDir { dir => + val conf = new SparkConf().set(SPARK_JARS, Seq(dir.getAbsolutePath + "/test")) + val client = new Client(new ClientArguments(Array()), conf, null) + withTempDir { distDir => + intercept[FileNotFoundException] { + client.prepareLocalResources(new Path(distDir.getAbsolutePath), Nil) + } + } + } + } + private val matching = Seq( ("files URI match test1", "file:///file1", "file:///file2"), ("files URI match test2", "file:///c:file1", "file://c:file2"), From 7720781695d47fe0375f6e1150f6981b886686bd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 15 Nov 2019 18:37:33 -0800 Subject: [PATCH 07/83] [SPARK-29127][SQL][PYTHON] Add a clue for Python related version information in integrated UDF tests ### What changes were proposed in this pull request? This PR proposes to show Python, pandas and PyArrow versions in integrated UDF tests as a clue so when the test cases fail, it show the related version information. I think we don't really need this kind of version information in the test case name for now since I intend that integrated SQL test cases do not target to test different combinations of Python, Pandas and PyArrow. ### Why are the changes needed? To make debug easier. ### Does this PR introduce any user-facing change? It will change test name to include related Python, pandas and PyArrow versions. ### How was this patch tested? Manually tested: ``` [info] - udf/postgreSQL/udf-case.sql - Scala UDF *** FAILED *** (8 seconds, 229 milliseconds) [info] udf/postgreSQL/udf-case.sql - Scala UDF ... [info] - udf/postgreSQL/udf-case.sql - Regular Python UDF *** FAILED *** (6 seconds, 298 milliseconds) [info] udf/postgreSQL/udf-case.sql - Regular Python UDF [info] Python: 3.7 ... [info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** (6 seconds, 376 milliseconds) [info] udf/postgreSQL/udf-case.sql - Scalar Pandas UDF [info] Python: 3.7 Pandas: 0.25.3 PyArrow: 0.14.0 ``` Closes #26538 from HyukjinKwon/investigate-flaky-test. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../spark/sql/IntegratedUDFTestUtils.scala | 20 ++++++++++++++++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 16 ++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index d62fe961117a9..3f6fe6eb62020 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -122,7 +122,7 @@ object IntegratedUDFTestUtils extends SQLHelper { true }.getOrElse(false) - private lazy val pythonVer = if (isPythonAvailable) { + lazy val pythonVer: String = if (isPythonAvailable) { Process( Seq(pythonExec, "-c", "import sys; print('%d.%d' % sys.version_info[:2])"), None, @@ -131,6 +131,24 @@ object IntegratedUDFTestUtils extends SQLHelper { throw new RuntimeException(s"Python executable [$pythonExec] is unavailable.") } + lazy val pandasVer: String = if (isPandasAvailable) { + Process( + Seq(pythonExec, "-c", "import pandas; print(pandas.__version__)"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!!.trim() + } else { + throw new RuntimeException("Pandas is unavailable.") + } + + lazy val pyarrowVer: String = if (isPyArrowAvailable) { + Process( + Seq(pythonExec, "-c", "import pyarrow; print(pyarrow.__version__)"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!!.trim() + } else { + throw new RuntimeException("PyArrow is unavailable.") + } + // Dynamically pickles and reads the Python instance into JVM side in order to mimic // Python native function within Python UDF. private lazy val pythonFunc: Array[Byte] = if (shouldTestPythonUDFs) { 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 2ce50dd73a13d..ffea03474ac4a 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 @@ -384,7 +384,21 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // This is a temporary workaround for SPARK-28894. The test names are truncated after // the last dot due to a bug in SBT. This makes easier to debug via Jenkins test result // report. See SPARK-28894. - withClue(s"${testCase.name}${System.lineSeparator()}") { + // See also SPARK-29127. It is difficult to see the version information in the failed test + // cases so the version information related to Python was also added. + val clue = testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && shouldTestPythonUDFs => + s"${testCase.name}${System.lineSeparator()}Python: $pythonVer${System.lineSeparator()}" + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && shouldTestScalarPandasUDFs => + s"${testCase.name}${System.lineSeparator()}" + + s"Python: $pythonVer Pandas: $pandasVer PyArrow: $pyarrowVer${System.lineSeparator()}" + case _ => + s"${testCase.name}${System.lineSeparator()}" + } + + withClue(clue) { // Read back the golden file. val expectedOutputs: Seq[QueryOutput] = { val goldenOutput = fileToString(new File(testCase.resultFile)) From 16e7195299d864b9e98ed17a9747d53c6a001024 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Fri, 15 Nov 2019 18:50:42 -0800 Subject: [PATCH 08/83] [SPARK-29834][SQL] DESC DATABASE should look up catalog like v2 commands ### What changes were proposed in this pull request? Add DescribeNamespaceStatement, DescribeNamespace and DescribeNamespaceExec to make "DESC DATABASE" look up catalog like v2 commands. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "DESC NAMESPACE" whose function is same as "DESC DATABASE" and "DESC SCHEMA". ### How was this patch tested? New unit test Closes #26513 from fuwhu/SPARK-29834. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 + .../sql/catalyst/parser/AstBuilder.scala | 15 +++++ .../catalyst/plans/logical/statements.scala | 7 ++ .../catalyst/plans/logical/v2Commands.scala | 17 ++++- .../sql/catalyst/parser/DDLParserSuite.scala | 7 ++ .../analysis/ResolveSessionCatalog.scala | 7 ++ .../spark/sql/execution/SparkSqlParser.scala | 12 ---- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../v2/DescribeNamespaceExec.scala | 66 +++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 20 +++++- .../execution/command/DDLParserSuite.scala | 19 ------ 12 files changed, 146 insertions(+), 35 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7ca33a8ecc0c4..8a6bd5ed27360 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -199,7 +199,8 @@ statement | SHOW CREATE TABLE multipartIdentifier #showCreateTable | SHOW CURRENT NAMESPACE #showCurrentNamespace | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction - | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase + | (DESC | DESCRIBE) (database | NAMESPACE) EXTENDED? + multipartIdentifier #describeNamespace | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index f1a8e5bfda4a9..22caf33b50ce6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -172,6 +172,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) => DropNamespace(catalog, nameParts, ifExists, cascade) + case DescribeNamespaceStatement(NonSessionCatalog(catalog, nameParts), extended) => + DescribeNamespace(catalog, nameParts, extended) + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 56025ccbf0cd8..73233f934bedc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2541,6 +2541,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.pattern).map(string)) } + /** + * Create a [[DescribeNamespaceStatement]]. + * + * For example: + * {{{ + * DESCRIBE (DATABASE|SCHEMA|NAMESPACE) [EXTENDED] database; + * }}} + */ + override def visitDescribeNamespace(ctx: DescribeNamespaceContext): LogicalPlan = + withOrigin(ctx) { + DescribeNamespaceStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.EXTENDED != null) + } + /** * Create a table, returning a [[CreateTableStatement]] logical plan. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 03d02a2b13cd7..903f5cd5912c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -269,6 +269,13 @@ case class DescribeTableStatement( partitionSpec: TablePartitionSpec, isExtended: Boolean) extends ParsedStatement +/** + * A DESCRIBE NAMESPACE statement, as parsed from SQL. + */ +case class DescribeNamespaceStatement( + namespace: Seq[String], + extended: Boolean) extends ParsedStatement + /** * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 7d8e9a0c18f65..3179bab8c2f7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.{DataType, StringType, StructType} +import org.apache.spark.sql.types.{DataType, MetadataBuilder, StringType, StructType} /** * Base trait for DataSourceV2 write commands @@ -255,6 +255,21 @@ case class DropNamespace( ifExists: Boolean, cascade: Boolean) extends Command +/** + * The logical plan of the DESCRIBE NAMESPACE command that works for v2 catalogs. + */ +case class DescribeNamespace( + catalog: CatalogPlugin, + namespace: Seq[String], + extended: Boolean) extends Command { + + override def output: Seq[Attribute] = Seq( + AttributeReference("name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("value", StringType, nullable = true, + new MetadataBuilder().putString("comment", "value of the column").build())()) +} + /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 5fa25300793a4..b998127ebe02a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -653,6 +653,13 @@ class DDLParserSuite extends AnalysisTest { "DESC TABLE COLUMN for a specific partition is not supported")) } + test("describe database") { + val sql1 = "DESCRIBE DATABASE EXTENDED a.b" + val sql2 = "DESCRIBE DATABASE a.b" + comparePlans(parsePlan(sql1), DescribeNamespaceStatement(Seq("a", "b"), extended = true)) + comparePlans(parsePlan(sql2), DescribeNamespaceStatement(Seq("a", "b"), extended = false)) + } + test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 7adcd7836e952..3913f889eb128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -158,6 +158,13 @@ class ResolveSessionCatalog( case AlterViewUnsetPropertiesStatement(SessionCatalog(catalog, tableName), keys, ifExists) => AlterTableUnsetPropertiesCommand(tableName.asTableIdentifier, keys, ifExists, isView = true) + case d @ DescribeNamespaceStatement(SessionCatalog(_, nameParts), _) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + DescribeDatabaseCommand(nameParts.head, d.extended) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { 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 ab5b1126dc536..bfea908d106df 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 @@ -258,18 +258,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ) } - /** - * Create a [[DescribeDatabaseCommand]] command. - * - * For example: - * {{{ - * DESCRIBE DATABASE [EXTENDED] database; - * }}} - */ - override def visitDescribeDatabase(ctx: DescribeDatabaseContext): LogicalPlan = withOrigin(ctx) { - DescribeDatabaseCommand(ctx.db.getText, ctx.EXTENDED != null) - } - /** * Create a plan for a DESCRIBE FUNCTION command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 01ff4a9303e98..a04bceb18b9b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -192,6 +192,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } + case desc @ DescribeNamespace(catalog, namespace, extended) => + DescribeNamespaceExec(desc.output, catalog, namespace, extended) :: Nil + case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => DescribeTableExec(desc.output, table, isExtended) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala new file mode 100644 index 0000000000000..5c20e5ae08383 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.COMMENT_TABLE_PROP +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.LOCATION_TABLE_PROP +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.RESERVED_PROPERTIES +import org.apache.spark.sql.types.StructType + +/** + * Physical plan node for describing a namespace. + */ +case class DescribeNamespaceExec( + output: Seq[Attribute], + catalog: CatalogPlugin, + namespace: Seq[String], + isExtended: Boolean) extends V2CommandExec { + + private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() + + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val rows = new ArrayBuffer[InternalRow]() + val nsCatalog = catalog.asNamespaceCatalog + val ns = namespace.toArray + val metadata = nsCatalog.loadNamespaceMetadata(ns) + + rows += toCatalystRow("Namespace Name", ns.last) + rows += toCatalystRow("Description", metadata.get(COMMENT_TABLE_PROP)) + rows += toCatalystRow("Location", metadata.get(LOCATION_TABLE_PROP)) + if (isExtended) { + val properties = metadata.asScala.toSeq.filter(p => !RESERVED_PROPERTIES.contains(p._1)) + if (properties.nonEmpty) { + rows += toCatalystRow("Properties", properties.mkString("(", ",", ")")) + } + } + rows + } + + private def toCatalystRow(strs: String*): InternalRow = { + encoder.toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 19b2ad601328c..095bcff31ccd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -834,7 +834,6 @@ class DataSourceV2SQLSuite assert(catalogPath.equals(catalogPath)) } } - // TODO: Add tests for validating namespace metadata when DESCRIBE NAMESPACE is available. } test("CreateNameSpace: test handling of 'IF NOT EXIST'") { @@ -915,6 +914,25 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("Namespace 'ns1' not found")) } + test("DescribeNamespace using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test'") + val descriptionDf = sql("DESCRIBE NAMESPACE testcat.ns1.ns2") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === + Seq( + ("name", StringType), + ("value", StringType) + )) + val description = descriptionDf.collect() + assert(description === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") 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 d98f2ca62972c..b2185f8559f36 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 @@ -108,25 +108,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed1, expected1) } - test("describe database") { - // DESCRIBE DATABASE [EXTENDED] db_name; - val sql1 = "DESCRIBE DATABASE EXTENDED db_name" - val sql2 = "DESCRIBE DATABASE db_name" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - - val expected1 = DescribeDatabaseCommand( - "db_name", - extended = true) - val expected2 = DescribeDatabaseCommand( - "db_name", - extended = false) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("create function") { val sql1 = """ From 6d6b233791a0a18713a3234f10ba234d92083d68 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 15 Nov 2019 18:54:02 -0800 Subject: [PATCH 09/83] [SPARK-29343][SQL][FOLLOW-UP] Remove floating-point Sum/Average/CentralMomentAgg from order-insensitive aggregates ### What changes were proposed in this pull request? This pr is to remove floating-point `Sum/Average/CentralMomentAgg` from order-insensitive aggregates in `EliminateSorts`. This pr comes from the gatorsmile suggestion: https://github.com/apache/spark/pull/26011#discussion_r344583899 ### Why are the changes needed? Bug fix. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added tests in `SubquerySuite`. Closes #26534 from maropu/SPARK-29343-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/Optimizer.scala | 11 +++++------ .../org/apache/spark/sql/SubquerySuite.scala | 17 +++++++++++++++++ 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b78bdf082f333..473f846c9313b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1002,12 +1002,11 @@ object EliminateSorts extends Rule[LogicalPlan] { private def isOrderIrrelevantAggs(aggs: Seq[NamedExpression]): Boolean = { def isOrderIrrelevantAggFunction(func: AggregateFunction): Boolean = func match { - case _: Sum => true - case _: Min => true - case _: Max => true - case _: Count => true - case _: Average => true - case _: CentralMomentAgg => true + case _: Min | _: Max | _: Count => true + // Arithmetic operations for floating-point values are order-sensitive + // (they are not associative). + case _: Sum | _: Average | _: CentralMomentAgg => + !Seq(FloatType, DoubleType).exists(_.sameType(func.children.head.dataType)) case _ => false } 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 c117ee7818c01..5020c1047f8dd 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 @@ -1271,6 +1271,23 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } + test("Cannot remove sort for floating-point order-sensitive aggregates from subquery") { + Seq("float", "double").foreach { typeName => + Seq("SUM", "AVG", "KURTOSIS", "SKEWNESS", "STDDEV_POP", "STDDEV_SAMP", + "VAR_POP", "VAR_SAMP").foreach { aggName => + val query = + s""" + |SELECT k, $aggName(v) FROM ( + | SELECT k, v + | FROM VALUES (1, $typeName(2.0)), (2, $typeName(1.0)) t(k, v) + | ORDER BY v) + |GROUP BY k + """.stripMargin + assert(getNumSortsInQuery(query) == 1) + } + } + } + test("SPARK-25482: Forbid pushdown to datasources of filters containing subqueries") { withTempView("t1", "t2") { sql("create temporary view t1(a int) using parquet") From 1112fc6029f48918cdd05d0736e02f446262f398 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 15 Nov 2019 21:44:39 -0800 Subject: [PATCH 10/83] [SPARK-29867][ML][PYTHON] Add __repr__ in Python ML Models ### What changes were proposed in this pull request? Add ```__repr__``` in Python ML Models ### Why are the changes needed? In Python ML Models, some of them have ```__repr__```, others don't. In the doctest, when calling Model.setXXX, some of the Models print out the xxxModel... correctly, some of them can't because of lacking the ```__repr__``` method. For example: ``` >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10) >>> model = gm.fit(df) >>> model.setPredictionCol("newPrediction") GaussianMixture... ``` After the change, the above code will become the following: ``` >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10) >>> model = gm.fit(df) >>> model.setPredictionCol("newPrediction") GaussianMixtureModel... ``` ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? doctest Closes #26489 from huaxingao/spark-29876. Authored-by: Huaxin Gao Signed-off-by: Dongjoon Hyun --- python/pyspark/ml/classification.py | 11 ++++------- python/pyspark/ml/clustering.py | 8 +++++--- python/pyspark/ml/feature.py | 30 +++++++++++++++++++++++------ python/pyspark/ml/fpm.py | 2 +- python/pyspark/ml/recommendation.py | 2 ++ python/pyspark/ml/regression.py | 10 +++++----- python/pyspark/ml/tree.py | 6 ------ python/pyspark/ml/wrapper.py | 3 +++ 8 files changed, 44 insertions(+), 28 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index f9465bffc9a1a..66b8ffd7bc8d8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -192,11 +192,11 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable 0.01 >>> model = svm.fit(df) >>> model.setPredictionCol("newPrediction") - LinearSVC... + LinearSVCModel... >>> model.getPredictionCol() 'newPrediction' >>> model.setThreshold(0.5) - LinearSVC... + LinearSVCModel... >>> model.getThreshold() 0.5 >>> model.coefficients @@ -812,9 +812,6 @@ def evaluate(self, dataset): java_blr_summary = self._call_java("evaluate", dataset) return BinaryLogisticRegressionSummary(java_blr_summary) - def __repr__(self): - return self._call_java("toString") - class LogisticRegressionSummary(JavaWrapper): """ @@ -1921,7 +1918,7 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial", weightCol="weight") >>> model = nb.fit(df) >>> model.setFeaturesCol("features") - NaiveBayes_... + NaiveBayesModel... >>> model.getSmoothing() 1.0 >>> model.pi @@ -2114,7 +2111,7 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer 100 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") - MultilayerPerceptronClassifier... + MultilayerPerceptronClassificationModel... >>> model.layers [2, 2, 2] >>> model.weights.size diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 39cc62670ae88..5aab7a3f5077b 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -234,7 +234,7 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav >>> model.getFeaturesCol() 'features' >>> model.setPredictionCol("newPrediction") - GaussianMixture... + GaussianMixtureModel... >>> model.predict(df.head().features) 2 >>> model.predictProbability(df.head().features) @@ -532,7 +532,7 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): >>> model.getDistanceMeasure() 'euclidean' >>> model.setPredictionCol("newPrediction") - KMeans... + KMeansModel... >>> model.predict(df.head().features) 0 >>> centers = model.clusterCenters() @@ -794,7 +794,7 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav >>> model.getMaxIter() 20 >>> model.setPredictionCol("newPrediction") - BisectingKMeans... + BisectingKMeansModel... >>> model.predict(df.head().features) 0 >>> centers = model.clusterCenters() @@ -1265,6 +1265,8 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): 10 >>> lda.clear(lda.maxIter) >>> model = lda.fit(df) + >>> model.setSeed(1) + DistributedLDAModel... >>> model.getTopicDistributionCol() 'topicDistribution' >>> model.isDistributed() diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index f6e531302317b..e771221d5f06d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -337,6 +337,8 @@ class BucketedRandomProjectionLSH(_LSH, _BucketedRandomProjectionLSHParams, >>> model = brp.fit(df) >>> model.getBucketLength() 1.0 + >>> model.setOutputCol("hashes") + BucketedRandomProjectionLSHModel... >>> model.transform(df).head() Row(id=0, features=DenseVector([-1.0, -1.0]), hashes=[DenseVector([-1.0])]) >>> data2 = [(4, Vectors.dense([2.0, 2.0 ]),), @@ -733,6 +735,8 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> cv.setOutputCol("vectors") CountVectorizer... >>> model = cv.fit(df) + >>> model.setInputCol("raw") + CountVectorizerModel... >>> model.transform(df).show(truncate=False) +-----+---------------+-------------------------+ |label|raw |vectors | @@ -1345,6 +1349,8 @@ class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): >>> idf.setOutputCol("idf") IDF... >>> model = idf.fit(df) + >>> model.setOutputCol("idf") + IDFModel... >>> model.getMinDocFreq() 3 >>> model.idf @@ -1519,6 +1525,8 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): >>> imputer.getRelativeError() 0.001 >>> model = imputer.fit(df) + >>> model.setInputCols(["a", "b"]) + ImputerModel... >>> model.getStrategy() 'mean' >>> model.surrogateDF.show() @@ -1810,7 +1818,7 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri MaxAbsScaler... >>> model = maScaler.fit(df) >>> model.setOutputCol("scaledOutput") - MaxAbsScaler... + MaxAbsScalerModel... >>> model.transform(df).show() +-----+------------+ | a|scaledOutput| @@ -1928,6 +1936,8 @@ class MinHashLSH(_LSH, HasInputCol, HasOutputCol, HasSeed, JavaMLReadable, JavaM >>> mh.setSeed(12345) MinHashLSH... >>> model = mh.fit(df) + >>> model.setInputCol("features") + MinHashLSHModel... >>> model.transform(df).head() Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([6179668... >>> data2 = [(3, Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), @@ -2056,7 +2066,7 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri MinMaxScaler... >>> model = mmScaler.fit(df) >>> model.setOutputCol("scaledOutput") - MinMaxScaler... + MinMaxScalerModel... >>> model.originalMin DenseVector([0.0]) >>> model.originalMax @@ -2421,6 +2431,8 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> ohe.setOutputCols(["output"]) OneHotEncoder... >>> model = ohe.fit(df) + >>> model.setOutputCols(["output"]) + OneHotEncoderModel... >>> model.getHandleInvalid() 'error' >>> model.transform(df).head().output @@ -2935,7 +2947,7 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri RobustScaler... >>> model = scaler.fit(df) >>> model.setOutputCol("output") - RobustScaler... + RobustScalerModel... >>> model.median DenseVector([2.0, -2.0]) >>> model.range @@ -3330,7 +3342,7 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM >>> model.getInputCol() 'a' >>> model.setOutputCol("output") - StandardScaler... + StandardScalerModel... >>> model.mean DenseVector([1.0]) >>> model.std @@ -3490,6 +3502,8 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW >>> stringIndexer.setHandleInvalid("error") StringIndexer... >>> model = stringIndexer.fit(stringIndDf) + >>> model.setHandleInvalid("error") + StringIndexerModel... >>> td = model.transform(stringIndDf) >>> sorted(set([(i[0], i[1]) for i in td.select(td.id, td.indexed).collect()]), ... key=lambda x: x[0]) @@ -4166,7 +4180,7 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW >>> indexer.getHandleInvalid() 'error' >>> model.setOutputCol("output") - VectorIndexer... + VectorIndexerModel... >>> model.transform(df).head().output DenseVector([1.0, 0.0]) >>> model.numFeatures @@ -4487,6 +4501,8 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): >>> model = word2Vec.fit(doc) >>> model.getMinCount() 5 + >>> model.setInputCol("sentence") + Word2VecModel... >>> model.getVectors().show() +----+--------------------+ |word| vector| @@ -4714,7 +4730,7 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): >>> model.getK() 2 >>> model.setOutputCol("output") - PCA... + PCAModel... >>> model.transform(df).collect()[0].output DenseVector([1.648..., -4.013...]) >>> model.explainedVariance @@ -5139,6 +5155,8 @@ class ChiSqSelector(JavaEstimator, _ChiSqSelectorParams, JavaMLReadable, JavaMLW >>> model = selector.fit(df) >>> model.getFeaturesCol() 'features' + >>> model.setFeaturesCol("features") + ChiSqSelectorModel... >>> model.transform(df).head().selectedFeatures DenseVector([18.0]) >>> model.selectedFeatures diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 5b34d555484d1..7d933daf9e032 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -166,7 +166,7 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): >>> fp = FPGrowth(minSupport=0.2, minConfidence=0.7) >>> fpm = fp.fit(data) >>> fpm.setPredictionCol("newPrediction") - FPGrowth... + FPGrowthModel... >>> fpm.freqItemsets.show(5) +---------+----+ | items|freq| diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 3ebd0ac2765f3..ee276962c898b 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -225,6 +225,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): >>> model = als.fit(df) >>> model.getUserCol() 'user' + >>> model.setUserCol("user") + ALSModel... >>> model.getItemCol() 'item' >>> model.setPredictionCol("newPrediction") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 84e39a035d80b..fdb04bb5115c5 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -105,9 +105,9 @@ class LinearRegression(JavaPredictor, _LinearRegressionParams, JavaMLWritable, J LinearRegression... >>> model = lr.fit(df) >>> model.setFeaturesCol("features") - LinearRegression... + LinearRegressionModel... >>> model.setPredictionCol("newPrediction") - LinearRegression... + LinearRegressionModel... >>> model.getMaxIter() 5 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) @@ -591,7 +591,7 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, >>> ir = IsotonicRegression() >>> model = ir.fit(df) >>> model.setFeaturesCol("features") - IsotonicRegression... + IsotonicRegressionModel... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 @@ -1546,7 +1546,7 @@ class AFTSurvivalRegression(JavaEstimator, _AFTSurvivalRegressionParams, >>> aftsr.clear(aftsr.maxIter) >>> model = aftsr.fit(df) >>> model.setFeaturesCol("features") - AFTSurvivalRegression... + AFTSurvivalRegressionModel... >>> model.predict(Vectors.dense(6.3)) 1.0 >>> model.predictQuantiles(Vectors.dense(6.3)) @@ -1881,7 +1881,7 @@ class GeneralizedLinearRegression(JavaPredictor, _GeneralizedLinearRegressionPar >>> glr.clear(glr.maxIter) >>> model = glr.fit(df) >>> model.setFeaturesCol("features") - GeneralizedLinearRegression... + GeneralizedLinearRegressionModel... >>> model.getMaxIter() 25 >>> model.getAggregationDepth() diff --git a/python/pyspark/ml/tree.py b/python/pyspark/ml/tree.py index f38a7375c2c54..d97a950c9276e 100644 --- a/python/pyspark/ml/tree.py +++ b/python/pyspark/ml/tree.py @@ -56,9 +56,6 @@ def predictLeaf(self, value): """ return self._call_java("predictLeaf", value) - def __repr__(self): - return self._call_java("toString") - class _DecisionTreeParams(HasCheckpointInterval, HasSeed, HasWeightCol): """ @@ -208,9 +205,6 @@ def predictLeaf(self, value): """ return self._call_java("predictLeaf", value) - def __repr__(self): - return self._call_java("toString") - class _TreeEnsembleParams(_DecisionTreeParams): """ diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 47e4921541ea2..ae3a6ba24ffa5 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -372,6 +372,9 @@ def __init__(self, java_model=None): self._resetUid(java_model.uid()) + def __repr__(self): + return self._call_java("toString") + @inherit_doc class _JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): From f77c10de38d0563b2e42d1200a1fbbdb3018c2e9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 15 Nov 2019 23:58:15 -0800 Subject: [PATCH 11/83] [SPARK-29923][SQL][TESTS] Set io.netty.tryReflectionSetAccessible for Arrow on JDK9+ ### What changes were proposed in this pull request? This PR aims to add `io.netty.tryReflectionSetAccessible=true` to the testing configuration for JDK11 because this is an officially documented requirement of Apache Arrow. Apache Arrow community documented this requirement at `0.15.0` ([ARROW-6206](https://github.com/apache/arrow/pull/5078)). > #### For java 9 or later, should set "-Dio.netty.tryReflectionSetAccessible=true". > This fixes `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available`. thrown by netty. ### Why are the changes needed? After ARROW-3191, Arrow Java library requires the property `io.netty.tryReflectionSetAccessible` to be set to true for JDK >= 9. After https://github.com/apache/spark/pull/26133, JDK11 Jenkins job seem to fail. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/676/ - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/677/ - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/678/ ```scala Previous exception in task: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available io.netty.util.internal.PlatformDependent.directBuffer(PlatformDependent.java:473) io.netty.buffer.NettyArrowBuf.getDirectBuffer(NettyArrowBuf.java:243) io.netty.buffer.NettyArrowBuf.nioBuffer(NettyArrowBuf.java:233) io.netty.buffer.ArrowBuf.nioBuffer(ArrowBuf.java:245) org.apache.arrow.vector.ipc.message.ArrowRecordBatch.computeBodyLength(ArrowRecordBatch.java:222) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with JDK11. Closes #26552 from dongjoon-hyun/SPARK-ARROW-JDK11. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/run-tests.sh | 2 +- pom.xml | 4 ++-- project/SparkBuild.scala | 1 + python/run-tests.py | 5 +++-- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive/pom.xml | 2 +- 7 files changed, 10 insertions(+), 8 deletions(-) diff --git a/R/run-tests.sh b/R/run-tests.sh index 86bd8aad5f113..51ca7d600caf0 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -23,7 +23,7 @@ FAILED=0 LOGFILE=$FWDIR/unit-tests.out rm -f $LOGFILE -SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" --conf spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" --conf spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" diff --git a/pom.xml b/pom.xml index a6a82b3339d08..44593b78c9a04 100644 --- a/pom.xml +++ b/pom.xml @@ -2326,7 +2326,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true From 40ea4a11d7f1534023669f0b81faf5d398174e46 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 16 Nov 2019 17:46:39 +0800 Subject: [PATCH 12/83] [SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" ### What changes were proposed in this pull request? Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" ### Why are the changes needed? The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled". To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect. For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL. ### Does this PR introduce any user-facing change? Yes, the config name changed. ### How was this patch tested? Existing UT. Closes #26444 from xuanyuanking/SPARK-29807. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 8 +++--- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 ++-- .../spark/sql/catalyst/expressions/Cast.scala | 4 +-- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../sql/catalyst/parser/ParseDriver.scala | 4 +-- .../apache/spark/sql/internal/SQLConf.scala | 28 ++++++++++++------- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +-- .../ArithmeticExpressionSuite.scala | 24 ++++++++-------- .../sql/catalyst/expressions/CastSuite.scala | 17 +++++++---- .../expressions/DecimalExpressionSuite.scala | 4 +-- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +-- .../parser/ExpressionParserSuite.scala | 10 +++---- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 +-- .../sql-tests/results/postgreSQL/text.sql.out | 8 +++--- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +-- .../ThriftServerQueryTestSuite.scala | 4 +-- 19 files changed, 77 insertions(+), 64 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 81d7ce37af178..79bc134596237 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. +When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.ansi.enabled` is false. +By default `spark.sql.dialect.spark.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 8a6bd5ed27360..9dda6cbabfe24 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -955,7 +955,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1155,9 +1155,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.ansi.enabled=true`. +// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index f3b58fa3137b1..9e802edbc5edd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -592,7 +592,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it * either returns null or throws an exception according to the value set for - * `spark.sql.ansi.enabled`. + * `spark.sql.dialect.spark.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -611,7 +611,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; + * If overflow occurs, if `spark.sql.dialect.spark.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 82a8e6d80a0bd..7650fb07a61cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -150,7 +150,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") // Name of the function for the exact version of this expression in [[Math]]. - // If the option "spark.sql.ansi.enabled" is enabled and there is corresponding + // If the option "spark.sql.dialect.spark.ansi.enabled" is enabled and there is corresponding // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. def exactMathMethod: Option[String] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index b66cae7979416..a9d5e7c7544a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -101,7 +101,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.ansi = conf.ansiEnabled + lexer.ansi = conf.dialectSparkAnsiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -109,7 +109,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - parser.ansi = conf.ansiEnabled + parser.ansi = conf.dialectSparkAnsiEnabled try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 759586a2936fd..5f3a984cc8d5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1673,6 +1673,20 @@ object SQLConf { .checkValues(Dialect.values.map(_.toString)) .createWithDefault(Dialect.SPARK.toString) + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .internal() + .doc("This configuration is deprecated and will be removed in the future releases." + + "It is replaced by spark.sql.dialect.spark.ansi.enabled.") + .booleanConf + .createWithDefault(false) + + val DIALECT_SPARK_ANSI_ENABLED = buildConf("spark.sql.dialect.spark.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .fallbackConf(ANSI_ENABLED) + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() @@ -1784,14 +1798,6 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) - val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") - .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + - "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + - "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + - "the SQL parser.") - .booleanConf - .createWithDefault(false) - val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2521,9 +2527,11 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: StoreAssignmentPolicy.Value = StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) - def ansiEnabled: Boolean = getConf(ANSI_ENABLED) + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString + + def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString() + def ansiEnabled: Boolean = usePostgreSQLDialect || dialectSparkAnsiEnabled def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index c1f1be3b30e4b..62e688e4d4bd6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -436,7 +436,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testAndVerifyNotLeakingReflectionObjects( s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 1a1cab823d4f3..fe068f7a5f6c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index ad8b1a1673679..6e3fc438e41ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index fb99fc805c45b..12ca3e798b13d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -891,7 +891,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to decimal type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { checkExceptionInExpression[ArithmeticException]( cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -957,7 +958,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to byte type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -982,7 +984,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to short type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1007,7 +1010,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to int type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key ->requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1024,7 +1028,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to long type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => @@ -1201,7 +1206,7 @@ class CastSuite extends CastSuiteBase { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) checkEvaluation( Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index 36bc3db580400..8609d888b7bc9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index c5ffc381b58e2..cf6ebfb0ecefb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index a707b456c6bd1..17a2213d0ccb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -615,7 +615,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"${sign}interval $intervalValue", expectedLiteral) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -701,12 +701,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit.toString) } => a @@ -804,12 +804,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index a9216174804d0..9560aec944d9a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -658,7 +658,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index 7abf903bc6bee..a1fe95462ecae 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -46,9 +46,9 @@ select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); -- [SPARK-28036] Built-in udf left/right has inconsistent behavior -- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.ansi.enabled=false; +set spark.sql.dialect.spark.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.ansi.enabled=true; +set spark.sql.dialect.spark.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 2e1d639974ec6..cac1e7ee5ab12 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -151,11 +151,11 @@ edcba -- !query 18 -set spark.sql.ansi.enabled=false +set spark.sql.dialect.spark.ansi.enabled=false -- !query 18 schema struct -- !query 18 output -spark.sql.ansi.enabled false +spark.sql.dialect.spark.ansi.enabled false -- !query 19 @@ -177,11 +177,11 @@ struct -- !query 20 -set spark.sql.ansi.enabled=true +set spark.sql.dialect.spark.ansi.enabled=true -- !query 20 schema struct -- !query 20 output -spark.sql.ansi.enabled true +spark.sql.dialect.spark.ansi.enabled true -- !query 21 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 40f2e923d1d78..c79e2e9c76bb9 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 @@ -163,7 +163,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() Seq(true, false).foreach { ansiEnabled => - withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { + withSQLConf((SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) if (!ansiEnabled) { checkAnswer(structDf, Row(null)) 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 ffea03474ac4a..d693ff0da33cd 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 @@ -343,10 +343,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) case _ => } 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 index 67193e8de16f4..82da4c049fd41 100644 --- 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 @@ -107,10 +107,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _: AnsiTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") case _ => } From d0470d639412ecbe6e126f8d8abf5a5819b9e278 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 16 Nov 2019 09:26:01 -0800 Subject: [PATCH 13/83] [MINOR][TESTS] Ignore GitHub Action and AppVeyor file changes in testing ### What changes were proposed in this pull request? This PR aims to ignore `GitHub Action` and `AppVeyor` file changes. When we touch these files, Jenkins job should not trigger a full testing. ### Why are the changes needed? Currently, these files are categorized to `root` and trigger the full testing and ends up wasting the Jenkins resources. - https://github.com/apache/spark/pull/26555 ``` [info] Using build tool sbt with Hadoop profile hadoop2.7 under environment amplab_jenkins From https://github.com/apache/spark * [new branch] master -> master [info] Found the following changed modules: sparkr, root [info] Setup the following environment variables for tests: ``` ### Does this PR introduce any user-facing change? No. (Jenkins testing only). ### How was this patch tested? Manually. ``` $ dev/run-tests.py -h -v ... Trying: [x.name for x in determine_modules_for_files([".github/workflows/master.yml", "appveyor.xml"])] Expecting: [] ... ``` Closes #26556 from dongjoon-hyun/SPARK-IGNORE-APPVEYOR. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/run-tests.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 82277720bb52f..fc8b7251a85f8 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -43,15 +43,20 @@ def determine_modules_for_files(filenames): """ Given a list of filenames, return the set of modules that contain those files. If a file is not associated with a more specific submodule, then this method will consider that - file to belong to the 'root' module. + file to belong to the 'root' module. GitHub Action and Appveyor files are ignored. >>> sorted(x.name for x in determine_modules_for_files(["python/pyspark/a.py", "sql/core/foo"])) ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] + >>> [x.name for x in determine_modules_for_files( \ + [".github/workflows/master.yml", "appveyor.yml"])] + [] """ changed_modules = set() for filename in filenames: + if filename in (".github/workflows/master.yml", "appveyor.yml"): + continue matched_at_least_one_module = False for module in modules.all_modules: if module.contains_file(filename): From 53364730049981e3d36b6e6a405b1df5314d321d Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Sat, 16 Nov 2019 13:20:05 -0600 Subject: [PATCH 14/83] [SPARK-29476][WEBUI] add tooltip for Thread ### What changes were proposed in this pull request? Adding tooltip for Thread Dump - Thread Locks Before: ![Screenshot from 2019-11-04 17-11-22](https://user-images.githubusercontent.com/51401130/68127349-b963f580-ff3b-11e9-8547-e01907382632.png) After: ![Screenshot from 2019-11-13 18-12-54](https://user-images.githubusercontent.com/51401130/68768698-08e7a700-0649-11ea-804b-2eb4d5f162b4.png) ### Why are the changes needed? Thread Dump tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual Closes #26386 from PavithraRamachandran/threadDump_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index a13037b5e24db..77564f48015f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -89,7 +89,12 @@ private[ui] class ExecutorThreadDumpPage( Thread ID Thread Name Thread State - Thread Locks + + + Thread Locks + + {dumpRows} From e88267cb5a26b687f69977c9792af18fc3833db1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 16 Nov 2019 18:01:25 -0800 Subject: [PATCH 15/83] [SPARK-29928][SQL][TESTS] Check parsing timestamps up to microsecond precision by JSON/CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to add tests from the commit https://github.com/apache/spark/commit/9c7e8be1dca8285296f3052c41f35043699d7d10 for Spark 2.4 that check parsing of timestamp strings for various seconds fractions. ### Why are the changes needed? To make sure that current behavior is the same as in Spark 2.4 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`. Closes #26558 from MaxGekk/parse-timestamp-micros-tests. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/util/TimestampFormatterSuite.scala | 39 +++++++++++++++++++ .../apache/spark/sql/JsonFunctionsSuite.scala | 10 +++++ .../execution/datasources/csv/CSVSuite.scala | 12 ++++++ 3 files changed, 61 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 6107a15f5c428..79e433e88437b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros +import org.apache.spark.unsafe.types.UTF8String class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { @@ -154,4 +155,42 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers formatter.parse("Tomorrow ") should be(tomorrow +- tolerance) } } + + test("parsing timestamp strings with various seconds fractions") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + def check(pattern: String, input: String, reference: String): Unit = { + val formatter = TimestampFormatter(pattern, zoneId) + val expected = DateTimeUtils.stringToTimestamp( + UTF8String.fromString(reference), zoneId).get + val actual = formatter.parse(input) + assert(actual === expected) + } + + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSSXXX", + "2019-10-14T09:39:07.3220000Z", "2019-10-14T09:39:07.322Z") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + "2019-10-14T09:39:07.322000", "2019-10-14T09:39:07.322") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSX", + "2019-10-14T09:39:07.123456Z", "2019-10-14T09:39:07.123456Z") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSX", + "2019-10-14T09:39:07.000010Z", "2019-10-14T09:39:07.00001Z") + check("yyyy HH:mm:ss.SSSSS", "1970 01:02:03.00004", "1970-01-01 01:02:03.00004") + check("yyyy HH:mm:ss.SSSS", "2019 00:00:07.0100", "2019-01-01 00:00:07.0100") + check("yyyy-MM-dd'T'HH:mm:ss.SSSX", + "2019-10-14T09:39:07.322Z", "2019-10-14T09:39:07.322Z") + check("yyyy-MM-dd'T'HH:mm:ss.SS", + "2019-10-14T09:39:07.10", "2019-10-14T09:39:07.1") + check("yyyy-MM-dd'T'HH:mm:ss.S", + "2019-10-14T09:39:07.1", "2019-10-14T09:39:07.1") + + try { + TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId) + .parse("2019/11/14 20#25#30.123456") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: java.time.format.DateTimeParseException => + assert(e.getMessage.contains("could not be parsed")) + } + } + } } 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 c41ca925721bf..af80fa3d9327a 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 @@ -627,4 +627,14 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) } } + + test("from_json - timestamp in micros") { + val df = Seq("""{"time": "1970-01-01T00:00:00.123456"}""").toDS() + val schema = new StructType().add("time", TimestampType) + val options = Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSSSS") + + checkAnswer( + df.select(from_json($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456")))) + } } 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 48a8c92aad759..dac305944dd0d 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 @@ -2168,4 +2168,16 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } } } + + test("parse timestamp in microsecond precision") { + withTempPath { path => + val t = "2019-11-14 20:35:30.123456" + Seq(t).toDF("t").write.text(path.getAbsolutePath) + val readback = spark.read + .schema("t timestamp") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } + } } From cc12cf6029b1b119599a381f4e4600c0a6525ff2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 16 Nov 2019 18:28:27 -0800 Subject: [PATCH 16/83] [SPARK-29378][R] Upgrade SparkR to use Arrow 0.15 API ### What changes were proposed in this pull request? [[SPARK-29376] Upgrade Apache Arrow to version 0.15.1](https://github.com/apache/spark/pull/26133) upgrades to Arrow 0.15 at Scala/Java/Python. This PR aims to upgrade `SparkR` to use Arrow 0.15 API. Currently, it's broken. ### Why are the changes needed? First of all, it turns out that our Jenkins jobs (including PR builder) ignores Arrow test. Arrow 0.15 has a breaking R API changes at [ARROW-5505](https://issues.apache.org/jira/browse/ARROW-5505) and we missed that. AppVeyor was the only one having SparkR Arrow tests but it's broken now. **Jenkins** ``` Skipped ------------------------------------------------------------------------ 1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#25) - arrow not installed ``` Second, Arrow throws OOM on AppVeyor environment (Windows JDK8) like the following because it still has Arrow 0.14. ``` Warnings ----------------------------------------------------------------------- 1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#39) - createDataFrame attempted Arrow optimization because 'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, failed, attempting non-optimization. Reason: Error in handleErrors(returnStatus, conn): java.lang.OutOfMemoryError: Java heap space at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) at org.apache.arrow.vector.ipc.message.MessageSerializer.readMessage(MessageSerializer.java:669) at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$3.readNextBatch(ArrowConverters.scala:243) ``` It is due to the version mismatch. ```java int messageLength = MessageSerializer.bytesToInt(buffer.array()); if (messageLength == IPC_CONTINUATION_TOKEN) { buffer.clear(); // ARROW-6313, if the first 4 bytes are continuation message, read the next 4 for the length if (in.readFully(buffer) == 4) { messageLength = MessageSerializer.bytesToInt(buffer.array()); } } // Length of 0 indicates end of stream if (messageLength != 0) { // Read the message into the buffer. ByteBuffer messageBuffer = ByteBuffer.allocate(messageLength); ``` After upgrading this to 0.15, we are hitting ARROW-5505. This PR upgrades Arrow version in AppVeyor and fix the issue. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the AppVeyor. This PR passed here. - https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark/builds/28909044 ``` SparkSQL Arrow optimization: Spark package found in SPARK_HOME: C:\projects\spark\bin\.. ................ ``` Closes #26555 from dongjoon-hyun/SPARK-R-TEST. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/pkg/R/SQLContext.R | 4 ++-- R/pkg/R/deserialize.R | 2 +- appveyor.yml | 4 +--- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index f27ef4ee28f16..cf96c658d34d2 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -166,9 +166,9 @@ writeToFileInArrow <- function(fileName, rdf, numPartitions) { for (rdf_slice in rdf_slices) { batch <- arrow::record_batch(rdf_slice) if (is.null(stream_writer)) { - stream <- arrow::FileOutputStream(fileName) + stream <- arrow::FileOutputStream$create(fileName) schema <- batch$schema - stream_writer <- arrow::RecordBatchStreamWriter(stream, schema) + stream_writer <- arrow::RecordBatchStreamWriter$create(stream, schema) } stream_writer$write_batch(batch) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index a6febb1cbd132..ca4a6e342d772 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -242,7 +242,7 @@ readDeserializeInArrow <- function(inputCon) { # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") - batches <- arrow::RecordBatchStreamReader(arrowData)$batches() + batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() if (useAsTibble) { as_tibble <- get("as_tibble", envir = asNamespace("arrow")) diff --git a/appveyor.yml b/appveyor.yml index b36175a787ae9..c3baa1f82a399 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,10 +42,8 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - # Use Arrow R 0.14.1 for now. 0.15.0 seems not working for now. See SPARK-29378. + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages('https://cran.r-project.org/src/contrib/Archive/arrow/arrow_0.14.1.tar.gz', repos=NULL, type='source')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. From 388a737b985b048ed4330764f1788160c278d929 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Sat, 16 Nov 2019 19:50:02 -0800 Subject: [PATCH 17/83] [SPARK-29858][SQL] ALTER DATABASE (SET DBPROPERTIES) should look up catalog like v2 commands ### What changes were proposed in this pull request? Add AlterNamespaceSetPropertiesStatement, AlterNamespaceSetProperties and AlterNamespaceSetPropertiesExec to make ALTER DATABASE (SET DBPROPERTIES) command look up catalog like v2 commands. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "ALTER NAMESPACE ... SET (DBPROPERTIES | PROPERTIES) ..." whose function is same as "ALTER DATABASE ... SET DBPROPERTIES ..." and "ALTER SCHEMA ... SET DBPROPERTIES ...". ### How was this patch tested? New unit test Closes #26551 from fuwhu/SPARK-29858. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 ++ .../sql/catalyst/parser/AstBuilder.scala | 17 ++++++++ .../catalyst/plans/logical/statements.scala | 7 +++ .../catalyst/plans/logical/v2Commands.scala | 9 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 32 ++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 7 +++ .../spark/sql/execution/SparkSqlParser.scala | 15 ------- .../v2/AlterNamespaceSetPropertiesExec.scala | 43 +++++++++++++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 5 ++- .../sql/connector/DataSourceV2SQLSuite.scala | 15 +++++++ .../execution/command/DDLParserSuite.scala | 19 -------- 12 files changed, 139 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 9dda6cbabfe24..d2d145606b627 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -91,8 +91,8 @@ statement ((COMMENT comment=STRING) | locationSpec | (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace - | ALTER database db=errorCapturingIdentifier - SET DBPROPERTIES tablePropertyList #setDatabaseProperties + | ALTER (database | NAMESPACE) multipartIdentifier + SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties | ALTER database db=errorCapturingIdentifier SET locationSpec #setDatabaseLocation | DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 22caf33b50ce6..0d8d58520edad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -93,6 +93,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + s"because view support in catalog has not been implemented yet") + case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) => + AlterNamespaceSetProperties(catalog, nameParts, properties) + case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 73233f934bedc..6be163b4b3c2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2528,6 +2528,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.CASCADE != null) } + /** + * Create an [[AlterNamespaceSetPropertiesStatement]] logical plan. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) database + * SET (DBPROPERTIES|PROPERTIES) (property_name=property_value, ...); + * }}} + */ + override def visitSetNamespaceProperties(ctx: SetNamespacePropertiesContext): LogicalPlan = { + withOrigin(ctx) { + AlterNamespaceSetPropertiesStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitPropertyKeyValues(ctx.tablePropertyList)) + } + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 903f5cd5912c6..875ff4e4a2821 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -350,6 +350,13 @@ case class DropNamespaceStatement( ifExists: Boolean, cascade: Boolean) extends ParsedStatement +/** + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) command, as parsed from SQL. + */ +case class AlterNamespaceSetPropertiesStatement( + namespace: Seq[String], + properties: Map[String, String]) extends ParsedStatement + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 3179bab8c2f7f..22edb3619a929 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -270,6 +270,15 @@ case class DescribeNamespace( new MetadataBuilder().putString("comment", "value of the column").build())()) } +/** + * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) + * command that works for v2 catalogs. + */ +case class AlterNamespaceSetProperties( + catalog: CatalogPlugin, + namespace: Seq[String], + properties: Map[String, String]) extends Command + /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b998127ebe02a..e87f53910c2c7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1160,6 +1160,38 @@ class DDLParserSuite extends AnalysisTest { DropNamespaceStatement(Seq("a", "b", "c"), ifExists = false, cascade = true)) } + test("set namespace properties") { + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a", "b" -> "b", "c" -> "c"))) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET PROPERTIES ('a'='a')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a"))) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET PROPERTIES ('b'='b')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("b" -> "b"))) + + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a", "b" -> "b", "c" -> "c"))) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET DBPROPERTIES ('a'='a')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a"))) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET DBPROPERTIES ('b'='b')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("b" -> "b"))) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 3913f889eb128..cac320edc47e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -165,6 +165,13 @@ class ResolveSessionCatalog( } DescribeDatabaseCommand(nameParts.head, d.extended) + case AlterNamespaceSetPropertiesStatement(SessionCatalog(_, nameParts), properties) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + AlterDatabasePropertiesCommand(nameParts.head, properties) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { 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 bfea908d106df..7fe4c00f5dc8d 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 @@ -227,21 +227,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create an [[AlterDatabasePropertiesCommand]] command. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA) database SET DBPROPERTIES (property_name=property_value, ...); - * }}} - */ - override def visitSetDatabaseProperties( - ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterDatabasePropertiesCommand( - ctx.db.getText, - visitPropertyKeyValues(ctx.tablePropertyList)) - } - /** * Create an [[AlterDatabaseSetLocationCommand]] command. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala new file mode 100644 index 0000000000000..fd6922cd1c3ab --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, NamespaceChange} + +/** + * Physical plan node for setting properties of namespace. + */ +case class AlterNamespaceSetPropertiesExec( + catalog: CatalogPlugin, + namespace: Seq[String], + props: Map[String, String]) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val changes = props.map{ case (k, v) => + NamespaceChange.setProperty(k, v) + }.toSeq + catalog.asNamespaceCatalog.alterNamespace(namespace.toArray, changes: _*) + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index a04bceb18b9b7..95caebe9e8938 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -204,6 +204,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case AlterNamespaceSetProperties(catalog, namespace, properties) => + AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil + case CreateNamespace(catalog, namespace, ifNotExists, properties) => CreateNamespaceExec(catalog, namespace, ifNotExists, properties) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 095bcff31ccd9..e5bf8e337c8d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -933,6 +933,21 @@ class DataSourceV2SQLSuite } } + test("AlterNamespaceSetProperties using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test' WITH PROPERTIES ('a'='a','b'='b','c'='c')") + sql("ALTER NAMESPACE testcat.ns1.ns2 SET PROPERTIES ('a'='b','b'='a')") + val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") + assert(descriptionDf.collect() === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test"), + Row("Properties", "((a,b),(b,a),(c,c))") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") 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 b2185f8559f36..b11e02a9c52bd 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 @@ -74,25 +74,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } - test("alter database set dbproperties") { - // ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) - val sql1 = "ALTER DATABASE database_name SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')" - val sql2 = "ALTER SCHEMA database_name SET DBPROPERTIES ('a'='a')" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - - val expected1 = AlterDatabasePropertiesCommand( - "database_name", - Map("a" -> "a", "b" -> "b", "c" -> "c")) - val expected2 = AlterDatabasePropertiesCommand( - "database_name", - Map("a" -> "a")) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("alter database - property values must be set") { assertUnsupported( sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", From a9959be2bcf55f753ef48e0a4daea6abe0d63c1c Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Sun, 17 Nov 2019 07:04:40 -0600 Subject: [PATCH 18/83] [SPARK-29456][WEBUI] Improve tooltip for Session Statistics Table column in JDBC/ODBC Server Tab What changes were proposed in this pull request? Some of the columns of JDBC/ODBC tab Session info in Web UI are hard to understand. Add tool tip for Start time, finish time , Duration and Total Execution ![Screenshot from 2019-10-16 12-33-17](https://user-images.githubusercontent.com/51401130/66901981-76d68980-f01d-11e9-9686-e20346a38c25.png) Why are the changes needed? To improve the understanding of the WebUI Does this PR introduce any user-facing change? No How was this patch tested? manual test Closes #26138 from PavithraRamachandran/JDBC_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../thriftserver/ui/ThriftServerPage.scala | 43 ++++++++++++++----- .../sql/hive/thriftserver/ui/ToolTips.scala | 7 +++ 2 files changed, 39 insertions(+), 11 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 368db975ec886..d3351f3d6ca14 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -443,10 +443,13 @@ private[ui] class SessionStatsPagedTable( val sessionTableHeaders = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", "Total Execute") + val tooltips = Seq(None, None, None, None, None, Some(THRIFT_SESSION_DURATION), + Some(THRIFT_SESSION_TOTAL_EXECUTE)) + assert(sessionTableHeaders.length == tooltips.length) val colWidthAttr = s"${100.toDouble / sessionTableHeaders.size}%" val headerRow: Seq[Node] = { - sessionTableHeaders.map { header => + sessionTableHeaders.zip(tooltips).map { case (header, tooltip) => if (header == sortColumn) { val headerLink = Unparsed( parameterPath + @@ -455,12 +458,22 @@ private[ui] class SessionStatsPagedTable( s"&$sessionStatsTableTag.pageSize=$pageSize" + s"#$sessionStatsTableTag") val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + { + if (tooltip.nonEmpty) { + + {header} {Unparsed(arrow)} + + } else { + + {header} {Unparsed(arrow)} + + } + } + + - - - {header} {Unparsed(arrow)} - - } else { val headerLink = Unparsed( parameterPath + @@ -468,11 +481,19 @@ private[ui] class SessionStatsPagedTable( s"&$sessionStatsTableTag.pageSize=$pageSize" + s"#$sessionStatsTableTag") - - - {header} - - + + + { + if (tooltip.nonEmpty) { + + {header} + + } else { + {header} + } + } + + } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala index 1990b8f2d3285..56ab766f4aabd 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala @@ -29,4 +29,11 @@ private[ui] object ToolTips { val THRIFT_SERVER_DURATION = "Difference between start time and close time" + + val THRIFT_SESSION_TOTAL_EXECUTE = + "Number of operations submitted in this session" + + val THRIFT_SESSION_DURATION = + "Elapsed time since session start, or until closed if the session was closed" + } From e1fc38b3e409e8a2c65d0cc1fc2ec63da527bbc6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 17 Nov 2019 10:09:46 -0800 Subject: [PATCH 19/83] [SPARK-29932][R][TESTS] lint-r should do non-zero exit in case of errors ### What changes were proposed in this pull request? This PR aims to make `lint-r` exits with non-zero in case of errors. Please note that `lint-r` works correctly when everything are installed correctly. ### Why are the changes needed? There are two cases which hide errors from Jenkins/AppVeyor/GitHubAction. 1. `lint-r` exits with zero if there is no R installation. ```bash $ dev/lint-r dev/lint-r: line 25: type: Rscript: not found ERROR: You should install R $ echo $? 0 ``` 2. `lint-r` exits with zero if we didn't do `R/install-dev.sh`. ```bash $ dev/lint-r Error: You should install SparkR in a local directory with `R/install-dev.sh`. In addition: Warning message: In library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE) : no library trees found in 'lib.loc' Execution halted lintr checks passed. // <=== Please note here $ echo $? 0 ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check the above two cases. Closes #26561 from dongjoon-hyun/SPARK-29932. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/lint-r | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/lint-r b/dev/lint-r index bfda0bca15eb7..b08f5efecd5d3 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -17,6 +17,9 @@ # limitations under the License. # +set -o pipefail +set -e + SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" @@ -24,7 +27,7 @@ LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" if ! type "Rscript" > /dev/null; then echo "ERROR: You should install R" - exit + exit 1 fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" From 5eb8973f871fef557fb4ca3f494406ed676a431a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 17 Nov 2019 10:14:04 -0800 Subject: [PATCH 20/83] [SPARK-29930][SQL] Remove SQL configs declared to be removed in Spark 3.0 ### What changes were proposed in this pull request? In the PR, I propose to remove the following SQL configs: 1. `spark.sql.fromJsonForceNullableSchema` 2. `spark.sql.legacy.compareDateTimestampInTimestamp` 3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` that are declared to be removed in Spark 3.0 ### Why are the changes needed? To make code cleaner and improve maintainability. ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? By `TypeCoercionSuite`, `JsonExpressionsSuite` and `DDLSuite`. Closes #26559 from MaxGekk/remove-sql-configs. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/TypeCoercion.scala | 7 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../expressions/jsonExpressions.scala | 4 +-- .../apache/spark/sql/internal/SQLConf.scala | 31 ---------------- .../catalyst/analysis/TypeCoercionSuite.scala | 29 +++++---------- .../expressions/JsonExpressionsSuite.scala | 36 +++++++++---------- 6 files changed, 29 insertions(+), 81 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index b27d6ed0efed8..83c76c2d4e2bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -132,12 +132,9 @@ object TypeCoercion { case (NullType, StringType) => Some(StringType) // Cast to TimestampType when we compare DateType with TimestampType - // if conf.compareDateTimestampInTimestamp is true // i.e. TimeStamp('2017-03-01 00:00:00') eq Date('2017-03-01') = true - case (TimestampType, DateType) - => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) - case (DateType, TimestampType) - => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) + case (TimestampType, DateType) => Some(TimestampType) + case (DateType, TimestampType) => Some(TimestampType) // There is no proper decimal type we can pick, // using double type is the best we can do. 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 be8526454f9f1..e72352714a697 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 @@ -327,8 +327,7 @@ class SessionCatalog( def validateTableLocation(table: CatalogTable): Unit = { // SPARK-19724: the default location of a managed table should be non-existent or empty. - if (table.tableType == CatalogTableType.MANAGED && - !conf.allowCreatingManagedTableUsingNonemptyLocation) { + if (table.tableType == CatalogTableType.MANAGED) { val tableLocation = new Path(table.storage.locationUri.getOrElse(defaultTablePath(table.identifier))) val fs = tableLocation.getFileSystem(hadoopConf) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index a13a6836c6be6..de7e1160185dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -515,12 +515,10 @@ case class JsonToStructs( timeZoneId: Option[String] = None) extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { - val forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) - // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. - val nullableSchema = if (forceNullableSchema) schema.asNullable else schema + val nullableSchema = schema.asNullable override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5f3a984cc8d5d..8b714d6dcdda3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -720,14 +720,6 @@ object SQLConf { .stringConf .createWithDefault("_corrupt_record") - val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.fromJsonForceNullableSchema") - .internal() - .doc("When true, force the output schema of the from_json() function to be nullable " + - "(including all the fields). Otherwise, the schema might not be compatible with" + - "actual data, which leads to corruptions. This config will be removed in Spark 3.0.") - .booleanConf - .createWithDefault(true) - val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) @@ -1687,14 +1679,6 @@ object SQLConf { "the SQL parser.") .fallbackConf(ANSI_ENABLED) - val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = - buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") - .internal() - .doc("When this option is set to true, creating managed tables with nonempty location " + - "is allowed. Otherwise, an analysis exception is thrown. ") - .booleanConf - .createWithDefault(false) - val VALIDATE_PARTITION_COLUMNS = buildConf("spark.sql.sources.validatePartitionColumns") .internal() @@ -1913,16 +1897,6 @@ object SQLConf { .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION) .createWithDefault(Deflater.DEFAULT_COMPRESSION) - val COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP = - buildConf("spark.sql.legacy.compareDateTimestampInTimestamp") - .internal() - .doc("When true (default), compare Date with Timestamp after converting both sides to " + - "Timestamp. This behavior is compatible with Hive 2.2 or later. See HIVE-15236. " + - "When false, restore the behavior prior to Spark 2.4. Compare Date with Timestamp after " + - "converting both sides to string. This config will be removed in Spark 3.0.") - .booleanConf - .createWithDefault(true) - val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + "The size function returns null for null input if the flag is disabled.") @@ -2236,8 +2210,6 @@ class SQLConf extends Serializable with Logging { def caseSensitiveInferenceMode: HiveCaseSensitiveInferenceMode.Value = HiveCaseSensitiveInferenceMode.withName(getConf(HIVE_CASE_SENSITIVE_INFERENCE)) - def compareDateTimestampInTimestamp : Boolean = getConf(COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP) - def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) @@ -2516,9 +2488,6 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) - def allowCreatingManagedTableUsingNonemptyLocation: Boolean = - getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) - def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) def partitionOverwriteMode: PartitionOverwriteMode.Value = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index c7371a7911df5..567cf5ec8ebe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1526,26 +1526,15 @@ class TypeCoercionSuite extends AnalysisTest { GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))), GreaterThan(Cast(Literal("1.5"), DoubleType), Cast(Literal(BigDecimal("0.5")), DoubleType))) - Seq(true, false).foreach { convertToTS => - withSQLConf( - SQLConf.COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP.key -> convertToTS.toString) { - val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) - val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) - val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) - if (convertToTS) { - // `Date` should be treated as timestamp at 00:00:00 See SPARK-23549 - ruleTest(rule, EqualTo(date0301, timestamp0301000000), - EqualTo(Cast(date0301, TimestampType), timestamp0301000000)) - ruleTest(rule, LessThan(date0301, timestamp0301000001), - LessThan(Cast(date0301, TimestampType), timestamp0301000001)) - } else { - ruleTest(rule, LessThan(date0301, timestamp0301000000), - LessThan(Cast(date0301, StringType), Cast(timestamp0301000000, StringType))) - ruleTest(rule, LessThan(date0301, timestamp0301000001), - LessThan(Cast(date0301, StringType), Cast(timestamp0301000001, StringType))) - } - } - } + // Checks that dates/timestamps are not promoted to strings + val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) + val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) + val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) + // `Date` should be treated as timestamp at 00:00:00 See SPARK-23549 + ruleTest(rule, EqualTo(date0301, timestamp0301000000), + EqualTo(Cast(date0301, TimestampType), timestamp0301000000)) + ruleTest(rule, LessThan(date0301, timestamp0301000001), + LessThan(Cast(date0301, TimestampType), timestamp0301000001)) } test("cast WindowFrame boundaries to the type they operate upon") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index f8400a590606a..d5cc1d4f0fdde 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -702,26 +702,22 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with } test("from_json missing fields") { - for (forceJsonNullableSchema <- Seq(false, true)) { - withSQLConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA.key -> forceJsonNullableSchema.toString) { - val input = - """{ - | "a": 1, - | "c": "foo" - |} - |""".stripMargin - val jsonSchema = new StructType() - .add("a", LongType, nullable = false) - .add("b", StringType, nullable = !forceJsonNullableSchema) - .add("c", StringType, nullable = false) - val output = InternalRow(1L, null, UTF8String.fromString("foo")) - val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) - checkEvaluation(expr, output) - val schema = expr.dataType - val schemaToCompare = if (forceJsonNullableSchema) jsonSchema.asNullable else jsonSchema - assert(schemaToCompare == schema) - } - } + val input = + """{ + | "a": 1, + | "c": "foo" + |} + |""".stripMargin + val jsonSchema = new StructType() + .add("a", LongType, nullable = false) + .add("b", StringType, nullable = false) + .add("c", StringType, nullable = false) + val output = InternalRow(1L, null, UTF8String.fromString("foo")) + val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) + checkEvaluation(expr, output) + val schema = expr.dataType + val schemaToCompare = jsonSchema.asNullable + assert(schemaToCompare == schema) } test("SPARK-24709: infer schema of json strings") { From c5f644c6ebb871d76f6b6b12b341ba761427492c Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Mon, 18 Nov 2019 10:05:42 +0800 Subject: [PATCH 21/83] [SPARK-16872][ML][PYSPARK] Impl Gaussian Naive Bayes Classifier ### What changes were proposed in this pull request? support `modelType` `gaussian` ### Why are the changes needed? current modelTypes do not support continuous data ### Does this PR introduce any user-facing change? yes, add a `modelType` option ### How was this patch tested? existing testsuites and added ones Closes #26413 from zhengruifeng/gnb. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- docs/ml-classification-regression.md | 9 +- .../spark/ml/classification/NaiveBayes.scala | 299 ++++++++++++++---- .../ml/classification/NaiveBayesSuite.scala | 193 ++++++++++- project/MimaExcludes.scala | 3 + python/pyspark/ml/classification.py | 24 +- 5 files changed, 451 insertions(+), 77 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index b83b4ba08a5fd..d8c7d8a729624 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -478,15 +478,16 @@ it computes the conditional probability distribution of each feature given each For prediction, it applies Bayes' theorem to compute the conditional probability distribution of each label given an observation. -MLlib supports both [multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) -and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). +MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +[Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) +and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). *Input data*: -These models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. A feature's value is the frequency of the term (in multinomial Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). -Feature values must be *non-negative*. The model type is selected with an optional parameter +Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter "multinomial" or "bernoulli" with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index bcca40d159c9b..8062870794418 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -18,18 +18,22 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.stat.Summarizer import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.VersionUtils /** * Params for Naive Bayes Classifiers. @@ -49,12 +53,13 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial" and "bernoulli". + * Supported options: "multinomial", "bernoulli", "gaussian". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default) and bernoulli.", + "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + + " and gaussian.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -72,7 +77,11 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * binary (0/1) data, it can also be used as Bernoulli NB * (see * here). - * The input feature values must be nonnegative. + * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it also supports Gaussian NB + * (see + * here) + * which can handle continuous data. */ // scalastyle:on line.size.limit @Since("1.5.0") @@ -103,7 +112,7 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> NaiveBayes.Multinomial) + setDefault(modelType -> Multinomial) /** * Sets the value of param [[weightCol]]. @@ -130,6 +139,9 @@ class NaiveBayes @Since("1.5.0") ( positiveLabel: Boolean): NaiveBayesModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + if (positiveLabel && isDefined(thresholds)) { val numClasses = getNumClasses(dataset) instr.logNumClasses(numClasses) @@ -138,44 +150,55 @@ class NaiveBayes @Since("1.5.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val validateInstance = $(modelType) match { - case Multinomial => - (instance: Instance) => requireNonnegativeValues(instance.features) - case Bernoulli => - (instance: Instance) => requireZeroOneBernoulliValues(instance.features) + $(modelType) match { + case Bernoulli | Multinomial => + trainDiscreteImpl(dataset, instr) + case Gaussian => + trainGaussianImpl(dataset, instr) case _ => // This should never happen. throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } + } - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, - probabilityCol, modelType, smoothing, thresholds) + private def trainDiscreteImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ - val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size - instr.logNumFeatures(numFeatures) + val validateUDF = $(modelType) match { + case Multinomial => + udf { vector: Vector => requireNonnegativeValues(vector); vector } + case Bernoulli => + udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } + } + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } // Aggregates term frequencies per label. - // TODO: Calling aggregateByKey and collect creates two stages, we can implement something - // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = extractInstances(dataset, validateInstance).map { instance => - (instance.label, (instance.weight, instance.features)) - }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( - seqOp = { - case ((weightSum, featureSum, count), (weight, features)) => - BLAS.axpy(weight, features, featureSum) - (weightSum + weight, featureSum, count + 1) - }, - combOp = { - case ((weightSum1, featureSum1, count1), (weightSum2, featureSum2, count2)) => - BLAS.axpy(1.0, featureSum2, featureSum1) - (weightSum1 + weightSum2, featureSum1, count1 + count2) - }).collect().sortBy(_._1) - - val numSamples = aggregated.map(_._2._3).sum + // TODO: Summarizer directly returns sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "count") + .summary(validateUDF(col($(featuresCol))), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.count") + .as[(Double, Double, Vector, Long)] + .map { case (label, weightSum, mean, count) => + BLAS.scal(weightSum, mean) + (label, weightSum, mean, count) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + val numSamples = aggregated.map(_._4).sum instr.logNumExamples(numSamples) val numLabels = aggregated.length instr.logNumClasses(numLabels) - val numDocuments = aggregated.map(_._2._1).sum + val numDocuments = aggregated.map(_._2).sum val labelArray = new Array[Double](numLabels) val piArray = new Array[Double](numLabels) @@ -184,19 +207,17 @@ class NaiveBayes @Since("1.5.0") ( val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, (n, sumTermFreqs, _)) => + aggregated.foreach { case (label, n, sumTermFreqs, _) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) + case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } var j = 0 + val offset = i * numFeatures while (j < numFeatures) { - thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + thetaArray(offset + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 @@ -204,7 +225,86 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + .setOldLabels(labelArray) + } + + private def trainGaussianImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + // Aggregates mean vector and square-sum vector per label. + // TODO: Summarizer directly returns square-sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") + .summary(col($(featuresCol)), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.normL2") + .as[(Double, Double, Vector, Vector)] + .map { case (label, weightSum, mean, normL2) => + (label, weightSum, mean, Vectors.dense(normL2.toArray.map(v => v * v))) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + + val numInstances = aggregated.map(_._2).sum + + // If the ratio of data variance between dimensions is too small, it + // will cause numerical errors. To address this, we artificially + // boost the variance by epsilon, a small fraction of the standard + // deviation of the largest dimension. + // Refer to scikit-learn's implementation + // [https://github.com/scikit-learn/scikit-learn/blob/0.21.X/sklearn/naive_bayes.py#L348] + // and discussion [https://github.com/scikit-learn/scikit-learn/pull/5349] for detail. + val epsilon = Iterator.range(0, numFeatures).map { j => + var globalSum = 0.0 + var globalSqrSum = 0.0 + aggregated.foreach { case (_, weightSum, mean, squareSum) => + globalSum += mean(j) * weightSum + globalSqrSum += squareSum(j) + } + globalSqrSum / numInstances - + globalSum * globalSum / numInstances / numInstances + }.max * 1e-9 + + val piArray = new Array[Double](numLabels) + + // thetaArray in Gaussian NB store the means of features per label + val thetaArray = new Array[Double](numLabels * numFeatures) + + // thetaArray in Gaussian NB store the variances of features per label + val sigmaArray = new Array[Double](numLabels * numFeatures) + + var i = 0 + val logNumInstances = math.log(numInstances) + aggregated.foreach { case (_, weightSum, mean, squareSum) => + piArray(i) = math.log(weightSum) - logNumInstances + var j = 0 + val offset = i * numFeatures + while (j < numFeatures) { + val m = mean(j) + thetaArray(offset + j) = m + sigmaArray(offset + j) = epsilon + squareSum(j) / weightSum - m * m + j += 1 + } + i += 1 + } + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + val sigma = new DenseMatrix(numLabels, numFeatures, sigmaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, sigma.compressed) } @Since("1.5.0") @@ -219,8 +319,11 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Bernoulli model type. */ private[classification] val Bernoulli: String = "bernoulli" + /** String name for Gaussian model type. */ + private[classification] val Gaussian: String = "gaussian" + /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { @@ -248,19 +351,24 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** * Model produced by [[NaiveBayes]] - * @param pi log of class priors, whose dimension is C (number of classes) + * + * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) + * @param sigma variance of each feature, whose dimension is C (number of classes) + * by D (number of features). This matrix is only available when modelType + * is set Gaussian. */ @Since("1.5.0") class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val pi: Vector, - @Since("2.0.0") val theta: Matrix) + @Since("2.0.0") val theta: Matrix, + @Since("3.0.0") val sigma: Matrix) extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial} + import NaiveBayes.{Bernoulli, Multinomial, Gaussian} /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -280,18 +388,36 @@ class NaiveBayesModel private[ml] ( * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra * application of this condition (in predict function). */ - private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { - case Multinomial => (None, None) + @transient private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { case Bernoulli => val negTheta = theta.map(value => math.log1p(-math.exp(value))) val ones = new DenseVector(Array.fill(theta.numCols) {1.0}) val thetaMinusNegTheta = theta.map { value => value - math.log1p(-math.exp(value)) } - (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) + (thetaMinusNegTheta, negTheta.multiply(ones)) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables thetaMinusNegTheta and negThetaSum should only be precomputed in Bernoulli NB.") + } + + /** + * Gaussian scoring requires sum of log(Variance). + * This precomputes sum of log(Variance) which are used for the linear algebra + * application of this condition (in predict function). + */ + @transient private lazy val logVarSum = $(modelType) match { + case Gaussian => + Array.tabulate(numClasses) { i => + Iterator.range(0, numFeatures).map { j => + math.log(sigma(i, j)) + }.sum + } case _ => // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables logVarSum should only be precomputed in Gaussian NB.") } @Since("1.6.0") @@ -311,24 +437,42 @@ class NaiveBayesModel private[ml] ( require(value == 0.0 || value == 1.0, s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") ) - val prob = thetaMinusNegTheta.get.multiply(features) + val prob = thetaMinusNegTheta.multiply(features) BLAS.axpy(1.0, pi, prob) - BLAS.axpy(1.0, negThetaSum.get, prob) + BLAS.axpy(1.0, negThetaSum, prob) prob } - override protected def predictRaw(features: Vector): Vector = { + private def gaussianCalculation(features: Vector) = { + val prob = Array.ofDim[Double](numClasses) + var i = 0 + while (i < numClasses) { + var s = 0.0 + var j = 0 + while (j < numFeatures) { + val d = features(j) - theta(i, j) + s += d * d / sigma(i, j) + j += 1 + } + prob(i) = pi(i) - (s + logVarSum(i)) / 2 + i += 1 + } + Vectors.dense(prob) + } + + @transient private lazy val predictRawFunc = { $(modelType) match { case Multinomial => - multinomialCalculation(features) + features: Vector => multinomialCalculation(features) case Bernoulli => - bernoulliCalculation(features) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + features: Vector => bernoulliCalculation(features) + case Gaussian => + features: Vector => gaussianCalculation(features) } } + override protected def predictRaw(features: Vector): Vector = predictRawFunc(features) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => @@ -354,7 +498,7 @@ class NaiveBayesModel private[ml] ( @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayesModel = { - copyValues(new NaiveBayesModel(uid, pi, theta).setParent(this.parent), extra) + copyValues(new NaiveBayesModel(uid, pi, theta, sigma).setParent(this.parent), extra) } @Since("1.5.0") @@ -378,34 +522,61 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { /** [[MLWriter]] instance for [[NaiveBayesModel]] */ private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { + import NaiveBayes._ private case class Data(pi: Vector, theta: Matrix) + private case class GaussianData(pi: Vector, theta: Matrix, sigma: Matrix) override protected def saveImpl(path: String): Unit = { // Save metadata and Params DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: pi, theta - val data = Data(instance.pi, instance.theta) val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + instance.getModelType match { + case Multinomial | Bernoulli => + // Save model data: pi, theta + require(instance.sigma == null) + val data = Data(instance.pi, instance.theta) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + case Gaussian => + require(instance.sigma != null) + val data = GaussianData(instance.pi, instance.theta, instance.sigma) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } } } private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { + import NaiveBayes._ /** Checked against metadata when loading model */ private val className = classOf[NaiveBayesModel].getName override def load(path: String): NaiveBayesModel = { + implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + val modelTypeJson = metadata.getParamValue("modelType") + val modelType = Param.jsonDecode[String](compact(render(modelTypeJson))) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") - val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") - .select("pi", "theta") - .head() - val model = new NaiveBayesModel(metadata.uid, pi, theta) + + val model = if (major.toInt < 3 || modelType != Gaussian) { + val Row(pi: Vector, theta: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, null) + } else { + val Row(pi: Vector, theta: Matrix, sigma: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") + .select("pi", "theta", "sigma") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, sigma) + } metadata.getAndSetParams(model) model diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9100ef1db6e12..9e4844ff89079 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -22,15 +22,15 @@ import scala.util.Random import breeze.linalg.{DenseVector => BDV, Vector => BV} import breeze.stats.distributions.{Multinomial => BrzMultinomial, RandBasis => BrzRandBasis} -import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.ml.classification.NaiveBayes.{Bernoulli, Multinomial} +import org.apache.spark.SparkException +import org.apache.spark.ml.classification.NaiveBayes._ import org.apache.spark.ml.classification.NaiveBayesSuite._ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @@ -38,6 +38,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var dataset: Dataset[_] = _ @transient var bernoulliDataset: Dataset[_] = _ + @transient var gaussianDataset: Dataset[_] = _ + @transient var gaussianDataset2: Dataset[_] = _ private val seed = 42 @@ -53,6 +55,23 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { dataset = generateNaiveBayesInput(pi, theta, 100, seed).toDF() bernoulliDataset = generateNaiveBayesInput(pi, theta, 100, seed, "bernoulli").toDF() + + // theta for gaussian nb + val theta2 = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + // sigma for gaussian nb + val sigma = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() + gaussianDataset2 = spark.read.format("libsvm") + .load("../data/mllib/sample_multiclass_classification_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -67,10 +86,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { def validateModelFit( piData: Vector, thetaData: Matrix, + sigmaData: Matrix, model: NaiveBayesModel): Unit = { assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~== Vectors.dense(piData.toArray.map(math.exp)) absTol 0.05, "pi mismatch") assert(model.theta.map(math.exp) ~== thetaData.map(math.exp) absTol 0.05, "theta mismatch") + if (sigmaData == null) { + assert(model.sigma == null, "sigma mismatch") + } else { + assert(model.sigma.map(math.exp) ~== sigmaData.map(math.exp) absTol 0.05, + "sigma mismatch") + } } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { @@ -90,6 +116,19 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Vectors.dense(classProbs.map(_ / classProbsSum)) } + def expectedGaussianProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { + val pi = model.pi.toArray.map(math.exp) + val classProbs = pi.indices.map { i => + feature.toArray.zipWithIndex.map { case (v, j) => + val mean = model.theta(i, j) + val variance = model.sigma(i, j) + math.exp(- (v - mean) * (v - mean) / variance / 2) / math.sqrt(variance * math.Pi * 2) + }.product * pi(i) + }.toArray + val classProbsSum = classProbs.sum + Vectors.dense(classProbs.map(_ / classProbsSum)) + } + def validateProbabilities( featureAndProbabilities: Seq[Row], model: NaiveBayesModel, @@ -102,6 +141,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { expectedMultinomialProbabilities(model, features) case Bernoulli => expectedBernoulliProbabilities(model, features) + case Gaussian => + expectedGaussianProbabilities(model, features) case _ => throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } @@ -112,12 +153,14 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("model types") { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") + assert(Gaussian === "gaussian") } test("params") { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), - theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4))) + theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4)), + sigma = null) ParamsSuite.checkParams(model) } @@ -146,7 +189,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) MLTestingUtils.checkCopyAndUids(nb, model) @@ -192,12 +235,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("Naive Bayes with weighted samples") { val numClasses = 3 def modelEquals(m1: NaiveBayesModel, m2: NaiveBayesModel): Unit = { + assert(m1.getModelType === m2.getModelType) assert(m1.pi ~== m2.pi relTol 0.01) assert(m1.theta ~== m2.theta relTol 0.01) + if (m1.getModelType == Gaussian) { + assert(m1.sigma ~== m2.sigma relTol 0.01) + } } val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), - ("multinomial", dataset) + ("multinomial", dataset), + ("gaussian", gaussianDataset) ) testParams.foreach { case (family, dataset) => // NaiveBayes is sensitive to constant scaling of the weights unless smoothing is set to 0 @@ -228,7 +276,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) val validationDataset = @@ -308,14 +356,112 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { } } + test("Naive Bayes Gaussian") { + val piArray = Array(0.5, 0.1, 0.4).map(math.log) + + val thetaArray = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + val sigmaArray = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) + val sigma = new DenseMatrix(3, 4, sigmaArray.flatten, true) + + val nPoints = 10000 + val testDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 42).toDF() + val gnb = new NaiveBayes().setModelType("gaussian") + val model = gnb.fit(testDataset) + + validateModelFit(pi, theta, sigma, model) + assert(model.hasParent) + + val validationDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 17).toDF() + + val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") + validatePrediction(predictionAndLabels.collect()) + + val featureAndProbabilities = model.transform(validationDataset) + .select("features", "probability") + validateProbabilities(featureAndProbabilities.collect(), model, "gaussian") + } + + test("Naive Bayes Gaussian - Model Coefficients") { + /* + Using the following Python code to verify the correctness. + + import numpy as np + from sklearn.naive_bayes import GaussianNB + from sklearn.datasets import load_svmlight_file + + path = "./data/mllib/sample_multiclass_classification_data.txt" + X, y = load_svmlight_file(path) + X = X.toarray() + clf = GaussianNB() + clf.fit(X, y) + + >>> clf.class_prior_ + array([0.33333333, 0.33333333, 0.33333333]) + >>> clf.theta_ + array([[ 0.27111101, -0.18833335, 0.54305072, 0.60500005], + [-0.60777778, 0.18166667, -0.84271174, -0.88000014], + [-0.09111114, -0.35833336, 0.10508474, 0.0216667 ]]) + >>> clf.sigma_ + array([[0.12230125, 0.07078052, 0.03430001, 0.05133607], + [0.03758145, 0.0988028 , 0.0033903 , 0.00782224], + [0.08058764, 0.06701387, 0.02486641, 0.02661392]]) + */ + + val gnb = new NaiveBayes().setModelType(Gaussian) + val model = gnb.fit(gaussianDataset2) + assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~= + Vectors.dense(0.33333333, 0.33333333, 0.33333333) relTol 1E-5) + + val thetaRows = model.theta.rowIter.toArray + assert(thetaRows(0) ~= + Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) + assert(thetaRows(1) ~= + Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) + assert(thetaRows(2) ~= + Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) + + val sigmaRows = model.sigma.rowIter.toArray + assert(sigmaRows(0) ~= + Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) + assert(sigmaRows(1) ~= + Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) + assert(sigmaRows(2) ~= + Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) + } + test("read/write") { def checkModelData(model: NaiveBayesModel, model2: NaiveBayesModel): Unit = { + assert(model.getModelType === model2.getModelType) assert(model.pi === model2.pi) assert(model.theta === model2.theta) + if (model.getModelType == "gaussian") { + assert(model.sigma === model2.sigma) + } else { + assert(model.sigma === null && model2.sigma === null) + } } val nb = new NaiveBayes() testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, NaiveBayesSuite.allParamSettings, checkModelData) + + val gnb = new NaiveBayes().setModelType("gaussian") + testEstimatorAndModelReadWrite(gnb, gaussianDataset, + NaiveBayesSuite.allParamSettingsForGaussian, + NaiveBayesSuite.allParamSettingsForGaussian, checkModelData) } test("should support all NumericType labels and weights, and not support other types") { @@ -324,6 +470,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) + assert(expected.sigma === null && actual.sigma === null) } } } @@ -340,6 +487,16 @@ object NaiveBayesSuite { "smoothing" -> 0.1 ) + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettingsForGaussian: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "modelType" -> "gaussian" + ) + private def calcLabel(p: Double, pi: Array[Double]): Int = { var sum = 0.0 for (j <- 0 until pi.length) { @@ -384,4 +541,26 @@ object NaiveBayesSuite { LabeledPoint(y, Vectors.dense(xi)) } } + + // Generate input + def generateGaussianNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + sigma: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.exp) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = Array.tabulate[Double] (D) { j => + val mean = theta(y)(j) + val variance = sigma(y)(j) + mean + rnd.nextGaussian() * math.sqrt(variance) + } + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8830061c0d9ed..617eb173f4f49 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -118,6 +118,9 @@ object MimaExcludes { // [SPARK-26632][Core] Separate Thread Configurations of Driver and Executor ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.SparkTransportConf.fromSparkConf"), + // [SPARK-16872][ML][PYSPARK] Impl Gaussian Naive Bayes Classifier + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + // [SPARK-25765][ML] Add training cost to BisectingKMeans summary ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.this"), diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 66b8ffd7bc8d8..03087c34b9d0b 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1878,7 +1878,8 @@ class _NaiveBayesParams(_JavaPredictorParams, HasWeightCol): smoothing = Param(Params._dummy(), "smoothing", "The smoothing parameter, should be >= 0, " + "default is 1.0", typeConverter=TypeConverters.toFloat) modelType = Param(Params._dummy(), "modelType", "The model type which is a string " + - "(case-sensitive). Supported options: multinomial (default) and bernoulli.", + "(case-sensitive). Supported options: multinomial (default), bernoulli " + + "and gaussian.", typeConverter=TypeConverters.toString) @since("1.5.0") @@ -1907,7 +1908,10 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, TF-IDF vectors, it can be used for document classification. By making every vector a binary (0/1) data, it can also be used as `Bernoulli NB `_. - The input feature values must be nonnegative. + The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + Since 3.0.0, it also supports Gaussian NB + `_. + which can handle continuous data. >>> from pyspark.sql import Row >>> from pyspark.ml.linalg import Vectors @@ -1925,6 +1929,8 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, DenseVector([-0.81..., -0.58...]) >>> model.theta DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) + >>> model.sigma == None + True >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() >>> model.predict(test0.head().features) 1.0 @@ -1955,6 +1961,12 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, >>> result = model3.transform(test0).head() >>> result.prediction 0.0 + >>> nb3 = NaiveBayes().setModelType("gaussian") + >>> model4 = nb3.fit(df) + >>> model4.getModelType() + 'gaussian' + >>> model4.sigma + DenseMatrix(2, 2, [0.0, 0.25, 0.0, 0.0], 1) .. versionadded:: 1.5.0 """ @@ -2037,6 +2049,14 @@ def theta(self): """ return self._call_java("theta") + @property + @since("3.0.0") + def sigma(self): + """ + variance of each feature. + """ + return self._call_java("sigma") + class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, HasTol, HasStepSize, HasSolver): From d83cacfcf5510e0b466bbe17459811bb42d72250 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Mon, 18 Nov 2019 11:48:56 +0800 Subject: [PATCH 22/83] [SPARK-29907][SQL] Move DELETE/UPDATE/MERGE relative rules to dmlStatementNoWith to support cte ### What changes were proposed in this pull request? SPARK-27444 introduced `dmlStatementNoWith` so that any dml that needs cte support can leverage it. It be better if we move DELETE/UPDATE/MERGE rules to `dmlStatementNoWith`. ### Why are the changes needed? Wit this change, we can support syntax like "With t AS (SELECT) DELETE FROM xxx", and so as UPDATE/MERGE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New cases added. Closes #26536 from xianyinxin/SPARK-29907. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 16 +-- .../command/PlanResolutionSuite.scala | 119 ++++++++++++++++-- 2 files changed, 116 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d2d145606b627..c062377ee02a9 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -218,14 +218,6 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration - | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable - | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable - | MERGE INTO target=multipartIdentifier targetAlias=tableAlias - USING (source=multipartIdentifier | - '(' sourceQuery=query')') sourceAlias=tableAlias - ON mergeCondition=booleanExpression - matchedClause* - notMatchedClause* #mergeIntoTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -402,6 +394,14 @@ resource dmlStatementNoWith : insertInto queryTerm queryOrganization #singleInsertQuery | fromClause multiInsertQueryBody+ #multiInsertQuery + | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable + | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable + | MERGE INTO target=multipartIdentifier targetAlias=tableAlias + USING (source=multipartIdentifier | + '(' sourceQuery=query')') sourceAlias=tableAlias + ON mergeCondition=booleanExpression + matchedClause* + notMatchedClause* #mergeIntoTable ; queryOrganization 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 50e35289b0a64..d5c65a7c1e1b6 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 @@ -26,11 +26,11 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{EqualTo, IntegerLiteral, StringLiteral} +import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -138,6 +138,7 @@ class PlanResolutionSuite extends AnalysisTest { } val analyzer = new Analyzer(catalogManager, conf) val rules = Seq( + CTESubstitution, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), analyzer.ResolveTables) @@ -859,10 +860,16 @@ class PlanResolutionSuite extends AnalysisTest { val sql1 = s"DELETE FROM $tblName" val sql2 = s"DELETE FROM $tblName where name='Robert'" val sql3 = s"DELETE FROM $tblName AS t where t.name='Robert'" + val sql4 = + s""" + |WITH s(name) AS (SELECT 'Robert') + |DELETE FROM $tblName AS t WHERE t.name IN (SELECT s.name FROM s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) + val parsed4 = parseAndResolve(sql4) parsed1 match { case DeleteFromTable(_: DataSourceV2Relation, None) => @@ -874,7 +881,7 @@ class PlanResolutionSuite extends AnalysisTest { _: DataSourceV2Relation, Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed2.treeString) } parsed3 match { @@ -882,7 +889,24 @@ class PlanResolutionSuite extends AnalysisTest { SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "t.name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed3.treeString) + } + + parsed4 match { + case DeleteFromTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Some(InSubquery(values, query))) => + assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) + assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") + query match { + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), + _, _, _) => + assert(projects.size == 1 && projects.head.name == "s.name") + assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") + case o => fail("Unexpected subquery: \n" + o.treeString) + } + + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed4.treeString) } } } @@ -892,10 +916,18 @@ class PlanResolutionSuite extends AnalysisTest { val sql1 = s"UPDATE $tblName SET name='Robert', age=32" val sql2 = s"UPDATE $tblName AS t SET name='Robert', age=32" val sql3 = s"UPDATE $tblName AS t SET name='Robert', age=32 WHERE p=1" + val sql4 = + s""" + |WITH s(name) AS (SELECT 'Robert') + |UPDATE $tblName AS t + |SET t.age=32 + |WHERE t.name IN (SELECT s.name FROM s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) + val parsed4 = parseAndResolve(sql4) parsed1 match { case UpdateTable( @@ -933,6 +965,25 @@ class PlanResolutionSuite extends AnalysisTest { case _ => fail("Expect UpdateTable, but got:\n" + parsed3.treeString) } + + parsed4 match { + case UpdateTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Seq(Assignment(key: UnresolvedAttribute, IntegerLiteral(32))), + Some(InSubquery(values, query))) => + assert(key.name == "t.age") + assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) + assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") + query match { + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), + _, _, _) => + assert(projects.size == 1 && projects.head.name == "s.name") + assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") + case o => fail("Unexpected subquery: \n" + o.treeString) + } + + case _ => fail("Expect UpdateTable, but got:\n" + parsed4.treeString) + } } val sql = "UPDATE non_existing SET id=1" @@ -1015,8 +1066,7 @@ class PlanResolutionSuite extends AnalysisTest { test("MERGE INTO TABLE") { Seq(("v2Table", "v2Table1"), ("testcat.tab", "testcat.tab1")).foreach { - case(target, source) => - + case(target, source) => // basic val sql1 = s""" @@ -1059,11 +1109,25 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN NOT MATCHED AND (target.s='insert') | THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin + // cte + val sql5 = + s""" + |WITH source(i, s) AS + | (SELECT * FROM $source) + |MERGE INTO $target AS target + |USING source + |ON target.i = source.i + |WHEN MATCHED AND (target.s='delete') THEN DELETE + |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s + |WHEN NOT MATCHED AND (target.s='insert') + |THEN INSERT (target.i, target.s) values (source.i, source.s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) val parsed4 = parseAndResolve(sql4) + val parsed5 = parseAndResolve(sql5) parsed1 match { case MergeIntoTable( @@ -1090,7 +1154,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed1.treeString) } parsed2 match { @@ -1130,7 +1194,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed3.treeString) } parsed4 match { @@ -1157,8 +1221,41 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) - } + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed4.treeString) + } + + parsed5 match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, + Project(projects, _: DataSourceV2Relation))), + EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), + Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), + updateAssigns)), + Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), + insertAssigns))) => + assert(outputColumnNames.size == 2 && + outputColumnNames.head == "i" && + outputColumnNames.last == "s") + assert(projects.size == 1 && projects.head.isInstanceOf[UnresolvedStar]) + assert(l.name == "target.i" && r.name == "source.i") + assert(dl.name == "target.s") + assert(ul.name == "target.s") + assert(il.name == "target.s") + assert(updateAssigns.size == 1) + assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && + updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") + assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && + updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") + assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && + insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") + assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && + insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") + + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed5.treeString) + } } // no aliases From f280c6aa54d80251da66ab370d32a7d93b01f225 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 18 Nov 2019 12:54:21 +0900 Subject: [PATCH 23/83] [SPARK-29378][R][FOLLOW-UP] Remove manual installation of Arrow dependencies in AppVeyor build ### What changes were proposed in this pull request? This PR remove manual installation of Arrow dependencies in AppVeyor build ### Why are the changes needed? It's unnecessary. See https://github.com/apache/spark/pull/26555#discussion_r347178368 ### Does this PR introduce any user-facing change? No ### How was this patch tested? AppVeyor will test. Closes #26566 from HyukjinKwon/SPARK-29378. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- appveyor.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/appveyor.yml b/appveyor.yml index c3baa1f82a399..00c688ba18eb6 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -43,7 +43,6 @@ install: - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. From 42f8f79ff0311f7de629177b8609b2cbc73ad1c4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 17 Nov 2019 21:01:01 -0800 Subject: [PATCH 24/83] [SPARK-29936][R] Fix SparkR lint errors and add lint-r GitHub Action ### What changes were proposed in this pull request? This PR fixes SparkR lint errors and adds `lint-r` GitHub Action to protect the branch. ### Why are the changes needed? It turns out that we currently don't run it. It's recovered yesterday. However, after that, our Jenkins linter jobs (`master`/`branch-2.4`) has been broken on `lint-r` tasks. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the GitHub Action on this PR in addition to Jenkins R and AppVeyor R. Closes #26564 from dongjoon-hyun/SPARK-29936. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 25 ++++++++++++++++++++++++- R/pkg/.lintr | 2 +- R/pkg/R/DataFrame.R | 8 ++++---- R/pkg/R/SQLContext.R | 4 ++-- R/pkg/R/context.R | 2 +- R/pkg/R/group.R | 2 +- R/pkg/R/utils.R | 6 +++--- R/pkg/inst/worker/worker.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 4 ++-- dev/lint-r.R | 2 +- 10 files changed, 40 insertions(+), 17 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5eb00c4aba0f9..d2b7dca3684f0 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -50,7 +50,7 @@ jobs: lint: runs-on: ubuntu-latest - name: Linters + name: Linters (Java/Scala/Python), licenses, dependencies steps: - uses: actions/checkout@master - uses: actions/setup-java@v1 @@ -72,3 +72,26 @@ jobs: run: ./dev/check-license - name: Dependencies run: ./dev/test-dependencies.sh + + lintr: + runs-on: ubuntu-latest + name: Linter (R) + steps: + - uses: actions/checkout@master + - uses: actions/setup-java@v1 + with: + java-version: '11' + - name: install R + run: | + echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list + sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + - name: install R packages + run: | + sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" + - name: package and install SparkR + run: ./R/install-dev.sh + - name: lint-r + run: ./dev/lint-r diff --git a/R/pkg/.lintr b/R/pkg/.lintr index c83ad2adfe0ef..67dc1218ea551 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) +linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE), object_usage_linter = NULL, cyclocomp_linter = NULL) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 6f3c7c120ba3c..593d3ca16220d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2252,7 +2252,7 @@ setMethod("mutate", # The last column of the same name in the specific columns takes effect deDupCols <- list() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { deDupCols[[ns[[i]]]] <- alias(cols[[i]], ns[[i]]) } @@ -2416,7 +2416,7 @@ setMethod("arrange", # builds a list of columns of type Column # example: [[1]] Column Species ASC # [[2]] Column Petal_Length DESC - jcols <- lapply(seq_len(length(decreasing)), function(i){ + jcols <- lapply(seq_len(length(decreasing)), function(i) { if (decreasing[[i]]) { desc(getColumn(x, by[[i]])) } else { @@ -2749,7 +2749,7 @@ genAliasesForIntersectedCols <- function(x, intersectedColNames, suffix) { col <- getColumn(x, colName) if (colName %in% intersectedColNames) { newJoin <- paste(colName, suffix, sep = "") - if (newJoin %in% allColNames){ + if (newJoin %in% allColNames) { stop("The following column name: ", newJoin, " occurs more than once in the 'DataFrame'.", "Please use different suffixes for the intersected columns.") } @@ -3475,7 +3475,7 @@ setMethod("str", cat(paste0("'", class(object), "': ", length(names), " variables:\n")) if (nrow(localDF) > 0) { - for (i in 1 : ncol(localDF)) { + for (i in seq_len(ncol(localDF))) { # Get the first elements for each column firstElements <- if (types[i] == "character") { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index cf96c658d34d2..f48a334ed6766 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -197,7 +197,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { as.list(schema) } if (is.null(names)) { - names <- lapply(1:length(firstRow), function(x) { + names <- lapply(seq_len(length(firstRow)), function(x) { paste0("_", as.character(x)) }) } @@ -213,7 +213,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { }) types <- lapply(firstRow, infer_type) - fields <- lapply(1:length(firstRow), function(i) { + fields <- lapply(seq_len(length(firstRow)), function(i) { structField(names[[i]], types[[i]], TRUE) }) schema <- do.call(structType, fields) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 93ba1307043a3..d96a287f818a2 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -416,7 +416,7 @@ spark.getSparkFiles <- function(fileName) { #' @examples #'\dontrun{ #' sparkR.session() -#' doubled <- spark.lapply(1:10, function(x){2 * x}) +#' doubled <- spark.lapply(1:10, function(x) {2 * x}) #'} #' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 6e8f4dc3a7907..2b7995e1e37f6 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -162,7 +162,7 @@ methods <- c("avg", "max", "mean", "min", "sum") #' @note pivot since 2.0.0 setMethod("pivot", signature(x = "GroupedData", colname = "character"), - function(x, colname, values = list()){ + function(x, colname, values = list()) { stopifnot(length(colname) == 1) if (length(values) == 0) { result <- callJMethod(x@sgd, "pivot", colname) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c3501977e64bc..15e2410d3a31c 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -131,7 +131,7 @@ hashCode <- function(key) { } else { asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) hashC <- 0 - for (k in 1:length(asciiVals)) { + for (k in seq_len(length(asciiVals))) { hashC <- mult31AndAdd(hashC, asciiVals[k]) } as.integer(hashC) @@ -724,7 +724,7 @@ assignNewEnv <- function(data) { stopifnot(length(cols) > 0) env <- new.env() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { assign(x = cols[i], value = data[, cols[i], drop = F], envir = env) } env @@ -750,7 +750,7 @@ launchScript <- function(script, combinedArgs, wait = FALSE, stdout = "", stderr if (.Platform$OS.type == "windows") { scriptWithArgs <- paste(script, combinedArgs, sep = " ") # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) - shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) } else { # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html # stdout = F means discard output diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index dfe69b7f4f1fb..1ef05ea621e83 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -194,7 +194,7 @@ if (isEmpty != 0) { } else { # gapply mode outputs <- list() - for (i in 1:length(data)) { + for (i in seq_len(length(data))) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index c2b2458ec064b..cb47353d600db 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -172,7 +172,7 @@ test_that("structField type strings", { typeList <- c(primitiveTypes, complexTypes) typeStrings <- names(typeList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- typeList[[i]] testField <- structField("_col", typeString) @@ -203,7 +203,7 @@ test_that("structField type strings", { errorList <- c(primitiveErrors, complexErrors) typeStrings <- names(errorList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- paste0("Unsupported type for SparkDataframe: ", errorList[[i]]) expect_error(structField("_col", typeString), expected) diff --git a/dev/lint-r.R b/dev/lint-r.R index a4261d266bbc0..7e165319e316a 100644 --- a/dev/lint-r.R +++ b/dev/lint-r.R @@ -27,7 +27,7 @@ if (! library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE)) { # Installs lintr from Github in a local directory. # NOTE: The CRAN's version is too old to adapt to our rules. if ("lintr" %in% row.names(installed.packages()) == FALSE) { - devtools::install_github("jimhester/lintr@5431140") + devtools::install_github("jimhester/lintr@v2.0.0") } library(lintr) From ee3bd6d76887ccc4961fd520c5d03f7edd3742ac Mon Sep 17 00:00:00 2001 From: Zhou Jiang Date: Mon, 18 Nov 2019 05:44:00 +0000 Subject: [PATCH 25/83] [SPARK-25694][SQL] Add a config for `URL.setURLStreamHandlerFactory` ### What changes were proposed in this pull request? Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory` ### Why are the changes needed? This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](https://github.com/apache/spark/pull/17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users. ### Does this PR introduce any user-facing change? Yes. This provides a new user-configurable property. By default, the behavior is unchanged. ### How was this patch tested? Manual testing. **BEFORE** ``` $ build/sbt package $ bin/spark-shell scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) java.lang.Error: factory already defined at java.net.URL.setURLStreamHandlerFactory(URL.java:1134) ... 47 elided ``` **AFTER** ``` $ build/sbt package $ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) ``` Closes #26530 from jiangzho/master. Lead-authored-by: Zhou Jiang Co-authored-by: Dongjoon Hyun Co-authored-by: zhou-jiang Signed-off-by: DB Tsai --- .../spark/sql/internal/SharedState.scala | 25 ++++++++++++---- .../spark/sql/internal/config/package.scala | 29 +++++++++++++++++++ 2 files changed, 49 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index b810bedac471d..81a9c76511d8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -52,6 +53,8 @@ private[sql] class SharedState( initialConfigs: scala.collection.Map[String, String]) extends Logging { + SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf) + // Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on // the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. val warehousePath: String = { @@ -191,11 +194,23 @@ private[sql] class SharedState( } object SharedState extends Logging { - try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) - } catch { - case e: Error => - logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") + @volatile private var fsUrlStreamHandlerFactoryInitialized = false + + private def setFsUrlStreamHandlerFactory(conf: SparkConf): Unit = { + if (!fsUrlStreamHandlerFactoryInitialized && + conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { + synchronized { + if (!fsUrlStreamHandlerFactoryInitialized) { + try { + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + fsUrlStreamHandlerFactoryInitialized = true + } catch { + case NonFatal(_) => + logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") + } + } + } + } } private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala new file mode 100644 index 0000000000000..e26c4aadaf135 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.internal + +import org.apache.spark.internal.config.ConfigBuilder + +package object config { + + private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = + ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .booleanConf + .createWithDefault(true) +} From 73912379d06e49e29aa8192ac4fb746d73c97fc5 Mon Sep 17 00:00:00 2001 From: gschiavon Date: Mon, 18 Nov 2019 16:07:05 +0900 Subject: [PATCH 26/83] [SPARK-29020][SQL] Improving array_sort behaviour ### What changes were proposed in this pull request? I've noticed that there are two functions to sort arrays sort_array and array_sort. sort_array is from 1.5.0 and it has the possibility of ordering both ascending and descending array_sort is from 2.4.0 and it only has the possibility of ordering in ascending. Basically I just added the possibility of ordering either ascending or descending using array_sort. I think it would be good to have unified behaviours and not having to user sort_array when you want to order in descending order. Imagine that you are new to spark, I'd like to be able to sort array using the newest spark functions. ### Why are the changes needed? Basically to be able to sort the array in descending order using *array_sort* instead of using *sort_array* from 1.5.0 ### Does this PR introduce any user-facing change? Yes, now you are able to sort the array in descending order. Note that it has the same behaviour with nulls than sort_array ### How was this patch tested? Test's added This is the link to the [jira](https://issues.apache.org/jira/browse/SPARK-29020) Closes #25728 from Gschiavon/improving-array-sort. Lead-authored-by: gschiavon Co-authored-by: Takuya UESHIN Co-authored-by: gschiavon Signed-off-by: HyukjinKwon --- .../expressions/collectionOperations.scala | 48 -------- .../expressions/higherOrderFunctions.scala | 108 ++++++++++++++++++ .../CollectionExpressionsSuite.scala | 10 -- .../HigherOrderFunctionsSuite.scala | 50 ++++++++ .../org/apache/spark/sql/functions.scala | 2 +- .../spark/sql/DataFrameFunctionsSuite.scala | 82 ++++++++++++- 6 files changed, 240 insertions(+), 60 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 5d964b602e634..d5d42510842ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -900,54 +900,6 @@ case class SortArray(base: Expression, ascendingOrder: Expression) override def prettyName: String = "sort_array" } - -/** - * Sorts the input array in ascending order according to the natural ordering of - * the array elements and returns it. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = """ - _FUNC_(array) - Sorts the input array in ascending order. The elements of the input array must - be orderable. Null elements will be placed at the end of the returned array. - """, - examples = """ - Examples: - > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); - ["a","b","c","d",null] - """, - since = "2.4.0") -// scalastyle:on line.size.limit -case class ArraySort(child: Expression) extends UnaryExpression with ArraySortLike { - - override def dataType: DataType = child.dataType - override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType) - - override def arrayExpression: Expression = child - override def nullOrder: NullOrder = NullOrder.Greatest - - override def checkInputDataTypes(): TypeCheckResult = child.dataType match { - case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => - TypeCheckResult.TypeCheckSuccess - case ArrayType(dt, _) => - val dtSimple = dt.catalogString - TypeCheckResult.TypeCheckFailure( - s"$prettyName does not support sorting array of type $dtSimple which is not orderable") - case _ => - TypeCheckResult.TypeCheckFailure(s"$prettyName only supports array input.") - } - - override def nullSafeEval(array: Any): Any = { - sortEval(array, true) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, c => sortCodegen(ctx, ev, c, "true")) - } - - override def prettyName: String = "array_sort" -} - /** * Returns a random permutation of the given array. */ 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 317ebb62c07ec..adeda0981fe8e 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Comparator import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable @@ -285,6 +286,113 @@ case class ArrayTransform( override def prettyName: String = "transform" } +/** + * Sorts elements in an array using a comparator function. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """_FUNC_(expr, func) - Sorts the input array in ascending order. The elements of the + input array must be orderable. Null elements will be placed at the end of the returned + array. Since 3.0.0 this function also sorts and returns the array based on the given + comparator function. The comparator will take two arguments + representing two elements of the array. + It returns -1, 0, or 1 as the first element is less than, equal to, or greater + than the second element. If the comparator function returns other + values (including null), the function will fail and raise an error. + """, + examples = """ + Examples: + > SELECT _FUNC_(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end); + [1,5,6] + > SELECT _FUNC_(array('bc', 'ab', 'dc'), (left, right) -> case when left is null and right is null then 0 when left is null then -1 when right is null then 1 when left < right then 1 when left > right then -1 else 0 end); + ["dc","bc","ab"] + > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); + ["a","b","c","d",null] + """, + since = "2.4.0") +// scalastyle:on line.size.limit +case class ArraySort( + argument: Expression, + function: Expression) + extends ArrayBasedSimpleHigherOrderFunction with CodegenFallback { + + def this(argument: Expression) = this(argument, ArraySort.defaultComparator) + + @transient lazy val elementType: DataType = + argument.dataType.asInstanceOf[ArrayType].elementType + + override def dataType: ArrayType = argument.dataType.asInstanceOf[ArrayType] + override def checkInputDataTypes(): TypeCheckResult = { + checkArgumentDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => + argument.dataType match { + case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => + if (function.dataType == IntegerType) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure("Return type of the given function has to be " + + "IntegerType") + } + case ArrayType(dt, _) => + val dtSimple = dt.catalogString + TypeCheckResult.TypeCheckFailure( + s"$prettyName does not support sorting array of type $dtSimple which is not " + + "orderable") + case _ => + TypeCheckResult.TypeCheckFailure(s"$prettyName only supports array input.") + } + case failure => failure + } + } + + override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArraySort = { + val ArrayType(elementType, containsNull) = argument.dataType + copy(function = + f(function, (elementType, containsNull) :: (elementType, containsNull) :: Nil)) + } + + @transient lazy val LambdaFunction(_, + Seq(firstElemVar: NamedLambdaVariable, secondElemVar: NamedLambdaVariable), _) = function + + def comparator(inputRow: InternalRow): Comparator[Any] = { + val f = functionForEval + (o1: Any, o2: Any) => { + firstElemVar.value.set(o1) + secondElemVar.value.set(o2) + f.eval(inputRow).asInstanceOf[Int] + } + } + + override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { + val arr = argumentValue.asInstanceOf[ArrayData].toArray[AnyRef](elementType) + if (elementType != NullType) { + java.util.Arrays.sort(arr, comparator(inputRow)) + } + new GenericArrayData(arr.asInstanceOf[Array[Any]]) + } + + override def prettyName: String = "array_sort" +} + +object ArraySort { + + def comparator(left: Expression, right: Expression): Expression = { + val lit0 = Literal(0) + val lit1 = Literal(1) + val litm1 = Literal(-1) + + If(And(IsNull(left), IsNull(right)), lit0, + If(IsNull(left), lit1, If(IsNull(right), litm1, + If(LessThan(left, right), litm1, If(GreaterThan(left, right), lit1, lit0))))) + } + + val defaultComparator: LambdaFunction = { + val left = UnresolvedNamedLambdaVariable(Seq("left")) + val right = UnresolvedNamedLambdaVariable(Seq("right")) + LambdaFunction(comparator(left, right), Seq(left, right)) + } +} + /** * Filters entries in a map using the provided function. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 3287c83b1dd87..dd6d437d8063e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -364,16 +364,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val arrayArrayStruct = Literal.create(Seq(aas2, aas1), typeAAS) checkEvaluation(new SortArray(arrayArrayStruct), Seq(aas1, aas2)) - - checkEvaluation(ArraySort(a0), Seq(1, 2, 3)) - checkEvaluation(ArraySort(a1), Seq[Integer]()) - checkEvaluation(ArraySort(a2), Seq("a", "b")) - checkEvaluation(ArraySort(a3), Seq("a", "b", null)) - checkEvaluation(ArraySort(a4), Seq(d1, d2)) - checkEvaluation(ArraySort(a5), Seq(null, null)) - checkEvaluation(ArraySort(arrayStruct), Seq(create_row(1), create_row(2))) - checkEvaluation(ArraySort(arrayArray), Seq(aa1, aa2)) - checkEvaluation(ArraySort(arrayArrayStruct), Seq(aas1, aas2)) } test("Array contains") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index 4cdee447fa45a..e7b713840b884 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -84,6 +84,15 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayTransform(expr, createLambda(et, cn, IntegerType, false, f)).bind(validateBinding) } + def arraySort(expr: Expression): Expression = { + arraySort(expr, ArraySort.comparator) + } + + def arraySort(expr: Expression, f: (Expression, Expression) => Expression): Expression = { + val ArrayType(et, cn) = expr.dataType + ArraySort(expr, createLambda(et, cn, et, cn, f)).bind(validateBinding) + } + def filter(expr: Expression, f: Expression => Expression): Expression = { val ArrayType(et, cn) = expr.dataType ArrayFilter(expr, createLambda(et, cn, f)).bind(validateBinding) @@ -167,6 +176,47 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq("[1, 3, 5]", null, "[4, 6]")) } + test("ArraySort") { + val a0 = Literal.create(Seq(2, 1, 3), ArrayType(IntegerType)) + val a1 = Literal.create(Seq[Integer](), ArrayType(IntegerType)) + val a2 = Literal.create(Seq("b", "a"), ArrayType(StringType)) + val a3 = Literal.create(Seq("b", null, "a"), ArrayType(StringType)) + val d1 = new Decimal().set(10) + val d2 = new Decimal().set(100) + val a4 = Literal.create(Seq(d2, d1), ArrayType(DecimalType(10, 0))) + val a5 = Literal.create(Seq(null, null), ArrayType(NullType)) + + val typeAS = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) + val arrayStruct = Literal.create(Seq(create_row(2), create_row(1)), typeAS) + + val typeAA = ArrayType(ArrayType(IntegerType)) + val aa1 = Array[java.lang.Integer](1, 2) + val aa2 = Array[java.lang.Integer](3, null, 4) + val arrayArray = Literal.create(Seq(aa2, aa1), typeAA) + + val typeAAS = ArrayType(ArrayType(StructType(StructField("a", IntegerType) :: Nil))) + val aas1 = Array(create_row(1)) + val aas2 = Array(create_row(2)) + val arrayArrayStruct = Literal.create(Seq(aas2, aas1), typeAAS) + + checkEvaluation(arraySort(a0), Seq(1, 2, 3)) + checkEvaluation(arraySort(a1), Seq[Integer]()) + checkEvaluation(arraySort(a2), Seq("a", "b")) + checkEvaluation(arraySort(a3), Seq("a", "b", null)) + checkEvaluation(arraySort(a4), Seq(d1, d2)) + checkEvaluation(arraySort(a5), Seq(null, null)) + checkEvaluation(arraySort(arrayStruct), Seq(create_row(1), create_row(2))) + checkEvaluation(arraySort(arrayArray), Seq(aa1, aa2)) + checkEvaluation(arraySort(arrayArrayStruct), Seq(aas1, aas2)) + + checkEvaluation(arraySort(a0, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(3, 2, 1)) + checkEvaluation(arraySort(a3, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(null, "b", "a")) + checkEvaluation(arraySort(a4, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(d2, d1)) + } + test("MapFilter") { def mapFilter(expr: Expression, f: (Expression, Expression) => Expression): Expression = { val MapType(kt, vt, vcn) = expr.dataType 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 2ba34647dbca8..72e9e337c4258 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 @@ -3335,7 +3335,7 @@ object functions { * @group collection_funcs * @since 2.4.0 */ - def array_sort(e: Column): Column = withExpr { ArraySort(e.expr) } + def array_sort(e: Column): Column = withExpr { new ArraySort(e.expr) } /** * Remove all elements that equal to element from the given array. 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 06484908f5e73..6c6d29a539197 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 @@ -312,6 +312,86 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) } + test("array_sort with lambda functions") { + + spark.udf.register("fAsc", (x: Int, y: Int) => { + if (x < y) -1 + else if (x == y) 0 + else 1 + }) + + spark.udf.register("fDesc", (x: Int, y: Int) => { + if (x < y) 1 + else if (x == y) 0 + else -1 + }) + + spark.udf.register("fString", (x: String, y: String) => { + if (x == null && y == null) 0 + else if (x == null) 1 + else if (y == null) -1 + else if (x < y) 1 + else if (x == y) 0 + else -1 + }) + + spark.udf.register("fStringLength", (x: String, y: String) => { + if (x == null && y == null) 0 + else if (x == null) 1 + else if (y == null) -1 + else if (x.length < y.length) -1 + else if (x.length == y.length) 0 + else 1 + }) + + val df1 = Seq(Array[Int](3, 2, 5, 1, 2)).toDF("a") + checkAnswer( + df1.selectExpr("array_sort(a, (x, y) -> fAsc(x, y))"), + Seq( + Row(Seq(1, 2, 2, 3, 5))) + ) + + checkAnswer( + df1.selectExpr("array_sort(a, (x, y) -> fDesc(x, y))"), + Seq( + Row(Seq(5, 3, 2, 2, 1))) + ) + + val df2 = Seq(Array[String]("bc", "ab", "dc")).toDF("a") + checkAnswer( + df2.selectExpr("array_sort(a, (x, y) -> fString(x, y))"), + Seq( + Row(Seq("dc", "bc", "ab"))) + ) + + val df3 = Seq(Array[String]("a", "abcd", "abc")).toDF("a") + checkAnswer( + df3.selectExpr("array_sort(a, (x, y) -> fStringLength(x, y))"), + Seq( + Row(Seq("a", "abc", "abcd"))) + ) + + val df4 = Seq((Array[Array[Int]](Array(2, 3, 1), Array(4, 2, 1, 4), + Array(1, 2)), "x")).toDF("a", "b") + checkAnswer( + df4.selectExpr("array_sort(a, (x, y) -> fAsc(cardinality(x), cardinality(y)))"), + Seq( + Row(Seq[Seq[Int]](Seq(1, 2), Seq(2, 3, 1), Seq(4, 2, 1, 4)))) + ) + + val df5 = Seq(Array[String]("bc", null, "ab", "dc")).toDF("a") + checkAnswer( + df5.selectExpr("array_sort(a, (x, y) -> fString(x, y))"), + Seq( + Row(Seq("dc", "bc", "ab", null))) + ) + + spark.sql("drop temporary function fAsc") + spark.sql("drop temporary function fDesc") + spark.sql("drop temporary function fString") + spark.sql("drop temporary function fStringLength") + } + test("sort_array/array_sort functions") { val df = Seq( (Array[Int](2, 1, 3), Array("b", "c", "a")), @@ -383,7 +463,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { assert(intercept[AnalysisException] { df3.selectExpr("array_sort(a)").collect() - }.getMessage().contains("only supports array input")) + }.getMessage().contains("argument 1 requires array type, however, '`a`' is of string type")) } def testSizeOfArray(sizeOfNull: Any): Unit = { From 5cebe587c7132fa6ea502084d45e0d8b203481b8 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 15:42:22 +0800 Subject: [PATCH 27/83] [SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table. Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval -- | -- | -- | -- | -- Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds SQL STANDARD | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06 ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S ### Why are the changes needed? for ANSI SQL support ### Does this PR introduce any user-facing change? yes,interval out now has 3 output styles ### How was this patch tested? add new unit tests cc cloud-fan maropu MaxGekk HyukjinKwon thanks. Closes #26418 from yaooqinn/SPARK-29783. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/CalendarInterval.java | 36 +----- .../unsafe/types/CalendarIntervalSuite.java | 30 ----- .../spark/sql/catalyst/expressions/Cast.scala | 18 +++ .../sql/catalyst/expressions/literals.scala | 1 + .../sql/catalyst/json/JacksonGenerator.scala | 11 +- .../sql/catalyst/util/IntervalUtils.scala | 106 ++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 19 ++++ .../catalyst/util/IntervalUtilsSuite.scala | 64 +++++++++++ .../spark/sql/execution/HiveResult.scala | 16 ++- .../inputs/interval-display-iso_8601.sql | 3 + .../inputs/interval-display-sql_standard.sql | 3 + .../sql-tests/inputs/interval-display.sql | 14 +++ .../sql-tests/inputs/postgreSQL/interval.sql | 10 +- .../results/interval-display-iso_8601.sql.out | 21 ++++ .../interval-display-sql_standard.sql.out | 21 ++++ .../results/interval-display.sql.out | 21 ++++ .../results/postgreSQL/interval.sql.out | 29 ++++- .../apache/spark/sql/DateFunctionsSuite.scala | 10 +- .../ThriftServerQueryTestSuite.scala | 5 +- 19 files changed, 361 insertions(+), 77 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display.sql.out diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index d2abb36b0727c..0464e886e1441 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; -import java.math.BigDecimal; import java.time.Duration; import java.time.Period; import java.time.temporal.ChronoUnit; @@ -80,39 +79,8 @@ public int compareTo(CalendarInterval that) { @Override public String toString() { - if (months == 0 && days == 0 && microseconds == 0) { - return "0 seconds"; - } - - StringBuilder sb = new StringBuilder(); - - if (months != 0) { - appendUnit(sb, months / 12, "years"); - appendUnit(sb, months % 12, "months"); - } - - appendUnit(sb, days, "days"); - - if (microseconds != 0) { - long rest = microseconds; - appendUnit(sb, rest / MICROS_PER_HOUR, "hours"); - rest %= MICROS_PER_HOUR; - appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes"); - rest %= MICROS_PER_MINUTE; - if (rest != 0) { - String s = BigDecimal.valueOf(rest, 6).stripTrailingZeros().toPlainString(); - sb.append(s).append(" seconds "); - } - } - - sb.setLength(sb.length() - 1); - return sb.toString(); - } - - private void appendUnit(StringBuilder sb, long value, String unit) { - if (value != 0) { - sb.append(value).append(' ').append(unit).append(' '); - } + return "CalendarInterval(months= " + months + ", days = " + days + ", microsecond = " + + microseconds + ")"; } /** diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 6397f26c02f3a..01bf7eb2438ad 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -46,36 +46,6 @@ public void equalsTest() { assertEquals(i1, i6); } - @Test - public void toStringTest() { - CalendarInterval i; - - i = new CalendarInterval(0, 0, 0); - assertEquals("0 seconds", i.toString()); - - i = new CalendarInterval(34, 0, 0); - assertEquals("2 years 10 months", i.toString()); - - i = new CalendarInterval(-34, 0, 0); - assertEquals("-2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 31, 0); - assertEquals("31 days", i.toString()); - - i = new CalendarInterval(0, -31, 0); - assertEquals("-31 days", i.toString()); - - i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString()); - - i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); - assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString()); - - i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds", - i.toString()); - } - @Test public void periodAndDurationTest() { CalendarInterval interval = new CalendarInterval(120, -40, 123456); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 9e802edbc5edd..ef99c8efef834 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -30,7 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -281,6 +283,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { + case CalendarIntervalType => SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toSqlStandardString(i))) + case ISO_8601 => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toIso8601String(i))) + case MULTI_UNITS => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toMultiUnitsString(i))) + } case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, @@ -985,6 +995,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit timestampFormatter.getClass) (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" + case CalendarIntervalType => + val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") + val funcName = SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => "toSqlStandardString" + case ISO_8601 => "toIso8601String" + case MULTI_UNITS => "toMultiUnitsString" + } + (c, evPrim, _) => code"""$evPrim = UTF8String.fromString($iu.$funcName($c));""" case ArrayType(et, _) => (c, evPrim, evNull) => { val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 5a5d7a17acd99..f426863de41a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -409,6 +409,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) s"TIMESTAMP('${formatter.format(v)}')" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" + case (v: CalendarInterval, CalendarIntervalType) => IntervalUtils.toMultiUnitsString(v) case _ => value.toString } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index aaf2ecf7923ce..544f6e9108325 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -119,6 +119,10 @@ private[sql] class JacksonGenerator( (row: SpecializedGetters, ordinal: Int) => gen.writeNumber(row.getDouble(ordinal)) + case CalendarIntervalType => + (row: SpecializedGetters, ordinal: Int) => + gen.writeString(IntervalUtils.toMultiUnitsString(row.getInterval(ordinal))) + case StringType => (row: SpecializedGetters, ordinal: Int) => gen.writeString(row.getUTF8String(ordinal).toString) @@ -214,10 +218,15 @@ private[sql] class JacksonGenerator( private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() + val keyString = mapType.keyType match { + case CalendarIntervalType => + (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) + case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString + } val valueArray = map.valueArray() var i = 0 while (i < map.numElements()) { - gen.writeFieldName(keyArray.get(i, mapType.keyType).toString) + gen.writeFieldName(keyString(i)) if (!valueArray.isNullAt(i)) { fieldWriter.apply(valueArray, i) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 991312bff30aa..104c65b6cd037 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.util +import java.math.BigDecimal import java.util.concurrent.TimeUnit import scala.util.control.NonFatal @@ -424,6 +425,111 @@ object IntervalUtils { fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num) } + def toMultiUnitsString(interval: CalendarInterval): String = { + if (interval.months == 0 && interval.days == 0 && interval.microseconds == 0) { + return "0 seconds" + } + val sb = new StringBuilder + if (interval.months != 0) { + appendUnit(sb, interval.months / 12, "years") + appendUnit(sb, interval.months % 12, "months") + } + appendUnit(sb, interval.days, "days") + if (interval.microseconds != 0) { + var rest = interval.microseconds + appendUnit(sb, rest / MICROS_PER_HOUR, "hours") + rest %= MICROS_PER_HOUR + appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes") + rest %= MICROS_PER_MINUTE + if (rest != 0) { + val s = BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString + sb.append(s).append(" seconds ") + } + } + sb.setLength(sb.length - 1) + sb.toString + } + + private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { + if (value != 0) sb.append(value).append(' ').append(unit).append(' ') + } + + def toSqlStandardString(interval: CalendarInterval): String = { + val yearMonthPart = if (interval.months < 0) { + val ma = math.abs(interval.months) + "-" + ma / 12 + "-" + ma % 12 + } else if (interval.months > 0) { + "+" + interval.months / 12 + "-" + interval.months % 12 + } else { + "" + } + + val dayPart = if (interval.days < 0) { + interval.days.toString + } else if (interval.days > 0) { + "+" + interval.days + } else { + "" + } + + val timePart = if (interval.microseconds != 0) { + val sign = if (interval.microseconds > 0) "+" else "-" + val sb = new StringBuilder(sign) + var rest = math.abs(interval.microseconds) + sb.append(rest / MICROS_PER_HOUR) + sb.append(':') + rest %= MICROS_PER_HOUR + val minutes = rest / MICROS_PER_MINUTE; + if (minutes < 10) { + sb.append(0) + } + sb.append(minutes) + sb.append(':') + rest %= MICROS_PER_MINUTE + val bd = BigDecimal.valueOf(rest, 6) + if (bd.compareTo(new BigDecimal(10)) < 0) { + sb.append(0) + } + val s = bd.stripTrailingZeros().toPlainString + sb.append(s) + sb.toString() + } else { + "" + } + + val intervalList = Seq(yearMonthPart, dayPart, timePart).filter(_.nonEmpty) + if (intervalList.nonEmpty) intervalList.mkString(" ") else "0" + } + + def toIso8601String(interval: CalendarInterval): String = { + val sb = new StringBuilder("P") + + val year = interval.months / 12 + if (year != 0) sb.append(year + "Y") + val month = interval.months % 12 + if (month != 0) sb.append(month + "M") + + if (interval.days != 0) sb.append(interval.days + "D") + + if (interval.microseconds != 0) { + sb.append('T') + var rest = interval.microseconds + val hour = rest / MICROS_PER_HOUR + if (hour != 0) sb.append(hour + "H") + rest %= MICROS_PER_HOUR + val minute = rest / MICROS_PER_MINUTE + if (minute != 0) sb.append(minute + "M") + rest %= MICROS_PER_MINUTE + if (rest != 0) { + val bd = BigDecimal.valueOf(rest, 6) + sb.append(bd.stripTrailingZeros().toPlainString + "S") + } + } else if (interval.days == 0 && interval.months == 0) { + sb.append("T0S") + } + sb.toString() + } + private object ParseState extends Enumeration { type ParseState = Value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8b714d6dcdda3..d4fcefe99ee75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1782,6 +1782,23 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) + object IntervalStyle extends Enumeration { + type IntervalStyle = Value + val SQL_STANDARD, ISO_8601, MULTI_UNITS = Value + } + + val INTERVAL_STYLE = buildConf("spark.sql.intervalOutputStyle") + .doc("When converting interval values to strings (i.e. for display), this config decides the" + + " interval string format. The value SQL_STANDARD will produce output matching SQL standard" + + " interval literals (i.e. '+3-2 +10 -00:00:01'). The value ISO_8601 will produce output" + + " matching the ISO 8601 standard (i.e. 'P3Y2M10DT-1S'). The value MULTI_UNITS (which is the" + + " default) will produce output in form of value unit pairs, (i.e. '3 year 2 months 10 days" + + " -1 seconds'") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(IntervalStyle.values.map(_.toString)) + .createWithDefault(IntervalStyle.MULTI_UNITS.toString) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2496,6 +2513,8 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: StoreAssignmentPolicy.Value = StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) + def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index f919bd1644871..65cfe896b2024 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -298,4 +298,68 @@ class IntervalUtilsSuite extends SparkFunSuite { new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR)) intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue))) } + + test("to ansi sql standard string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toSqlStandardString(i1) === "0") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toSqlStandardString(i2) === "+2-10") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toSqlStandardString(i3) === "-2-10") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toSqlStandardString(i4) === "+31") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toSqlStandardString(i5) === "-31") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toSqlStandardString(i6) === "+3:13:00.000123") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toSqlStandardString(i7) === "-3:13:00.000123") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toSqlStandardString(i8) === "-2-10 +31 +3:13:00.000123") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toSqlStandardString(i9) === "-3000:00:00") + } + + test("to iso 8601 string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toIso8601String(i1) === "PT0S") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toIso8601String(i2) === "P2Y10M") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toIso8601String(i3) === "P-2Y-10M") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toIso8601String(i4) === "P31D") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toIso8601String(i5) === "P-31D") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toIso8601String(i6) === "PT3H13M0.000123S") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toIso8601String(i7) === "PT-3H-13M-0.000123S") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toIso8601String(i8) === "P-2Y-10M31DT3H13M0.000123S") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toIso8601String(i9) === "PT-3000H") + } + + test("to multi units string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i1) === "0 seconds") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i2) === "2 years 10 months") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i3) === "-2 years -10 months") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toMultiUnitsString(i4) === "31 days") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toMultiUnitsString(i5) === "-31 days") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toMultiUnitsString(i6) === "3 hours 13 minutes 0.000123 seconds") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toMultiUnitsString(i7) === "-3 hours -13 minutes -0.000123 seconds") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toMultiUnitsString(i8) === + "-2 years -10 months 31 days 3 hours 13 minutes 0.000123 seconds") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toMultiUnitsString(i9) === "-3000 hours") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 75abac4cfd1da..d4e10b3ffc733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -22,9 +22,12 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * Runs a query returning the result in Hive compatible form. @@ -97,7 +100,12 @@ object HiveResult { case (null, _) => "null" case (s: String, StringType) => "\"" + s + "\"" case (decimal, DecimalType()) => decimal.toString - case (interval, CalendarIntervalType) => interval.toString + case (interval: CalendarInterval, CalendarIntervalType) => + SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => toSqlStandardString(interval) + case ISO_8601 => toIso8601String(interval) + case MULTI_UNITS => toMultiUnitsString(interval) + } case (other, tpe) if primitiveTypes contains tpe => other.toString } @@ -120,6 +128,12 @@ object HiveResult { DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (interval: CalendarInterval, CalendarIntervalType) => + SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => toSqlStandardString(interval) + case ISO_8601 => toIso8601String(interval) + case MULTI_UNITS => toMultiUnitsString(interval) + } case (interval, CalendarIntervalType) => interval.toString case (other, _ : UserDefinedType[_]) => other.toString case (other, tpe) if primitiveTypes.contains(tpe) => other.toString diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql new file mode 100644 index 0000000000000..62f3f43bd2c48 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql @@ -0,0 +1,3 @@ +-- tests for interval output style with iso_8601 format +--SET spark.sql.intervalOutputStyle = ISO_8601 +--import interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql new file mode 100644 index 0000000000000..375b4899e760e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql @@ -0,0 +1,3 @@ +-- tests for interval output style with sql standard format +--SET spark.sql.intervalOutputStyle = SQL_STANDARD +--import interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql new file mode 100644 index 0000000000000..ae19f1b6374ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql @@ -0,0 +1,14 @@ +-- tests for interval output style + +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql index 01df2a3fd1b21..3b25ef7334c0a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql @@ -270,10 +270,12 @@ SELECT interval '1 2:03:04' minute to second; -- test output of couple non-standard interval values in the sql style -- [SPARK-29406] Interval output styles -- SET IntervalStyle TO sql_standard; --- SELECT interval '1 day -1 hours', --- interval '-1 days +1 hours', --- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', --- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; +set spark.sql.intervalOutputStyle=SQL_STANDARD; +SELECT interval '1 day -1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; +set spark.sql.intervalOutputStyle=MULTI_UNITS; -- test outputting iso8601 intervals -- [SPARK-29406] Interval output styles diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out new file mode 100644 index 0000000000000..57fe8a3f4fcc6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out new file mode 100644 index 0000000000000..9e40f52151475 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out new file mode 100644 index 0000000000000..340496e404326 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 5ef1f8e5f0680..d981ed15e37f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 30 -- !query 0 @@ -216,3 +216,30 @@ SELECT interval '1 2:03:04' minute to second struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 26 output 1 days 2 hours 3 minutes 4 seconds + + +-- !query 27 +set spark.sql.intervalOutputStyle=SQL_STANDARD +-- !query 27 schema +struct +-- !query 27 output +spark.sql.intervalOutputStyle SQL_STANDARD + + +-- !query 28 +SELECT interval '1 day -1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 28 schema +struct<1 days -1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval> +-- !query 28 output ++1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 + + +-- !query 29 +set spark.sql.intervalOutputStyle=MULTI_UNITS +-- !query 29 schema +struct +-- !query 29 output +spark.sql.intervalOutputStyle MULTI_UNITS 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 6ea37baeaf24c..c80e675b149d0 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 @@ -23,7 +23,7 @@ import java.time.Instant import java.util.Locale import java.util.concurrent.TimeUnit -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -293,10 +293,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d + INTERVAL'$i'"), + df.selectExpr(s"d + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Date.valueOf("2015-10-02")), Row(Date.valueOf("2016-03-02")))) checkAnswer( - df.selectExpr(s"t + INTERVAL'$i'"), + df.selectExpr(s"t + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Timestamp.valueOf("2015-10-03 00:00:01")), Row(Timestamp.valueOf("2016-03-02 00:00:02")))) } @@ -309,10 +309,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d - INTERVAL'$i'"), + df.selectExpr(s"d - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Date.valueOf("2015-07-27")), Row(Date.valueOf("2015-12-26")))) checkAnswer( - df.selectExpr(s"t - INTERVAL'$i'"), + df.selectExpr(s"t - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Timestamp.valueOf("2015-07-29 23:59:59")), Row(Timestamp.valueOf("2015-12-27 00:00:00")))) } 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 index 82da4c049fd41..b50a6045f5378 100644 --- 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 @@ -93,7 +93,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "subquery/in-subquery/in-group-by.sql", "subquery/in-subquery/simple-in.sql", "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql" + "subquery/in-subquery/in-set-operations.sql", + // SPARK-29783: need to set conf + "interval-display-iso_8601.sql", + "interval-display-sql_standard.sql" ) override def runQueries( From 50f6d930dafc508d1e5162dd0dd580d1f91bfbd1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 15:50:06 +0800 Subject: [PATCH 28/83] [SPARK-29870][SQL] Unify the logic of multi-units interval string to CalendarInterval ### What changes were proposed in this pull request? We now have two different implementation for multi-units interval strings to CalendarInterval type values. One is used to covert interval string literals to CalendarInterval. This approach will re-delegate the interval string to spark parser which handles the string as a `singleInterval` -> `multiUnitsInterval` -> eventually call `IntervalUtils.fromUnitStrings` The other is used in `Cast`, which eventually calls `IntervalUtils.stringToInterval`. This approach is ~10 times faster than the other. We should unify these two for better performance and simple logic. this pr uses the 2nd approach. ### Why are the changes needed? We should unify these two for better performance and simple logic. ### Does this PR introduce any user-facing change? no ### How was this patch tested? we shall not fail on existing uts Closes #26491 from yaooqinn/SPARK-29870. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 - .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/TimeWindow.scala | 3 +- .../sql/catalyst/parser/AstBuilder.scala | 34 ++-- .../sql/catalyst/parser/ParseDriver.scala | 9 -- .../sql/catalyst/util/IntervalUtils.scala | 147 +++++------------- .../CollectionExpressionsSuite.scala | 37 +++-- .../expressions/DateExpressionsSuite.scala | 22 +-- .../expressions/HashExpressionsSuite.scala | 4 +- .../IntervalExpressionsSuite.scala | 16 +- .../expressions/MutableProjectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 3 +- .../expressions/UnsafeRowConverterSuite.scala | 3 +- .../parser/ExpressionParserSuite.scala | 13 +- .../catalyst/util/IntervalUtilsSuite.scala | 75 +++++---- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../execution/streaming/GroupStateImpl.scala | 4 +- .../sql/execution/streaming/Triggers.scala | 3 +- .../execution/command/DDLParserSuite.scala | 2 +- 19 files changed, 169 insertions(+), 222 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index c062377ee02a9..de0353494f003 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -79,10 +79,6 @@ singleTableSchema : colTypeList EOF ; -singleInterval - : INTERVAL? multiUnitsInterval EOF - ; - statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ef99c8efef834..8d11f4663a3ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -477,7 +477,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => IntervalUtils.stringToInterval(s)) + buildCast[UTF8String](_, s => IntervalUtils.safeStringToInterval(s)) } // LongConverter @@ -1234,7 +1234,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case StringType => val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") (c, evPrim, evNull) => - code"""$evPrim = $util.stringToInterval($c); + code"""$evPrim = $util.safeStringToInterval($c); if(${evPrim} == null) { ${evNull} = true; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 69badb9562dc3..caacb71814f17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String case class TimeWindow( timeColumn: Expression, @@ -103,7 +104,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - val cal = IntervalUtils.fromString(interval) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval)) if (cal.months != 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6be163b4b3c2a..68b1862a5d236 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -102,10 +102,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } - override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { - withOrigin(ctx)(visitMultiUnitsInterval(ctx.multiUnitsInterval)) - } - /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -1870,7 +1866,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => val interval = try { - IntervalUtils.fromString(value) + IntervalUtils.stringToInterval(UTF8String.fromString(value)) } catch { case e: IllegalArgumentException => val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) @@ -2069,22 +2065,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitMultiUnitsInterval(ctx: MultiUnitsIntervalContext): CalendarInterval = { withOrigin(ctx) { - val units = ctx.intervalUnit().asScala.map { unit => - val u = unit.getText.toLowerCase(Locale.ROOT) - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - if (u.endsWith("s")) u.substring(0, u.length - 1) else u - }.map(IntervalUtils.IntervalUnit.withName).toArray - - val values = ctx.intervalValue().asScala.map { value => - if (value.STRING() != null) { - string(value.STRING()) - } else { - value.getText - } - }.toArray - + val units = ctx.intervalUnit().asScala + val values = ctx.intervalValue().asScala try { - IntervalUtils.fromUnitStrings(units, values) + assert(units.length == values.length) + val kvs = units.indices.map { i => + val u = units(i).getText + val v = if (values(i).STRING() != null) { + string(values(i).STRING()) + } else { + values(i).getText + } + UTF8String.fromString(" " + v + " " + u) + } + IntervalUtils.stringToInterval(UTF8String.concat(kvs: _*)) } catch { case i: IllegalArgumentException => val e = new ParseException(i.getMessage, ctx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index a9d5e7c7544a6..e291d5f9cd84c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -29,21 +29,12 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.unsafe.types.CalendarInterval /** * Base SQL parsing infrastructure. */ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging { - /** - * Creates [[CalendarInterval]] for a given SQL String. Throws [[ParseException]] if the SQL - * string is not a valid interval format. - */ - def parseInterval(sqlText: String): CalendarInterval = parse(sqlText) { parser => - astBuilder.visitSingleInterval(parser.singleInterval()) - } - /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => astBuilder.visitSingleDataType(parser.singleDataType()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 104c65b6cd037..725ff81ef410e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal -import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -102,34 +101,6 @@ object IntervalUtils { Decimal(result, 18, 6) } - /** - * Converts a string to [[CalendarInterval]] case-insensitively. - * - * @throws IllegalArgumentException if the input string is not in valid interval format. - */ - def fromString(str: String): CalendarInterval = { - if (str == null) throw new IllegalArgumentException("Interval string cannot be null") - try { - CatalystSqlParser.parseInterval(str) - } catch { - case e: ParseException => - val ex = new IllegalArgumentException(s"Invalid interval string: $str\n" + e.message) - ex.setStackTrace(e.getStackTrace) - throw ex - } - } - - /** - * A safe version of `fromString`. It returns null for invalid input string. - */ - def safeFromString(str: String): CalendarInterval = { - try { - fromString(str) - } catch { - case _: IllegalArgumentException => null - } - } - private def toLongWithRange( fieldName: IntervalUnit, s: String, @@ -251,46 +222,6 @@ object IntervalUtils { } } - def fromUnitStrings(units: Array[IntervalUnit], values: Array[String]): CalendarInterval = { - assert(units.length == values.length) - var months: Int = 0 - var days: Int = 0 - var microseconds: Long = 0 - var i = 0 - while (i < units.length) { - try { - units(i) match { - case YEAR => - months = Math.addExact(months, Math.multiplyExact(values(i).toInt, 12)) - case MONTH => - months = Math.addExact(months, values(i).toInt) - case WEEK => - days = Math.addExact(days, Math.multiplyExact(values(i).toInt, 7)) - case DAY => - days = Math.addExact(days, values(i).toInt) - case HOUR => - val hoursUs = Math.multiplyExact(values(i).toLong, MICROS_PER_HOUR) - microseconds = Math.addExact(microseconds, hoursUs) - case MINUTE => - val minutesUs = Math.multiplyExact(values(i).toLong, MICROS_PER_MINUTE) - microseconds = Math.addExact(microseconds, minutesUs) - case SECOND => - microseconds = Math.addExact(microseconds, parseSecondNano(values(i))) - case MILLISECOND => - val millisUs = Math.multiplyExact(values(i).toLong, MICROS_PER_MILLIS) - microseconds = Math.addExact(microseconds, millisUs) - case MICROSECOND => - microseconds = Math.addExact(microseconds, values(i).toLong) - } - } catch { - case e: Exception => - throw new IllegalArgumentException(s"Error parsing interval string: ${e.getMessage}", e) - } - i += 1 - } - new CalendarInterval(months, days, microseconds) - } - // Parses a string with nanoseconds, truncates the result and returns microseconds private def parseNanos(nanosStr: String, isNegative: Boolean): Long = { if (nanosStr != null) { @@ -306,30 +237,6 @@ object IntervalUtils { } } - /** - * Parse second_nano string in ss.nnnnnnnnn format to microseconds - */ - private def parseSecondNano(secondNano: String): Long = { - def parseSeconds(secondsStr: String): Long = { - toLongWithRange( - SECOND, - secondsStr, - Long.MinValue / MICROS_PER_SECOND, - Long.MaxValue / MICROS_PER_SECOND) * MICROS_PER_SECOND - } - - secondNano.split("\\.") match { - case Array(secondsStr) => parseSeconds(secondsStr) - case Array("", nanosStr) => parseNanos(nanosStr, false) - case Array(secondsStr, nanosStr) => - val seconds = parseSeconds(secondsStr) - Math.addExact(seconds, parseNanos(nanosStr, seconds < 0)) - case _ => - throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn") - } - } - /** * Gets interval duration * @@ -558,18 +465,37 @@ object IntervalUtils { private final val millisStr = unitToUtf8(MILLISECOND) private final val microsStr = unitToUtf8(MICROSECOND) + /** + * A safe version of `stringToInterval`. It returns null for invalid input string. + */ + def safeStringToInterval(input: UTF8String): CalendarInterval = { + try { + stringToInterval(input) + } catch { + case _: IllegalArgumentException => null + } + } + + /** + * Converts a string to [[CalendarInterval]] case-insensitively. + * + * @throws IllegalArgumentException if the input string is not in valid interval format. + */ def stringToInterval(input: UTF8String): CalendarInterval = { import ParseState._ + def throwIAE(msg: String, e: Exception = null) = { + throw new IllegalArgumentException(s"Error parsing '$input' to interval, $msg", e) + } if (input == null) { - return null + throwIAE("interval string cannot be null") } // scalastyle:off caselocale .toLowerCase val s = input.trim.toLowerCase // scalastyle:on val bytes = s.getBytes if (bytes.isEmpty) { - return null + throwIAE("interval string cannot be empty") } var state = PREFIX var i = 0 @@ -588,13 +514,19 @@ object IntervalUtils { } } + def currentWord: UTF8String = { + val strings = s.split(UTF8String.blankString(1), -1) + val lenRight = s.substring(i, s.numBytes()).split(UTF8String.blankString(1), -1).length + strings(strings.length - lenRight) + } + while (i < bytes.length) { val b = bytes(i) state match { case PREFIX => if (s.startsWith(intervalStr)) { if (s.numBytes() == intervalStr.numBytes()) { - return null + throwIAE("interval string cannot be empty") } else { i += intervalStr.numBytes() } @@ -627,7 +559,7 @@ object IntervalUtils { fractionScale = (NANOS_PER_SECOND / 10).toInt i += 1 state = VALUE_FRACTIONAL_PART - case _ => return null + case _ => throwIAE( s"unrecognized number '$currentWord'") } case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) case VALUE => @@ -636,13 +568,13 @@ object IntervalUtils { try { currentValue = Math.addExact(Math.multiplyExact(10, currentValue), (b - '0')) } catch { - case _: ArithmeticException => return null + case e: ArithmeticException => throwIAE(e.getMessage, e) } case ' ' => state = TRIM_BEFORE_UNIT case '.' => fractionScale = (NANOS_PER_SECOND / 10).toInt state = VALUE_FRACTIONAL_PART - case _ => return null + case _ => throwIAE(s"invalid value '$currentWord'") } i += 1 case VALUE_FRACTIONAL_PART => @@ -653,14 +585,17 @@ object IntervalUtils { case ' ' => fraction /= NANOS_PER_MICROS.toInt state = TRIM_BEFORE_UNIT - case _ => return null + case _ if '0' <= b && b <= '9' => + throwIAE(s"interval can only support nanosecond precision, '$currentWord' is out" + + s" of range") + case _ => throwIAE(s"invalid value '$currentWord'") } i += 1 case TRIM_BEFORE_UNIT => trimToNextState(b, UNIT_BEGIN) case UNIT_BEGIN => // Checks that only seconds can have the fractional part if (b != 's' && fractionScale >= 0) { - return null + throwIAE(s"'$currentWord' cannot have fractional part") } if (isNegative) { currentValue = -currentValue @@ -704,18 +639,18 @@ object IntervalUtils { } else if (s.matchAt(microsStr, i)) { microseconds = Math.addExact(microseconds, currentValue) i += microsStr.numBytes() - } else return null - case _ => return null + } else throwIAE(s"invalid unit '$currentWord'") + case _ => throwIAE(s"invalid unit '$currentWord'") } } catch { - case _: ArithmeticException => return null + case e: ArithmeticException => throwIAE(e.getMessage, e) } state = UNIT_SUFFIX case UNIT_SUFFIX => b match { case 's' => state = UNIT_END case ' ' => state = TRIM_BEFORE_SIGN - case _ => return null + case _ => throwIAE(s"invalid unit '$currentWord'") } i += 1 case UNIT_END => @@ -723,7 +658,7 @@ object IntervalUtils { case ' ' => i += 1 state = TRIM_BEFORE_SIGN - case _ => return null + case _ => throwIAE(s"invalid unit '$currentWord'") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index dd6d437d8063e..b4343b648110f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -32,9 +32,12 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH +import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + def testSize(sizeOfNull: Any): Unit = { val a0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType)) val a1 = Literal.create(Seq[Integer](), ArrayType(IntegerType)) @@ -711,7 +714,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:00")), - Literal(fromString("interval 12 hours"))), + Literal(stringToInterval("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -720,7 +723,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:01")), - Literal(fromString("interval 12 hours"))), + Literal(stringToInterval("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -729,7 +732,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 12 hours")))), + Literal(negate(stringToInterval("interval 12 hours")))), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -738,7 +741,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(negate(fromString("interval 12 hours")))), + Literal(negate(stringToInterval("interval 12 hours")))), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -747,7 +750,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -756,7 +759,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 1 month")))), + Literal(negate(stringToInterval("interval 1 month")))), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -765,7 +768,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 1 month 1 day")))), + Literal(negate(stringToInterval("interval 1 month 1 day")))), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -774,7 +777,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-31 00:00:00")), Literal(Timestamp.valueOf("2018-04-30 00:00:00")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-31 00:00:00"), Timestamp.valueOf("2018-02-28 00:00:00"), @@ -784,7 +787,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(fromString("interval 1 month 1 second"))), + Literal(stringToInterval("interval 1 month 1 second"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:01"))) @@ -792,7 +795,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:04:06")), - Literal(fromString("interval 1 month 2 minutes 3 seconds"))), + Literal(stringToInterval("interval 1 month 2 minutes 3 seconds"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:02:03"), @@ -830,7 +833,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), - Literal(fromString("interval 30 minutes"))), + Literal(stringToInterval("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-03-25 01:30:00"), Timestamp.valueOf("2018-03-25 03:00:00"), @@ -840,7 +843,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-10-28 01:30:00")), Literal(Timestamp.valueOf("2018-10-28 03:30:00")), - Literal(fromString("interval 30 minutes"))), + Literal(stringToInterval("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-10-28 01:30:00"), noDST(Timestamp.valueOf("2018-10-28 02:00:00")), @@ -857,7 +860,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), - Literal(fromString("interval 2 days"))), + Literal(stringToInterval("interval 2 days"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-01-03"), @@ -866,7 +869,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-03-01")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-02-01"), @@ -875,7 +878,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-31")), Literal(Date.valueOf("2018-04-30")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Date.valueOf("2018-01-31"), Date.valueOf("2018-02-28"), @@ -896,14 +899,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-02")), Literal(Date.valueOf("1970-01-01")), - Literal(fromString("interval 1 day"))), + Literal(stringToInterval("interval 1 day"))), EmptyRow, "sequence boundaries: 1 to 0 by 1") checkExceptionInExpression[IllegalArgumentException]( new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(negate(fromString("interval 1 month")))), + Literal(negate(stringToInterval("interval 1 month")))), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * MICROS_PER_DAY}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 5f043ce972bed..5cd4d11e32f7a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1090,17 +1090,17 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), - IntervalUtils.fromString("interval " + - "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds"))) checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), - IntervalUtils.fromString("interval " + - "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds"))) checkEvaluation( SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), - IntervalUtils.fromString("interval " + - "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds"))) } test("subtract dates") { @@ -1108,18 +1108,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractDates(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), - IntervalUtils.fromString("interval 1 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 days"))) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), - IntervalUtils.fromString("interval -1 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval -1 days"))) val epochDate = Literal(LocalDate.ofEpochDay(0)) checkEvaluation(SubtractDates(Literal(end), epochDate), - IntervalUtils.fromString("interval 49 years 9 months 4 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 49 years 9 months 4 days"))) checkEvaluation(SubtractDates(epochDate, Literal(end)), - IntervalUtils.fromString("interval -49 years -9 months -4 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval -49 years -9 months -4 days"))) checkEvaluation( SubtractDates( Literal(LocalDate.of(10000, 1, 1)), Literal(LocalDate.of(1, 1, 1))), - IntervalUtils.fromString("interval 9999 years")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 9999 years"))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 4b2da73abe562..3a68847ecb1f4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) test("md5") { checkEvaluation(Md5(Literal("ABC".getBytes(StandardCharsets.UTF_8))), @@ -252,7 +253,8 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for CalendarInterval type") { def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = { - checkHiveHash(IntervalUtils.fromString(interval), CalendarIntervalType, expected) + checkHiveHash(IntervalUtils.stringToInterval(UTF8String.fromString(interval)), + CalendarIntervalType, expected) } // ----- MICROSEC ----- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index e483f028ffff3..ddcb6a66832af 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -21,13 +21,15 @@ import scala.language.implicitConversions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeConstants._ -import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.sql.catalyst.util.IntervalUtils.stringToInterval import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + implicit def interval(s: String): Literal = { - Literal(fromString("interval " + s)) + Literal(stringToInterval( "interval " + s)) } test("millenniums") { @@ -197,8 +199,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("multiply") { def check(interval: String, num: Double, expected: String): Unit = { checkEvaluation( - MultiplyInterval(Literal(fromString(interval)), Literal(num)), - if (expected == null) null else fromString(expected)) + MultiplyInterval(Literal(stringToInterval(interval)), Literal(num)), + if (expected == null) null else stringToInterval(expected)) } check("0 seconds", 10, "0 seconds") @@ -215,8 +217,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("divide") { def check(interval: String, num: Double, expected: String): Unit = { checkEvaluation( - DivideInterval(Literal(fromString(interval)), Literal(num)), - if (expected == null) null else fromString(expected)) + DivideInterval(Literal(stringToInterval(interval)), Literal(num)), + if (expected == null) null else stringToInterval(expected)) } check("0 seconds", 10, "0 seconds") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 23ba9c6ec7388..63700a1e94a3e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -56,7 +57,8 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("variable-length types") { val proj = createMutableProjection(variableLengthTypes) - val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.fromString("interval 1 day"), + val scalaValues = Seq("abc", BigDecimal(10), + IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 day")), Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), new java.lang.Integer(5)) val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index 4ccd4f7ce798d..ef7764dba1e9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -485,7 +485,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ("abcd".getBytes, BinaryType), ("abcd", StringType), (BigDecimal.valueOf(10), DecimalType.IntDecimal), - (IntervalUtils.fromString("interval 3 day"), CalendarIntervalType), + (IntervalUtils.stringToInterval(UTF8String.fromString("interval 3 day")), + CalendarIntervalType), (java.math.BigDecimal.valueOf(10), DecimalType.BigIntDecimal), (Array(3, 2, 1), ArrayType(IntegerType)) ).foreach { case (input, dt) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 20e77254ecdad..b80b30a4e07ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -531,7 +531,8 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // Simple tests val inputRow = InternalRow.fromSeq(Seq( false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), - Decimal(255), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2) + Decimal(255), IntervalUtils.stringToInterval(UTF8String.fromString( "interval 1 day")), + Array[Byte](1, 2) )) val fields1 = Array( BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 17a2213d0ccb1..a7dbd4e4e3459 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * Test basic expression parsing. @@ -43,6 +43,8 @@ class ExpressionParserSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + val defaultParser = CatalystSqlParser def assertEqual( @@ -434,13 +436,13 @@ class ExpressionParserSuite extends AnalysisTest { intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. - val intervalLiteral = Literal(IntervalUtils.fromString("interval 3 month 1 hour")) + val intervalLiteral = Literal(IntervalUtils.stringToInterval("interval 3 month 1 hour")) assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") assertEqual( "-interval '3 month 1 hour'", - Literal(IntervalUtils.fromString("interval -3 month -1 hour"))) + Literal(IntervalUtils.stringToInterval("interval -3 month -1 hour"))) // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) @@ -602,7 +604,7 @@ class ExpressionParserSuite extends AnalysisTest { MICROSECOND) def intervalLiteral(u: IntervalUnit, s: String): Literal = { - Literal(IntervalUtils.fromUnitStrings(Array(u), Array(s))) + Literal(IntervalUtils.stringToInterval(s + " " + u.toString)) } test("intervals") { @@ -651,7 +653,8 @@ class ExpressionParserSuite extends AnalysisTest { 0, 0, 13 * MICROS_PER_SECOND + 123 * MICROS_PER_MILLIS + 456))) - checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond"))) + checkIntervals("1.001 second", + Literal(IntervalUtils.stringToInterval("1 second 1 millisecond"))) // Non Existing unit intercept("interval 10 nanoseconds", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 65cfe896b2024..73a2adbaec1db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -28,20 +28,31 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class IntervalUtilsSuite extends SparkFunSuite { private def checkFromString(input: String, expected: CalendarInterval): Unit = { - assert(fromString(input) === expected) assert(stringToInterval(UTF8String.fromString(input)) === expected) + assert(safeStringToInterval(UTF8String.fromString(input)) === expected) + } + + private def checkFromStringWithFunc( + input: String, + months: Int, + days: Int, + us: Long, + func: CalendarInterval => CalendarInterval): Unit = { + val expected = new CalendarInterval(months, days, us) + assert(func(stringToInterval(UTF8String.fromString(input))) === expected) + assert(func(safeStringToInterval(UTF8String.fromString(input))) === expected) } private def checkFromInvalidString(input: String, errorMsg: String): Unit = { try { - fromString(input) + stringToInterval(UTF8String.fromString(input)) fail("Expected to throw an exception for the invalid input") } catch { case e: IllegalArgumentException => val msg = e.getMessage assert(msg.contains(errorMsg)) } - assert(stringToInterval(UTF8String.fromString(input)) === null) + assert(safeStringToInterval(UTF8String.fromString(input)) === null) } private def testSingleUnit( @@ -69,7 +80,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString(null, "cannot be null") for (input <- Seq("", " ", "interval", "interval1 day", "foo", "foo 1 day")) { - checkFromInvalidString(input, "Invalid interval string") + checkFromInvalidString(input, "Error parsing") } } @@ -93,8 +104,18 @@ class IntervalUtilsSuite extends SparkFunSuite { // Allow duplicated units and summarize their values checkFromString("1 day 10 day", new CalendarInterval(0, 11, 0)) // Only the seconds units can have the fractional part - checkFromInvalidString("1.5 days", "Error parsing interval string") - checkFromInvalidString("1. hour", "Error parsing interval string") + checkFromInvalidString("1.5 days", "'days' cannot have fractional part") + checkFromInvalidString("1. hour", "'hour' cannot have fractional part") + checkFromInvalidString("1 hourX", "invalid unit 'hourx'") + checkFromInvalidString("~1 hour", "unrecognized number '~1'") + checkFromInvalidString("1 Mour", "invalid unit 'mour'") + checkFromInvalidString("1 aour", "invalid unit 'aour'") + checkFromInvalidString("1a1 hour", "invalid value '1a1'") + checkFromInvalidString("1.1a1 seconds", "invalid value '1.1a1'") + checkFromInvalidString("2234567890 days", "integer overflow") + checkFromInvalidString("\n", "Error parsing '\n' to interval") + checkFromInvalidString("\t", "Error parsing '\t' to interval") + } test("string to interval: seconds with fractional part") { @@ -107,7 +128,7 @@ class IntervalUtilsSuite extends SparkFunSuite { // truncate nanoseconds to microseconds checkFromString("0.999999999 seconds", new CalendarInterval(0, 0, 999999)) checkFromString(".999999999 seconds", new CalendarInterval(0, 0, 999999)) - checkFromInvalidString("0.123456789123 seconds", "Error parsing interval string") + checkFromInvalidString("0.123456789123 seconds", "'0.123456789123' is out of range") } test("from year-month string") { @@ -174,7 +195,7 @@ class IntervalUtilsSuite extends SparkFunSuite { test("interval duration") { def duration(s: String, unit: TimeUnit, daysPerMonth: Int): Long = { - IntervalUtils.getDuration(fromString(s), unit, daysPerMonth) + IntervalUtils.getDuration(stringToInterval(UTF8String.fromString(s)), unit, daysPerMonth) } assert(duration("0 seconds", TimeUnit.MILLISECONDS, 31) === 0) @@ -193,7 +214,7 @@ class IntervalUtilsSuite extends SparkFunSuite { test("negative interval") { def isNegative(s: String, daysPerMonth: Int): Boolean = { - IntervalUtils.isNegative(fromString(s), daysPerMonth) + IntervalUtils.isNegative(stringToInterval(UTF8String.fromString(s)), daysPerMonth) } assert(isNegative("-1 months", 28)) @@ -269,33 +290,27 @@ class IntervalUtilsSuite extends SparkFunSuite { } test("justify days") { - assert(justifyDays(fromString("1 month 35 day")) === new CalendarInterval(2, 5, 0)) - assert(justifyDays(fromString("-1 month 35 day")) === new CalendarInterval(0, 5, 0)) - assert(justifyDays(fromString("1 month -35 day")) === new CalendarInterval(0, -5, 0)) - assert(justifyDays(fromString("-1 month -35 day")) === new CalendarInterval(-2, -5, 0)) - assert(justifyDays(fromString("-1 month 2 day")) === new CalendarInterval(0, -28, 0)) + checkFromStringWithFunc("1 month 35 day", 2, 5, 0, justifyDays) + checkFromStringWithFunc("-1 month 35 day", 0, 5, 0, justifyDays) + checkFromStringWithFunc("1 month -35 day", 0, -5, 0, justifyDays) + checkFromStringWithFunc("-1 month -35 day", -2, -5, 0, justifyDays) + checkFromStringWithFunc("-1 month 2 day", 0, -28, 0, justifyDays) } test("justify hours") { - assert(justifyHours(fromString("29 day 25 hour")) === - new CalendarInterval(0, 30, 1 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("29 day -25 hour")) === - new CalendarInterval(0, 27, 23 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("-29 day 25 hour")) === - new CalendarInterval(0, -27, -23 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("-29 day -25 hour")) === - new CalendarInterval(0, -30, -1 * MICROS_PER_HOUR)) + checkFromStringWithFunc("29 day 25 hour", 0, 30, 1 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("29 day -25 hour", 0, 27, 23 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("-29 day 25 hour", 0, -27, -23 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("-29 day -25 hour", 0, -30, -1 * MICROS_PER_HOUR, justifyHours) } test("justify interval") { - assert(justifyInterval(fromString("1 month 29 day 25 hour")) === - new CalendarInterval(2, 0, 1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("-1 month 29 day -25 hour")) === - new CalendarInterval(0, -2, -1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("1 month -29 day -25 hour")) === - new CalendarInterval(0, 0, -1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("-1 month -29 day -25 hour")) === - new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR)) + checkFromStringWithFunc("1 month 29 day 25 hour", 2, 0, 1 * MICROS_PER_HOUR, justifyInterval) + checkFromStringWithFunc("-1 month 29 day -25 hour", 0, -2, -1 * MICROS_PER_HOUR, + justifyInterval) + checkFromStringWithFunc("1 month -29 day -25 hour", 0, 0, -1 * MICROS_PER_HOUR, justifyInterval) + checkFromStringWithFunc("-1 month -29 day -25 hour", -2, 0, -1 * MICROS_PER_HOUR, + justifyInterval) intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index aaa3f9dd71594..0fcd82276392c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -60,7 +60,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils private[sql] object Dataset { @@ -725,7 +725,7 @@ class Dataset[T] private[sql]( def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = try { - IntervalUtils.fromString(delayThreshold) + IntervalUtils.stringToInterval(UTF8String.fromString(delayThreshold)) } catch { case e: IllegalArgumentException => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index aac5da8104a8b..59ce7c3707b27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, Processing import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.UTF8String /** @@ -160,7 +160,7 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - val cal = IntervalUtils.fromString(duration) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(duration)) if (IntervalUtils.isNegative(cal)) { throw new IllegalArgumentException(s"Provided duration ($duration) is negative") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 2dd287cb734bf..1a27fe61d9602 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -24,6 +24,7 @@ import scala.concurrent.duration.Duration import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.UTF8String private object Triggers { def validate(intervalMs: Long): Unit = { @@ -31,7 +32,7 @@ private object Triggers { } def convert(interval: String): Long = { - val cal = IntervalUtils.fromString(interval) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval)) if (cal.months != 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } 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 b11e02a9c52bd..9c28529ed710f 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 @@ -751,7 +751,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assertError("select interval '23:61:15' hour to second", "minute 61 outside range [0, 59]") assertError("select interval '.1111111111' second", - "nanosecond 1111111111 outside range") + "'.1111111111' is out of range") } test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { From c32e2286898fe8660c7deed9303f1c5c15b60757 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Mon, 18 Nov 2019 20:40:23 +0800 Subject: [PATCH 29/83] [SPARK-29859][SQL] ALTER DATABASE (SET LOCATION) should look up catalog like v2 commands ### What changes were proposed in this pull request? Add AlterNamespaceSetLocationStatement, AlterNamespaceSetLocation, AlterNamespaceSetLocationExec to make ALTER DATABASE (SET LOCATION) look up catalog like v2 commands. And also refine the code of AlterNamespaceSetProperties, AlterNamespaceSetPropertiesExec, DescribeNamespace, DescribeNamespaceExec to use SupportsNamespaces instead of CatalogPlugin for catalog parameter. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "ALTER NAMESPACE ... SET LOCATION" whose function is same as "ALTER DATABASE ... SET LOCATION" and "ALTER SCHEMA ... SET LOCATION". ### How was this patch tested? New unit tests Closes #26562 from fuwhu/SPARK-29859. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 ++-- .../sql/catalyst/analysis/ResolveCatalogs.scala | 8 ++++++-- .../spark/sql/catalyst/parser/AstBuilder.scala | 16 ++++++++++++++++ .../sql/catalyst/plans/logical/statements.scala | 7 +++++++ .../sql/catalyst/plans/logical/v2Commands.scala | 4 ++-- .../sql/catalyst/parser/DDLParserSuite.scala | 14 ++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 7 +++++++ .../spark/sql/execution/SparkSqlParser.scala | 16 ---------------- .../v2/AlterNamespaceSetPropertiesExec.scala | 11 ++++------- .../datasources/v2/DescribeNamespaceExec.scala | 10 +++------- .../sql/connector/DataSourceV2SQLSuite.scala | 14 ++++++++++++++ .../sql/execution/command/DDLParserSuite.scala | 9 --------- 12 files changed, 75 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index de0353494f003..13f1f64ba7b8d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -89,8 +89,8 @@ statement (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace | ALTER (database | NAMESPACE) multipartIdentifier SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties - | ALTER database db=errorCapturingIdentifier - SET locationSpec #setDatabaseLocation + | ALTER (database | NAMESPACE) multipartIdentifier + SET locationSpec #setNamespaceLocation | DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier (RESTRICT | CASCADE)? #dropNamespace | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 0d8d58520edad..5a4fb2e865903 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -94,7 +94,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"because view support in catalog has not been implemented yet") case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) => - AlterNamespaceSetProperties(catalog, nameParts, properties) + AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, properties) + + case AlterNamespaceSetLocationStatement(NonSessionCatalog(catalog, nameParts), location) => + AlterNamespaceSetProperties( + catalog.asNamespaceCatalog, nameParts, Map("location" -> location)) case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => @@ -176,7 +180,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) DropNamespace(catalog, nameParts, ifExists, cascade) case DescribeNamespaceStatement(NonSessionCatalog(catalog, nameParts), extended) => - DescribeNamespace(catalog, nameParts, extended) + DescribeNamespace(catalog.asNamespaceCatalog, nameParts, extended) case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 68b1862a5d236..59c79f4021fb0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2539,6 +2539,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create an [[AlterNamespaceSetLocationStatement]] logical plan. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) namespace SET LOCATION path; + * }}} + */ + override def visitSetNamespaceLocation(ctx: SetNamespaceLocationContext): LogicalPlan = { + withOrigin(ctx) { + AlterNamespaceSetLocationStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitLocationSpec(ctx.locationSpec)) + } + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 875ff4e4a2821..7a98cccc3d7a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -357,6 +357,13 @@ case class AlterNamespaceSetPropertiesStatement( namespace: Seq[String], properties: Map[String, String]) extends ParsedStatement +/** + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command, as parsed from SQL. + */ +case class AlterNamespaceSetLocationStatement( + namespace: Seq[String], + location: String) extends ParsedStatement + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 22edb3619a929..f7f8b2778d234 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -259,7 +259,7 @@ case class DropNamespace( * The logical plan of the DESCRIBE NAMESPACE command that works for v2 catalogs. */ case class DescribeNamespace( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], extended: Boolean) extends Command { @@ -275,7 +275,7 @@ case class DescribeNamespace( * command that works for v2 catalogs. */ case class AlterNamespaceSetProperties( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], properties: Map[String, String]) extends Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e87f53910c2c7..d3ca6f7a8eee2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1192,6 +1192,20 @@ class DDLParserSuite extends AnalysisTest { Seq("a", "b", "c"), Map("b" -> "b"))) } + test("set namespace location") { + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index cac320edc47e2..708203119f4bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -172,6 +172,13 @@ class ResolveSessionCatalog( } AlterDatabasePropertiesCommand(nameParts.head, properties) + case AlterNamespaceSetLocationStatement(SessionCatalog(_, nameParts), location) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + AlterDatabaseSetLocationCommand(nameParts.head, location) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { 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 7fe4c00f5dc8d..8241f850e3aa7 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 @@ -227,22 +227,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create an [[AlterDatabaseSetLocationCommand]] command. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA) database SET LOCATION path; - * }}} - */ - override def visitSetDatabaseLocation( - ctx: SetDatabaseLocationContext): LogicalPlan = withOrigin(ctx) { - AlterDatabaseSetLocationCommand( - ctx.db.getText, - visitLocationSpec(ctx.locationSpec) - ) - } - /** * Create a plan for a DESCRIBE FUNCTION command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala index fd6922cd1c3ab..1eebe4cdb6a86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala @@ -19,23 +19,20 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, NamespaceChange} +import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} /** * Physical plan node for setting properties of namespace. */ case class AlterNamespaceSetPropertiesExec( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], - props: Map[String, String]) - extends V2CommandExec { + props: Map[String, String]) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val changes = props.map{ case (k, v) => NamespaceChange.setProperty(k, v) }.toSeq - catalog.asNamespaceCatalog.alterNamespace(namespace.toArray, changes: _*) + catalog.alterNamespace(namespace.toArray, changes: _*) Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index 5c20e5ae08383..7c5cfcbbc7e3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.COMMENT_TABLE_PROP import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.LOCATION_TABLE_PROP import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.RESERVED_PROPERTIES @@ -34,19 +34,15 @@ import org.apache.spark.sql.types.StructType */ case class DescribeNamespaceExec( output: Seq[Attribute], - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], isExtended: Boolean) extends V2CommandExec { - private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() override protected def run(): Seq[InternalRow] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val rows = new ArrayBuffer[InternalRow]() - val nsCatalog = catalog.asNamespaceCatalog val ns = namespace.toArray - val metadata = nsCatalog.loadNamespaceMetadata(ns) + val metadata = catalog.loadNamespaceMetadata(ns) rows += toCatalystRow("Namespace Name", ns.last) rows += toCatalystRow("Description", metadata.get(COMMENT_TABLE_PROP)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e5bf8e337c8d3..36d2deaa309ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -948,6 +948,20 @@ class DataSourceV2SQLSuite } } + test("AlterNamespaceSetLocation using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test_1'") + sql("ALTER NAMESPACE testcat.ns1.ns2 SET LOCATION '/tmp/ns_test_2'") + val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") + assert(descriptionDf.collect() === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test_2") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") 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 9c28529ed710f..11131310fa4f2 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 @@ -80,15 +80,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { containsThesePhrases = Seq("key_without_value")) } - test("alter database set location") { - // ALTER (DATABASE|SCHEMA) database_name SET LOCATION - val sql1 = "ALTER DATABASE database_name SET LOCATION '/home/user/db'" - val parsed1 = parser.parsePlan(sql1) - - val expected1 = AlterDatabaseSetLocationCommand("database_name", "/home/user/db") - comparePlans(parsed1, expected1) - } - test("create function") { val sql1 = """ From ae6b711b262de21b701434b8e4d031ca265cc376 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 23:30:31 +0800 Subject: [PATCH 30/83] [SPARK-29941][SQL] Add ansi type aliases for char and decimal ### What changes were proposed in this pull request? Checked with SQL Standard and PostgreSQL > CHAR is equivalent to CHARACTER. DEC is equivalent to DECIMAL. INT is equivalent to INTEGER. VARCHAR is equivalent to CHARACTER VARYING. ... ```sql postgres=# select dec '1.0'; numeric --------- 1.0 (1 row) postgres=# select CHARACTER '. second'; bpchar ---------- . second (1 row) postgres=# select CHAR '. second'; bpchar ---------- . second (1 row) ``` ### Why are the changes needed? For better ansi support ### Does this PR introduce any user-facing change? yes, we add character as char and dec as decimal ### How was this patch tested? add ut Closes #26574 from yaooqinn/SPARK-29941. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 8 ++++---- .../spark/sql/catalyst/parser/DataTypeParserSuite.scala | 3 +++ 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 59c79f4021fb0..bb8eab0de6089 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2153,12 +2153,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case ("date", Nil) => DateType case ("timestamp", Nil) => TimestampType case ("string", Nil) => StringType - case ("char", length :: Nil) => CharType(length.getText.toInt) + case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) case ("binary", Nil) => BinaryType - case ("decimal", Nil) => DecimalType.USER_DEFAULT - case ("decimal", precision :: Nil) => DecimalType(precision.getText.toInt, 0) - case ("decimal", precision :: scale :: Nil) => + case ("decimal" | "dec", Nil) => DecimalType.USER_DEFAULT + case ("decimal" | "dec", precision :: Nil) => DecimalType(precision.getText.toInt, 0) + case ("decimal" | "dec", precision :: scale :: Nil) => DecimalType(precision.getText.toInt, scale.getText.toInt) case ("interval", Nil) => CalendarIntervalType case (dt, params) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index 1a6286067a618..d519fdf378786 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -51,10 +51,13 @@ class DataTypeParserSuite extends SparkFunSuite { checkDataType("dOUBle", DoubleType) checkDataType("decimal(10, 5)", DecimalType(10, 5)) checkDataType("decimal", DecimalType.USER_DEFAULT) + checkDataType("Dec(10, 5)", DecimalType(10, 5)) + checkDataType("deC", DecimalType.USER_DEFAULT) checkDataType("DATE", DateType) checkDataType("timestamp", TimestampType) checkDataType("string", StringType) checkDataType("ChaR(5)", StringType) + checkDataType("ChaRacter(5)", StringType) checkDataType("varchAr(20)", StringType) checkDataType("cHaR(27)", StringType) checkDataType("BINARY", BinaryType) From ea010a2bc226197b3161a5c4db9cfe5a4444c7a4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 19 Nov 2019 01:32:13 +0800 Subject: [PATCH 31/83] [SPARK-29873][SQL][TEST][FOLLOWUP] set operations should not escape when regen golden file with --SET --import both specified ### What changes were proposed in this pull request? When regenerating golden files, the set operations via `--SET` will not be done, but those with --import should be exceptions because we need the set command. ### Why are the changes needed? fix test tool. ### Does this PR introduce any user-facing change? ### How was this patch tested? add ut, but I'm not sure we need these tests for tests itself. cc maropu cloud-fan Closes #26557 from yaooqinn/SPARK-29873. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../inputs/ansi/higher-order-functions.sql | 2 +- .../sql-tests/inputs/ansi/interval.sql | 2 +- .../sql-tests/inputs/ansi/literals.sql | 2 +- .../inputs/interval-display-iso_8601.sql | 2 +- .../inputs/interval-display-sql_standard.sql | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 18 +++++++++++------- 6 files changed, 16 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql index 4068a27fcb2a7..1e2424fe47cad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql @@ -1 +1 @@ ---import higher-order-functions.sql +--IMPORT higher-order-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index 215ee7c074fa6..087914eebb077 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,4 +1,4 @@ ---import interval.sql +--IMPORT interval.sql -- the `interval` keyword can be omitted with ansi mode select 1 year 2 days; diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql index 170690ea699c0..698e8fa886307 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql @@ -1,2 +1,2 @@ --- malformed interval literal with ansi mode ---import literals.sql +--IMPORT literals.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql index 62f3f43bd2c48..3b63c715a6aa1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql @@ -1,3 +1,3 @@ -- tests for interval output style with iso_8601 format --SET spark.sql.intervalOutputStyle = ISO_8601 ---import interval-display.sql +--IMPORT interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql index 375b4899e760e..d96865b160bb6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql @@ -1,3 +1,3 @@ -- tests for interval output style with sql standard format --SET spark.sql.intervalOutputStyle = SQL_STANDARD ---import interval-display.sql +--IMPORT interval-display.sql 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 d693ff0da33cd..ae0ccf556fbc3 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 @@ -65,7 +65,7 @@ import org.apache.spark.tags.ExtendedSQLTest * 1. A list of SQL queries separated by semicolon. * 2. Lines starting with -- are treated as comments and ignored. * 3. Lines starting with --SET are used to run the file with the following set of configs. - * 4. Lines starting with --import are used to load queries from another test file. + * 4. Lines starting with --IMPORT are used to load queries from another test file. * * For example: * {{{ @@ -265,9 +265,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) - // If `--import` found, load code from another test case file, then insert them + // If `--IMPORT` found, load code from another test case file, then insert them // into the head in this test. - val importedTestCaseName = comments.filter(_.startsWith("--import ")).map(_.substring(9)) + val importedTestCaseName = comments.filter(_.startsWith("--IMPORT ")).map(_.substring(9)) val importedCode = importedTestCaseName.flatMap { testCaseName => listTestCases.find(_.name == testCaseName).map { testCase => val input = fileToString(new File(testCase.inputFile)) @@ -283,13 +283,17 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") - // When we are regenerating the golden files, we don't need to set any config as they - // all need to return the same result - if (regenerateGoldenFiles || !isTestWithConfigSets) { + // When we are regenerating the golden files for test cases without '--IMPORT' specified, or + // running test cases against [[ThriftServerQueryTestSuite], we don't need to set any config as + // they all need to return the same result. + // When we use '--SET' and '--IMPORT' together for those import queries, we want to run the + // same queries from the original file but with different settings and save the answers. So the + // `--SET` will be respected in this case. + if ((regenerateGoldenFiles && importedTestCaseName.isEmpty) || !isTestWithConfigSets) { runQueries(queries, testCase, None) } else { val configSets = { - val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) + val configLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) val configs = configLines.map(_.split(",").map { confAndValue => val (conf, value) = confAndValue.span(_ != '=') conf.trim -> value.substring(1).trim From 9514b822a70d77a6298ece48e6c053200360302c Mon Sep 17 00:00:00 2001 From: Hossein Date: Tue, 19 Nov 2019 09:04:59 +0900 Subject: [PATCH 32/83] [SPARK-29777][SPARKR] SparkR::cleanClosure aggressively removes a function required by user function ### What changes were proposed in this pull request? The implementation for walking through the user function AST and picking referenced variables and functions, had an optimization to skip a branch if it had already seen it. This runs into an interesting problem in the following example ``` df <- createDataFrame(data.frame(x=1)) f1 <- function(x) x + 1 f2 <- function(x) f1(x) + 2 dapplyCollect(df, function(x) { f1(x); f2(x) }) ``` Results in error: ``` org.apache.spark.SparkException: R computation failed with Error in f1(x) : could not find function "f1" Calls: compute -> computeFunc -> f2 ``` ### Why are the changes needed? Bug fix ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit tests in `test_utils.R` Closes #26429 from falaki/SPARK-29777. Authored-by: Hossein Signed-off-by: HyukjinKwon --- R/pkg/R/utils.R | 8 ++++++-- R/pkg/tests/fulltests/test_utils.R | 9 +++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 15e2410d3a31c..a8c1ddb3dd20b 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -543,10 +543,14 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, ifnotfound = list(list(NULL)))[[1]] found <- sapply(funcList, function(func) { - ifelse(identical(func, obj), TRUE, FALSE) + ifelse( + identical(func, obj) && + # Also check if the parent environment is identical to current parent + identical(parent.env(environment(func)), func.env), + TRUE, FALSE) }) if (sum(found) > 0) { - # If function has been examined, ignore. + # If function has been examined ignore break } # Function has not been examined, record it and recursively clean its closure. diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R index b2b6f34aaa085..c4fcbecee18e9 100644 --- a/R/pkg/tests/fulltests/test_utils.R +++ b/R/pkg/tests/fulltests/test_utils.R @@ -110,6 +110,15 @@ test_that("cleanClosure on R functions", { actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) + # Test for combination for nested and sequenctial functions in a closure + f1 <- function(x) x + 1 + f2 <- function(x) f1(x) + 2 + userFunc <- function(x) { f1(x); f2(x) } + cUserFuncEnv <- environment(cleanClosure(userFunc)) + expect_equal(length(cUserFuncEnv), 2) + innerCUserFuncEnv <- environment(cUserFuncEnv$f2) + expect_equal(length(innerCUserFuncEnv), 1) + # Test for function (and variable) definitions. f <- function(x) { g <- function(y) { y * 2 } From 8469614c0513fbed87977d4e741649db3fdd8add Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 19 Nov 2019 09:08:20 +0900 Subject: [PATCH 33/83] [SPARK-25694][SQL][FOLLOW-UP] Move 'spark.sql.defaultUrlStreamHandlerFactory.enabled' into StaticSQLConf.scala ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency. ### Why are the changes needed? To put the similar configurations together and for readability. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested as described in https://github.com/apache/spark/pull/26530. Closes #26570 from HyukjinKwon/SPARK-25694. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../spark/sql/internal/StaticSQLConf.scala | 7 +++++ .../spark/sql/internal/SharedState.scala | 2 -- .../spark/sql/internal/config/package.scala | 29 ------------------- 3 files changed, 7 insertions(+), 31 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index d665d16ae4195..d2f27da239016 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -161,4 +161,11 @@ object StaticSQLConf { "defaults, dropping any overrides in its parent SparkSession.") .booleanConf .createWithDefault(false) + + val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = + buildStaticConf("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .internal() + .booleanConf + .createWithDefault(true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 81a9c76511d8b..de3805e105802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -36,8 +36,6 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED -import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala deleted file mode 100644 index e26c4aadaf135..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala +++ /dev/null @@ -1,29 +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.internal - -import org.apache.spark.internal.config.ConfigBuilder - -package object config { - - private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = - ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled") - .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") - .booleanConf - .createWithDefault(true) -} From 882f54b0a323fb5cd827d600b3c3332e1fcdf65a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 19 Nov 2019 09:11:41 +0900 Subject: [PATCH 34/83] [SPARK-29870][SQL][FOLLOW-UP] Keep CalendarInterval's toString ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/26418. This PR removed `CalendarInterval`'s `toString` with an unfinished changes. ### Why are the changes needed? 1. Ideally we should make each PR isolated and separate targeting one issue without touching unrelated codes. 2. There are some other places where the string formats were exposed to users. For example: ```scala scala> sql("select interval 1 days as a").selectExpr("to_csv(struct(a))").show() ``` ``` +--------------------------+ |to_csv(named_struct(a, a))| +--------------------------+ | "CalendarInterval...| +--------------------------+ ``` 3. Such fixes: ```diff private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() + val keyString = mapType.keyType match { + case CalendarIntervalType => + (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) + case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString + } ``` can cause performance regression due to type dispatch for each map. ### Does this PR introduce any user-facing change? Yes, see 2. case above. ### How was this patch tested? Manually tested. Closes #26572 from HyukjinKwon/SPARK-29783. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../spark/unsafe/types/CalendarInterval.java | 36 +++++++++++++++++-- .../sql/catalyst/expressions/literals.scala | 1 - .../sql/catalyst/json/JacksonGenerator.scala | 11 +----- .../sql/catalyst/util/IntervalUtils.scala | 30 ++-------------- 4 files changed, 37 insertions(+), 41 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 0464e886e1441..d2abb36b0727c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; +import java.math.BigDecimal; import java.time.Duration; import java.time.Period; import java.time.temporal.ChronoUnit; @@ -79,8 +80,39 @@ public int compareTo(CalendarInterval that) { @Override public String toString() { - return "CalendarInterval(months= " + months + ", days = " + days + ", microsecond = " + - microseconds + ")"; + if (months == 0 && days == 0 && microseconds == 0) { + return "0 seconds"; + } + + StringBuilder sb = new StringBuilder(); + + if (months != 0) { + appendUnit(sb, months / 12, "years"); + appendUnit(sb, months % 12, "months"); + } + + appendUnit(sb, days, "days"); + + if (microseconds != 0) { + long rest = microseconds; + appendUnit(sb, rest / MICROS_PER_HOUR, "hours"); + rest %= MICROS_PER_HOUR; + appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes"); + rest %= MICROS_PER_MINUTE; + if (rest != 0) { + String s = BigDecimal.valueOf(rest, 6).stripTrailingZeros().toPlainString(); + sb.append(s).append(" seconds "); + } + } + + sb.setLength(sb.length() - 1); + return sb.toString(); + } + + private void appendUnit(StringBuilder sb, long value, String unit) { + if (value != 0) { + sb.append(value).append(' ').append(unit).append(' '); + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index f426863de41a1..5a5d7a17acd99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -409,7 +409,6 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) s"TIMESTAMP('${formatter.format(v)}')" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" - case (v: CalendarInterval, CalendarIntervalType) => IntervalUtils.toMultiUnitsString(v) case _ => value.toString } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 544f6e9108325..aaf2ecf7923ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -119,10 +119,6 @@ private[sql] class JacksonGenerator( (row: SpecializedGetters, ordinal: Int) => gen.writeNumber(row.getDouble(ordinal)) - case CalendarIntervalType => - (row: SpecializedGetters, ordinal: Int) => - gen.writeString(IntervalUtils.toMultiUnitsString(row.getInterval(ordinal))) - case StringType => (row: SpecializedGetters, ordinal: Int) => gen.writeString(row.getUTF8String(ordinal).toString) @@ -218,15 +214,10 @@ private[sql] class JacksonGenerator( private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() - val keyString = mapType.keyType match { - case CalendarIntervalType => - (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) - case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString - } val valueArray = map.valueArray() var i = 0 while (i < map.numElements()) { - gen.writeFieldName(keyString(i)) + gen.writeFieldName(keyArray.get(i, mapType.keyType).toString) if (!valueArray.isNullAt(i)) { fieldWriter.apply(valueArray, i) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 725ff81ef410e..61bd9242f3e24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -332,34 +332,8 @@ object IntervalUtils { fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num) } - def toMultiUnitsString(interval: CalendarInterval): String = { - if (interval.months == 0 && interval.days == 0 && interval.microseconds == 0) { - return "0 seconds" - } - val sb = new StringBuilder - if (interval.months != 0) { - appendUnit(sb, interval.months / 12, "years") - appendUnit(sb, interval.months % 12, "months") - } - appendUnit(sb, interval.days, "days") - if (interval.microseconds != 0) { - var rest = interval.microseconds - appendUnit(sb, rest / MICROS_PER_HOUR, "hours") - rest %= MICROS_PER_HOUR - appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes") - rest %= MICROS_PER_MINUTE - if (rest != 0) { - val s = BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString - sb.append(s).append(" seconds ") - } - } - sb.setLength(sb.length - 1) - sb.toString - } - - private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { - if (value != 0) sb.append(value).append(' ').append(unit).append(' ') - } + // `toString` implementation in CalendarInterval is the multi-units format currently. + def toMultiUnitsString(interval: CalendarInterval): String = interval.toString def toSqlStandardString(interval: CalendarInterval): String = { val yearMonthPart = if (interval.months < 0) { From 28a502c6e92ce44601e798d203364422063c1e07 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 18 Nov 2019 18:13:11 -0800 Subject: [PATCH 35/83] [SPARK-28527][FOLLOW-UP][SQL][TEST] Add guides for ThriftServerQueryTestSuite ### What changes were proposed in this pull request? This PR add guides for `ThriftServerQueryTestSuite`. ### Why are the changes needed? Add guides ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26587 from wangyum/SPARK-28527-FOLLOW-UP. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../thriftserver/ThriftServerQueryTestSuite.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) 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 index b50a6045f5378..0c56cf9ef5d50 100644 --- 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 @@ -37,7 +37,16 @@ import org.apache.spark.sql.types._ /** * Re-run all the tests in SQLQueryTestSuite via Thrift Server. - * Note that this TestSuite does not support maven. + * + * To run the entire test suite: + * {{{ + * build/sbt "hive-thriftserver/test-only *ThriftServerQueryTestSuite" -Phive-thriftserver + * }}} + * + * This test suite won't generate golden files. To re-generate golden files for entire suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} * * TODO: * 1. Support UDF testing. @@ -74,6 +83,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } + // We only test this test suite with the default configuration to reduce test time. override val isTestWithConfigSets = false /** List of test cases to ignore, in lower cases. */ From a834dba120e3569e44c5e4b9f8db9c6eef58161b Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 18 Nov 2019 18:44:16 -0800 Subject: [PATCH 36/83] Revert "[SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils This reverts commit f7e53865 i.e PR #26301 from master Closes #26583 from shivsood/revert_29644_master. Authored-by: shivsood Signed-off-by: Dongjoon Hyun --- .../jdbc/MsSqlServerIntegrationSuite.scala | 48 ++----------------- .../sql/jdbc/MySQLIntegrationSuite.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 12 ++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 42 ---------------- 5 files changed, 13 insertions(+), 97 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index f1cd3343b7925..efd7ca74c796b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -59,7 +59,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """ |INSERT INTO numbers VALUES ( |0, - |127, 32767, 2147483647, 9223372036854775807, + |255, 32767, 2147483647, 9223372036854775807, |123456789012345.123456789012345, 123456789012345.123456789012345, |123456789012345.123456789012345, |123, 12345.12, @@ -119,7 +119,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { val types = row.toSeq.map(x => x.getClass.toString) assert(types.length == 12) assert(types(0).equals("class java.lang.Boolean")) - assert(types(1).equals("class java.lang.Byte")) + assert(types(1).equals("class java.lang.Integer")) assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Long")) @@ -131,7 +131,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(10).equals("class java.math.BigDecimal")) assert(types(11).equals("class java.math.BigDecimal")) assert(row.getBoolean(0) == false) - assert(row.getByte(1) == 127) + assert(row.getInt(1) == 255) assert(row.getShort(2) == 32767) assert(row.getInt(3) == 2147483647) assert(row.getLong(4) == 9223372036854775807L) @@ -202,46 +202,4 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { df2.write.jdbc(jdbcUrl, "datescopy", new Properties) df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) } - - test("SPARK-29644: Write tables with ShortType") { - import testImplicits._ - val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") - val tablename = "shorttable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Short") - } - - test("SPARK-29644: Write tables with ByteType") { - import testImplicits._ - val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") - val tablename = "bytetable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Byte") - } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index 8401b0a8a752f..bba1b5275269b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -84,7 +84,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types.length == 9) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Long")) - assert(types(2).equals("class java.lang.Short")) + assert(types(2).equals("class java.lang.Integer")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Integer")) assert(types(5).equals("class java.lang.Long")) @@ -93,7 +93,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(8).equals("class java.lang.Double")) assert(rows(0).getBoolean(0) == false) assert(rows(0).getLong(1) == 0x225) - assert(rows(0).getShort(2) == 17) + assert(rows(0).getInt(2) == 17) assert(rows(0).getInt(3) == 77777) assert(rows(0).getInt(4) == 123456789) assert(rows(0).getLong(5) == 123456789012345L) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index e3baa2a028da2..c1e1aed83bae5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -170,8 +170,8 @@ object JdbcUtils extends Logging { case LongType => Option(JdbcType("BIGINT", java.sql.Types.BIGINT)) case DoubleType => Option(JdbcType("DOUBLE PRECISION", java.sql.Types.DOUBLE)) case FloatType => Option(JdbcType("REAL", java.sql.Types.FLOAT)) - case ShortType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) - case ByteType => Option(JdbcType("TINYINT", java.sql.Types.TINYINT)) + case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) + case ByteType => Option(JdbcType("BYTE", java.sql.Types.TINYINT)) case BooleanType => Option(JdbcType("BIT(1)", java.sql.Types.BIT)) case StringType => Option(JdbcType("TEXT", java.sql.Types.CLOB)) case BinaryType => Option(JdbcType("BLOB", java.sql.Types.BLOB)) @@ -235,7 +235,7 @@ object JdbcUtils extends Logging { case java.sql.Types.REF => StringType case java.sql.Types.REF_CURSOR => null case java.sql.Types.ROWID => LongType - case java.sql.Types.SMALLINT => ShortType + case java.sql.Types.SMALLINT => IntegerType case java.sql.Types.SQLXML => StringType case java.sql.Types.STRUCT => StringType case java.sql.Types.TIME => TimestampType @@ -244,7 +244,7 @@ object JdbcUtils extends Logging { case java.sql.Types.TIMESTAMP => TimestampType case java.sql.Types.TIMESTAMP_WITH_TIMEZONE => null - case java.sql.Types.TINYINT => ByteType + case java.sql.Types.TINYINT => IntegerType case java.sql.Types.VARBINARY => BinaryType case java.sql.Types.VARCHAR => StringType case _ => @@ -546,11 +546,11 @@ object JdbcUtils extends Logging { case ShortType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setShort(pos + 1, row.getShort(pos)) + stmt.setInt(pos + 1, row.getShort(pos)) case ByteType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setByte(pos + 1, row.getByte(pos)) + stmt.setInt(pos + 1, row.getByte(pos)) case BooleanType => (stmt: PreparedStatement, row: Row, pos: Int) => 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 432697015a482..43f6381c19790 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 @@ -578,8 +578,8 @@ class JDBCSuite extends QueryTest assert(rows.length === 1) assert(rows(0).getInt(0) === 1) assert(rows(0).getBoolean(1) === false) - assert(rows(0).getByte(2) === 3.toByte) - assert(rows(0).getShort(3) === 4.toShort) + assert(rows(0).getInt(2) === 3) + assert(rows(0).getInt(3) === 4) assert(rows(0).getLong(4) === 1234567890123L) } 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 f43776ad91987..8021ef1a17a18 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 @@ -574,48 +574,6 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { } } - test("SPARK-29644: Write tables with ShortType") { - import testImplicits._ - val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") - val tablename = "shorttable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", url) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", url) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Short") - } - - test("SPARK-29644: Write tables with ByteType") { - import testImplicits._ - val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") - val tablename = "bytetable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", url) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", url) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Byte") - } - private def runAndVerifyRecordsWritten(expected: Long)(job: => Unit): Unit = { assert(expected === runAndReturnMetrics(job, _.taskMetrics.outputMetrics.recordsWritten)) } From 3d45779b687e3f8008044aa251d57024f2a31807 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 19 Nov 2019 12:03:29 +0800 Subject: [PATCH 37/83] [SPARK-29728][SQL] Datasource V2: Support ALTER TABLE RENAME TO ### What changes were proposed in this pull request? This PR adds `ALTER TABLE a.b.c RENAME TO x.y.x` support for V2 catalogs. ### Why are the changes needed? The current implementation doesn't support this command V2 catalogs. ### Does this PR introduce any user-facing change? Yes, now the renaming table works for v2 catalogs: ``` scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show +---------+---------+ |namespace|tableName| +---------+---------+ | ns1.ns2| old| +---------+---------+ scala> spark.sql("ALTER TABLE testcat.ns1.ns2.old RENAME TO testcat.ns1.ns2.new").show scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show +---------+---------+ |namespace|tableName| +---------+---------+ | ns1.ns2| new| +---------+---------+ ``` ### How was this patch tested? Added unit tests. Closes #26539 from imback82/rename_table. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 6 +++ .../sql/catalyst/parser/AstBuilder.scala | 16 ++++++++ .../catalyst/plans/logical/statements.scala | 8 ++++ .../catalyst/plans/logical/v2Commands.scala | 8 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 9 +++++ .../analysis/ResolveSessionCatalog.scala | 3 ++ .../spark/sql/execution/SparkSqlParser.scala | 16 -------- .../datasources/v2/DataSourceV2Strategy.scala | 5 ++- .../datasources/v2/RenameTableExec.scala | 40 +++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 18 +++++++++ .../execution/command/DDLParserSuite.scala | 26 ------------ 12 files changed, 114 insertions(+), 45 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 13f1f64ba7b8d..41f8e3552977d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -140,8 +140,8 @@ statement '(' columns=multipartIdentifierList ')' #dropTableColumns | ALTER TABLE multipartIdentifier DROP (COLUMN | COLUMNS) columns=multipartIdentifierList #dropTableColumns - | ALTER (TABLE | VIEW) from=tableIdentifier - RENAME TO to=tableIdentifier #renameTable + | ALTER (TABLE | VIEW) from=multipartIdentifier + RENAME TO to=multipartIdentifier #renameTable | ALTER (TABLE | VIEW) multipartIdentifier SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) multipartIdentifier diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 5a4fb2e865903..2f2e4e619eb4a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -100,6 +100,12 @@ class ResolveCatalogs(val catalogManager: CatalogManager) AlterNamespaceSetProperties( catalog.asNamespaceCatalog, nameParts, Map("location" -> location)) + case RenameTableStatement(NonSessionCatalog(catalog, oldName), newNameParts, isView) => + if (isView) { + throw new AnalysisException("Renaming view is not supported in v2 catalogs.") + } + RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier) + case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index bb8eab0de6089..5b9c204fcd4ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3245,6 +3245,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging query = plan(ctx.query)) } + /** + * Create a [[RenameTableStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE multi_part_name1 RENAME TO multi_part_name2; + * ALTER VIEW multi_part_name1 RENAME TO multi_part_name2; + * }}} + */ + override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { + RenameTableStatement( + visitMultipartIdentifier(ctx.from), + visitMultipartIdentifier(ctx.to), + ctx.VIEW != null) + } + /** * A command for users to list the properties for a table. If propertyKey is specified, the value * for the propertyKey is returned. If propertyKey is not specified, all the keys and their diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 7a98cccc3d7a4..7d7d6bdbfdd2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -246,6 +246,14 @@ case class AlterViewAsStatement( originalText: String, query: LogicalPlan) extends ParsedStatement +/** + * ALTER TABLE ... RENAME TO command, as parsed from SQL. + */ +case class RenameTableStatement( + oldName: Seq[String], + newName: Seq[String], + isView: Boolean) extends ParsedStatement + /** * A DROP TABLE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f7f8b2778d234..d87758a7df7b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -400,6 +400,14 @@ case class AlterTable( } } +/** + * The logical plan of the ALTER TABLE RENAME command that works for v2 tables. + */ +case class RenameTable( + catalog: TableCatalog, + oldIdent: Identifier, + newIdent: Identifier) extends Command + /** * The logical plan of the SHOW TABLE command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d3ca6f7a8eee2..d2575dabf847c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -623,6 +623,15 @@ class DDLParserSuite extends AnalysisTest { } } + test("alter table/view: rename table/view") { + comparePlans( + parsePlan("ALTER TABLE a.b.c RENAME TO x.y.z"), + RenameTableStatement(Seq("a", "b", "c"), Seq("x", "y", "z"), isView = false)) + comparePlans( + parsePlan("ALTER VIEW a.b.c RENAME TO x.y.z"), + RenameTableStatement(Seq("a", "b", "c"), Seq("x", "y", "z"), isView = true)) + } + test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), DescribeColumnStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 708203119f4bb..eb53e3accc3d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -179,6 +179,9 @@ class ResolveSessionCatalog( } AlterDatabaseSetLocationCommand(nameParts.head, location) + case RenameTableStatement(SessionCatalog(_, oldName), newNameParts, isView) => + AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { 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 8241f850e3aa7..44e60767e6b1a 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 @@ -316,22 +316,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.TEMPORARY != null) } - /** - * Create a [[AlterTableRenameCommand]] command. - * - * For example: - * {{{ - * ALTER TABLE table1 RENAME TO table2; - * ALTER VIEW view1 RENAME TO view2; - * }}} - */ - override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenameCommand( - visitTableIdentifier(ctx.from), - visitTableIdentifier(ctx.to), - ctx.VIEW != null) - } - /** * Convert a nested constants list into a sequence of string sequences. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 95caebe9e8938..a0d10f1d09e63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -204,6 +204,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case RenameTable(catalog, oldIdent, newIdent) => + RenameTableExec(catalog, oldIdent, newIdent) :: Nil + case AlterNamespaceSetProperties(catalog, namespace, properties) => AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala new file mode 100644 index 0000000000000..a650607d5f129 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +/** + * Physical plan node for renaming a table. + */ +case class RenameTableExec( + catalog: TableCatalog, + oldIdent: Identifier, + newIdent: Identifier) extends V2CommandExec { + + override def output: Seq[Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + catalog.invalidateTable(oldIdent) + catalog.renameTable(oldIdent, newIdent) + + Seq.empty + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 36d2deaa309ee..2a44251e102a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1535,6 +1535,24 @@ class DataSourceV2SQLSuite } } + test("AlterTable: rename table basic test") { + withTable("testcat.ns1.new") { + sql(s"CREATE TABLE testcat.ns1.ns2.old USING foo AS SELECT id, data FROM source") + checkAnswer(sql("SHOW TABLES FROM testcat.ns1.ns2"), Seq(Row("ns1.ns2", "old"))) + + sql(s"ALTER TABLE testcat.ns1.ns2.old RENAME TO ns1.new") + checkAnswer(sql("SHOW TABLES FROM testcat.ns1.ns2"), Seq.empty) + checkAnswer(sql("SHOW TABLES FROM testcat.ns1"), Seq(Row("ns1", "new"))) + } + } + + test("AlterTable: renaming views are not supported") { + val e = intercept[AnalysisException] { + sql(s"ALTER VIEW testcat.ns.tbl RENAME TO ns.view") + } + assert(e.getMessage.contains("Renaming view is not supported in v2 catalogs")) + } + test("ANALYZE TABLE") { val t = "testcat.ns1.ns2.tbl" withTable(t) { 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 11131310fa4f2..41a2168b4a18a 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 @@ -373,32 +373,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "Directory path and 'path' in OPTIONS should be specified one, but not both")) } - // ALTER TABLE table_name RENAME TO new_table_name; - // ALTER VIEW view_name RENAME TO new_view_name; - test("alter table/view: rename table/view") { - val sql_table = "ALTER TABLE table_name RENAME TO new_table_name" - val sql_view = sql_table.replace("TABLE", "VIEW") - val parsed_table = parser.parsePlan(sql_table) - val parsed_view = parser.parsePlan(sql_view) - val expected_table = AlterTableRenameCommand( - TableIdentifier("table_name"), - TableIdentifier("new_table_name"), - isView = false) - val expected_view = AlterTableRenameCommand( - TableIdentifier("table_name"), - TableIdentifier("new_table_name"), - isView = true) - comparePlans(parsed_table, expected_table) - comparePlans(parsed_view, expected_view) - } - - test("alter table: rename table with database") { - val query = "ALTER TABLE db1.tbl RENAME TO db1.tbl2" - val plan = parseAs[AlterTableRenameCommand](query) - assert(plan.oldName == TableIdentifier("tbl", Some("db1"))) - assert(plan.newName == TableIdentifier("tbl2", Some("db1"))) - } - test("alter table - property values must be set") { assertUnsupported( sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", From 2e71a6e7ba20743e22a234f5209c1b120f2a7948 Mon Sep 17 00:00:00 2001 From: yudovin Date: Mon, 18 Nov 2019 22:05:34 -0800 Subject: [PATCH 38/83] [SPARK-27558][CORE] Gracefully cleanup task when it fails with OOM exception ### What changes were proposed in this pull request? When a task fails with OOM exception, the `UnsafeInMemorySorter.array` could be `null`. In the meanwhile, the `cleanupResources()` on task completion would call `UnsafeInMemorySorter.getMemoryUsage` in turn, and that lead to another NPE thrown. ### Why are the changes needed? Check if `array` is null in `UnsafeInMemorySorter.getMemoryUsage` and it should help to avoid NPE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? It was tested manually. Closes #26349 from ayudovin/fix-npe-in-listener. Authored-by: yudovin Signed-off-by: Xingbo Jiang --- .../util/collection/unsafe/sort/UnsafeInMemorySorter.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 1a9453a8b3e80..e14964d68119b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -205,6 +205,10 @@ public long getSortTimeNanos() { } public long getMemoryUsage() { + if (array == null) { + return 0L; + } + return array.size() * 8; } From 5ac37a82656f440e8f58564974668ed9e0ca6b72 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Nov 2019 15:22:08 +0800 Subject: [PATCH 39/83] [SPARK-29869][SQL] improve error message in HiveMetastoreCatalog#convertToLogicalRelation ### What changes were proposed in this pull request? In our production, HiveMetastoreCatalog#convertToLogicalRelation throws AssertError occasionally: ```sql scala> spark.table("hive_table").show java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:208) at org.apache.spark.sql.hive.HiveMetastoreCatalog.convertToLogicalRelation(HiveMetastoreCatalog.scala:261) at org.apache.spark.sql.hive.HiveMetastoreCatalog.convert(HiveMetastoreCatalog.scala:137) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:220) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:207) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$2(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$4(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:73) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:72) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:29) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:207) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:191) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:130) at scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60) at scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68) at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:49) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:127) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:119) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:119) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:168) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:162) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:122) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:98) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:98) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:146) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:145) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:66) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:63) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:63) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:55) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:86) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:585) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:581) ... 47 elided ```` Most of cases occurred in reading a table which created by an old Spark version. After recreated the table, the issue will be gone. After deep dive, the root cause is this external table is a non-partitioned table but the `LOCATION` set to a partitioned path {{/tablename/dt=yyyymmdd}}. The partitionSpec is inferred. ### Why are the changes needed? Above error message is very confused. We need more details about assert failure information. This issue caused by `PartitioningAwareFileIndex#inferPartitioning()`. For non-HiveMetastore Spark, it's useful. But for Hive table, it shouldn't infer partition if Hive tell us it's a non partitioned table. (new added) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add UT. Closes #26499 from LantaoJin/SPARK-29869. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++++++++++--- .../sql/hive/HiveMetastoreCatalogSuite.scala | 22 ++++++++++++++++++- 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 5ad2caba07fc0..2981e391c0439 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -257,8 +257,20 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } // The inferred schema may have different field names as the table schema, we should respect // it, but also respect the exprId in table relation output. - assert(result.output.length == relation.output.length && - result.output.zip(relation.output).forall { case (a1, a2) => a1.dataType == a2.dataType }) + if (result.output.length != relation.output.length) { + throw new AnalysisException( + s"Converted table has ${result.output.length} columns, " + + s"but source Hive table has ${relation.output.length} columns. " + + s"Set ${HiveUtils.CONVERT_METASTORE_PARQUET.key} to false, " + + s"or recreate table ${relation.tableMeta.identifier} to workaround.") + } + if (!result.output.zip(relation.output).forall { + case (a1, a2) => a1.dataType == a2.dataType }) { + throw new AnalysisException( + s"Column in converted table has different data type with source Hive table's. " + + s"Set ${HiveUtils.CONVERT_METASTORE_PARQUET.key} to false, " + + s"or recreate table ${relation.tableMeta.identifier} to workaround.") + } val newOutput = result.output.zip(relation.output).map { case (a1, a2) => a1.withExprId(a2.exprId) } 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 0e45e18c4b175..9f2906df19bff 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -358,4 +358,24 @@ class DataSourceWithHiveMetastoreCatalogSuite Seq(table("src").count().toString)) } } + + test("SPARK-29869: Fix convertToLogicalRelation throws unclear AssertionError") { + withTempPath(dir => { + val baseDir = s"${dir.getCanonicalFile.toURI.toString}/non_partition_table" + val partitionLikeDir = s"$baseDir/dt=20191113" + spark.range(3).selectExpr("id").write.parquet(partitionLikeDir) + withTable("non_partition_table") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { + spark.sql( + s""" + |CREATE TABLE non_partition_table (id bigint) + |STORED AS PARQUET LOCATION '$baseDir' + |""".stripMargin) + val e = intercept[AnalysisException]( + spark.table("non_partition_table")).getMessage + assert(e.contains("Converted table has 2 columns, but source Hive table has 1 columns.")) + } + } + }) + } } From 16134d6d0f349a6880c90770ab7e393a266b978d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2019 15:37:35 +0800 Subject: [PATCH 40/83] [SPARK-29948][SQL] make the default alias consistent between date, timestamp and interval ### What changes were proposed in this pull request? Update `Literal.sql` to make date, timestamp and interval consistent. They should all use the `TYPE 'value'` format. ### Why are the changes needed? Make the default alias consistent. For example, without this patch we will see ``` scala> sql("select interval '1 day', date '2000-10-10'").show +------+-----------------+ |1 days|DATE '2000-10-10'| +------+-----------------+ |1 days| 2000-10-10| +------+-----------------+ ``` ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26579 from cloud-fan/sql. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/literals.scala | 4 +- .../ExpressionSQLBuilderSuite.scala | 6 +- .../expressions/LiteralExpressionSuite.scala | 2 +- .../sql-tests/results/ansi/interval.sql.out | 136 +++++++++--------- .../sql-tests/results/ansi/literals.sql.out | 2 +- .../resources/sql-tests/results/array.sql.out | 2 +- .../resources/sql-tests/results/cast.sql.out | 2 +- .../sql-tests/results/datetime.sql.out | 4 +- .../results/interval-display-iso_8601.sql.out | 2 +- .../interval-display-sql_standard.sql.out | 2 +- .../results/interval-display.sql.out | 2 +- .../sql-tests/results/interval.sql.out | 126 ++++++++-------- .../sql-tests/results/literals.sql.out | 2 +- .../results/postgreSQL/interval.sql.out | 50 +++---- .../subquery/in-subquery/in-joins.sql.out | 13 +- .../typeCoercion/native/arrayJoin.sql.out | 2 +- .../native/dateTimeOperations.sql.out | 78 +++++----- .../sql-tests/results/udf/udf-window.sql.out | 2 +- .../sql-tests/results/window.sql.out | 2 +- 19 files changed, 225 insertions(+), 214 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 5a5d7a17acd99..48b8c9c0fbf8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -407,7 +407,9 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case (v: Long, TimestampType) => val formatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) - s"TIMESTAMP('${formatter.format(v)}')" + s"TIMESTAMP '${formatter.format(v)}'" + case (i: CalendarInterval, CalendarIntervalType) => + s"INTERVAL '${IntervalUtils.toMultiUnitsString(i)}'" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" case _ => value.toString } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index ada3f7abd7e3a..492d97ba9d524 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -89,7 +89,7 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0, 987654321) .atZone(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) .toInstant - checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00.987654')") + checkSQL(Literal(timestamp), "TIMESTAMP '2016-01-01 00:00:00.987654'") // TODO tests for decimals } @@ -169,12 +169,12 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { checkSQL( TimeAdd('a, interval), - "`a` + 1 hours" + "`a` + INTERVAL '1 hours'" ) checkSQL( TimeSub('a, interval), - "`a` - 1 hours" + "`a` - INTERVAL '1 hours'" ) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 03c9cf9c8a94d..4714635a3370b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -302,7 +302,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { val timestamp = LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000) .atZone(ZoneOffset.UTC) .toInstant - val expected = "TIMESTAMP('2019-03-21 01:02:03.456')" + val expected = "TIMESTAMP '2019-03-21 01:02:03.456'" val literalStr = Literal.create(timestamp).sql assert(literalStr === expected) } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 73bf299c509cf..11c70d4987eec 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -5,7 +5,7 @@ -- !query 0 select interval '1 day' > interval '23 hour' -- !query 0 schema -struct<(1 days > 23 hours):boolean> +struct<(INTERVAL '1 days' > INTERVAL '23 hours'):boolean> -- !query 0 output true @@ -13,7 +13,7 @@ true -- !query 1 select interval '-1 day' >= interval '-23 hour' -- !query 1 schema -struct<(-1 days >= -23 hours):boolean> +struct<(INTERVAL '-1 days' >= INTERVAL '-23 hours'):boolean> -- !query 1 output false @@ -21,7 +21,7 @@ false -- !query 2 select interval '-1 day' > null -- !query 2 schema -struct<(-1 days > CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '-1 days' > CAST(NULL AS INTERVAL)):boolean> -- !query 2 output NULL @@ -29,7 +29,7 @@ NULL -- !query 3 select null > interval '-1 day' -- !query 3 schema -struct<(CAST(NULL AS INTERVAL) > -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) > INTERVAL '-1 days'):boolean> -- !query 3 output NULL @@ -37,7 +37,7 @@ NULL -- !query 4 select interval '1 minutes' < interval '1 hour' -- !query 4 schema -struct<(1 minutes < 1 hours):boolean> +struct<(INTERVAL '1 minutes' < INTERVAL '1 hours'):boolean> -- !query 4 output true @@ -45,7 +45,7 @@ true -- !query 5 select interval '-1 day' <= interval '-23 hour' -- !query 5 schema -struct<(-1 days <= -23 hours):boolean> +struct<(INTERVAL '-1 days' <= INTERVAL '-23 hours'):boolean> -- !query 5 output true @@ -53,7 +53,7 @@ true -- !query 6 select interval '1 year' = interval '360 days' -- !query 6 schema -struct<(1 years = 360 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '360 days'):boolean> -- !query 6 output true @@ -61,7 +61,7 @@ true -- !query 7 select interval '1 year 2 month' = interval '420 days' -- !query 7 schema -struct<(1 years 2 months = 420 days):boolean> +struct<(INTERVAL '1 years 2 months' = INTERVAL '420 days'):boolean> -- !query 7 output true @@ -69,7 +69,7 @@ true -- !query 8 select interval '1 year' = interval '365 days' -- !query 8 schema -struct<(1 years = 365 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '365 days'):boolean> -- !query 8 output false @@ -77,7 +77,7 @@ false -- !query 9 select interval '1 month' = interval '30 days' -- !query 9 schema -struct<(1 months = 30 days):boolean> +struct<(INTERVAL '1 months' = INTERVAL '30 days'):boolean> -- !query 9 output true @@ -85,7 +85,7 @@ true -- !query 10 select interval '1 minutes' = interval '1 hour' -- !query 10 schema -struct<(1 minutes = 1 hours):boolean> +struct<(INTERVAL '1 minutes' = INTERVAL '1 hours'):boolean> -- !query 10 output false @@ -93,7 +93,7 @@ false -- !query 11 select interval '1 minutes' = null -- !query 11 schema -struct<(1 minutes = CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' = CAST(NULL AS INTERVAL)):boolean> -- !query 11 output NULL @@ -101,7 +101,7 @@ NULL -- !query 12 select null = interval '-1 day' -- !query 12 schema -struct<(CAST(NULL AS INTERVAL) = -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) = INTERVAL '-1 days'):boolean> -- !query 12 output NULL @@ -109,7 +109,7 @@ NULL -- !query 13 select interval '1 minutes' <=> null -- !query 13 schema -struct<(1 minutes <=> CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' <=> CAST(NULL AS INTERVAL)):boolean> -- !query 13 output false @@ -117,7 +117,7 @@ false -- !query 14 select null <=> interval '1 minutes' -- !query 14 schema -struct<(CAST(NULL AS INTERVAL) <=> 1 minutes):boolean> +struct<(CAST(NULL AS INTERVAL) <=> INTERVAL '1 minutes'):boolean> -- !query 14 output false @@ -125,7 +125,7 @@ false -- !query 15 select INTERVAL '9 years 1 months -1 weeks -4 days -10 hours -46 minutes' > interval '1 minutes' -- !query 15 schema -struct<(9 years 1 months -11 days -10 hours -46 minutes > 1 minutes):boolean> +struct<(INTERVAL '9 years 1 months -11 days -10 hours -46 minutes' > INTERVAL '1 minutes'):boolean> -- !query 15 output true @@ -143,7 +143,7 @@ struct -- !query 17 select interval '1 month 120 days' > interval '2 month' -- !query 17 schema -struct<(1 months 120 days > 2 months):boolean> +struct<(INTERVAL '1 months 120 days' > INTERVAL '2 months'):boolean> -- !query 17 output true @@ -151,7 +151,7 @@ true -- !query 18 select interval '1 month 30 days' = interval '2 month' -- !query 18 schema -struct<(1 months 30 days = 2 months):boolean> +struct<(INTERVAL '1 months 30 days' = INTERVAL '2 months'):boolean> -- !query 18 output true @@ -159,7 +159,7 @@ true -- !query 19 select interval '1 month 29 days 40 hours' > interval '2 month' -- !query 19 schema -struct<(1 months 29 days 40 hours > 2 months):boolean> +struct<(INTERVAL '1 months 29 days 40 hours' > INTERVAL '2 months'):boolean> -- !query 19 output true @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -191,7 +191,7 @@ struct +struct -- !query 23 output 6 months 21 days 0.000005 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours @@ -207,7 +207,7 @@ struct +struct -- !query 25 output NULL @@ -215,7 +215,7 @@ NULL -- !query 26 select interval '2 seconds' / null -- !query 26 schema -struct +struct -- !query 26 output NULL @@ -223,7 +223,7 @@ NULL -- !query 27 select interval '2 seconds' * null -- !query 27 schema -struct +struct -- !query 27 output NULL @@ -231,7 +231,7 @@ NULL -- !query 28 select null * interval '2 seconds' -- !query 28 schema -struct +struct -- !query 28 output NULL @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 30 output 1 months -1 days 1 seconds @@ -255,7 +255,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct<-1 months 1 days -1 seconds:interval> -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 32 output -1 months 1 days -1 seconds @@ -407,7 +407,7 @@ NULL -- !query 50 select justify_days(interval '1 month 59 day 25 hour') -- !query 50 schema -struct +struct -- !query 50 output 2 months 29 days 25 hours @@ -415,7 +415,7 @@ struct -- !query 51 select justify_hours(interval '1 month 59 day 25 hour') -- !query 51 schema -struct +struct -- !query 51 output 1 months 60 days 1 hours @@ -423,7 +423,7 @@ struct -- !query 52 select justify_interval(interval '1 month 59 day 25 hour') -- !query 52 schema -struct +struct -- !query 52 output 3 months 1 hours @@ -431,7 +431,7 @@ struct -- !query 53 select justify_days(interval '1 month -59 day 25 hour') -- !query 53 schema -struct +struct -- !query 53 output -29 days 25 hours @@ -439,7 +439,7 @@ struct -- !query 54 select justify_hours(interval '1 month -59 day 25 hour') -- !query 54 schema -struct +struct -- !query 54 output 1 months -57 days -23 hours @@ -447,7 +447,7 @@ struct -- !query 55 select justify_interval(interval '1 month -59 day 25 hour') -- !query 55 schema -struct +struct -- !query 55 output -27 days -23 hours @@ -455,7 +455,7 @@ struct -- !query 56 select justify_days(interval '1 month 59 day -25 hour') -- !query 56 schema -struct +struct -- !query 56 output 2 months 29 days -25 hours @@ -463,7 +463,7 @@ struct -- !query 57 select justify_hours(interval '1 month 59 day -25 hour') -- !query 57 schema -struct +struct -- !query 57 output 1 months 57 days 23 hours @@ -471,7 +471,7 @@ struct -- !query 58 select justify_interval(interval '1 month 59 day -25 hour') -- !query 58 schema -struct +struct -- !query 58 output 2 months 27 days 23 hours @@ -479,7 +479,7 @@ struct -- !query 59 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 59 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> +struct -- !query 59 output 13.123456 seconds -13.123456 seconds @@ -487,7 +487,7 @@ struct<13.123456 seconds:interval,-13.123456 seconds:interval> -- !query 60 select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond -- !query 60 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +struct -- !query 60 output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds @@ -495,7 +495,7 @@ struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> -- !query 61 select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 61 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +struct -- !query 61 output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds @@ -503,7 +503,7 @@ struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interva -- !query 62 select interval '0 0:0:0.1' day to second -- !query 62 schema -struct<0.1 seconds:interval> +struct -- !query 62 output 0.1 seconds @@ -511,7 +511,7 @@ struct<0.1 seconds:interval> -- !query 63 select interval '10-9' year to month -- !query 63 schema -struct<10 years 9 months:interval> +struct -- !query 63 output 10 years 9 months @@ -519,7 +519,7 @@ struct<10 years 9 months:interval> -- !query 64 select interval '20 15:40:32.99899999' day to hour -- !query 64 schema -struct<20 days 15 hours:interval> +struct -- !query 64 output 20 days 15 hours @@ -527,7 +527,7 @@ struct<20 days 15 hours:interval> -- !query 65 select interval '20 15:40:32.99899999' day to minute -- !query 65 schema -struct<20 days 15 hours 40 minutes:interval> +struct -- !query 65 output 20 days 15 hours 40 minutes @@ -535,7 +535,7 @@ struct<20 days 15 hours 40 minutes:interval> -- !query 66 select interval '20 15:40:32.99899999' day to second -- !query 66 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 66 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -543,7 +543,7 @@ struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> -- !query 67 select interval '15:40:32.99899999' hour to minute -- !query 67 schema -struct<15 hours 40 minutes:interval> +struct -- !query 67 output 15 hours 40 minutes @@ -551,7 +551,7 @@ struct<15 hours 40 minutes:interval> -- !query 68 select interval '15:40.99899999' hour to second -- !query 68 schema -struct<15 minutes 40.998999 seconds:interval> +struct -- !query 68 output 15 minutes 40.998999 seconds @@ -559,7 +559,7 @@ struct<15 minutes 40.998999 seconds:interval> -- !query 69 select interval '15:40' hour to second -- !query 69 schema -struct<15 hours 40 minutes:interval> +struct -- !query 69 output 15 hours 40 minutes @@ -567,7 +567,7 @@ struct<15 hours 40 minutes:interval> -- !query 70 select interval '15:40:32.99899999' hour to second -- !query 70 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 70 output 15 hours 40 minutes 32.998999 seconds @@ -575,7 +575,7 @@ struct<15 hours 40 minutes 32.998999 seconds:interval> -- !query 71 select interval '20 40:32.99899999' minute to second -- !query 71 schema -struct<20 days 40 minutes 32.998999 seconds:interval> +struct -- !query 71 output 20 days 40 minutes 32.998999 seconds @@ -583,7 +583,7 @@ struct<20 days 40 minutes 32.998999 seconds:interval> -- !query 72 select interval '40:32.99899999' minute to second -- !query 72 schema -struct<40 minutes 32.998999 seconds:interval> +struct -- !query 72 output 40 minutes 32.998999 seconds @@ -591,7 +591,7 @@ struct<40 minutes 32.998999 seconds:interval> -- !query 73 select interval '40:32' minute to second -- !query 73 schema -struct<40 minutes 32 seconds:interval> +struct -- !query 73 output 40 minutes 32 seconds @@ -627,7 +627,7 @@ select interval 10 nanoseconds -- !query 76 select map(1, interval 1 day, 2, interval 3 week) -- !query 76 schema -struct> +struct> -- !query 76 output {1:1 days,2:21 days} @@ -635,7 +635,7 @@ struct> -- !query 77 select interval 'interval 3 year 1 hour' -- !query 77 schema -struct<3 years 1 hours:interval> +struct -- !query 77 output 3 years 1 hours @@ -643,7 +643,7 @@ struct<3 years 1 hours:interval> -- !query 78 select interval '3 year 1 hour' -- !query 78 schema -struct<3 years 1 hours:interval> +struct -- !query 78 output 3 years 1 hours @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1056,7 +1056,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1067,7 +1067,7 @@ select interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic -- !query 114 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> -- !query 114 output 5 years 5 months -1 years -1 months @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1099,7 +1099,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 @@ -1110,7 +1110,7 @@ select interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 117 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> -- !query 117 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds @@ -1118,7 +1118,7 @@ struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minute -- !query 118 select 1 year 2 days -- !query 118 schema -struct<1 years 2 days:interval> +struct -- !query 118 output 1 years 2 days @@ -1126,7 +1126,7 @@ struct<1 years 2 days:interval> -- !query 119 select '10-9' year to month -- !query 119 schema -struct<10 years 9 months:interval> +struct -- !query 119 output 10 years 9 months @@ -1134,7 +1134,7 @@ struct<10 years 9 months:interval> -- !query 120 select '20 15:40:32.99899999' day to second -- !query 120 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 120 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -1156,7 +1156,7 @@ select 30 day day -- !query 122 select date'2012-01-01' - '2-2' year to month -- !query 122 schema -struct +struct -- !query 122 output 2009-11-01 @@ -1164,7 +1164,7 @@ struct +struct -- !query 123 output 1 months -1 days diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index e43e88c8c72a8..6ee7148755c34 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -301,7 +301,7 @@ select date 'mar 11 2016' -- !query 32 select tImEstAmp '2016-03-11 20:54:00.000' -- !query 32 schema -struct +struct -- !query 32 output 2016-03-11 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 5f5d988771847..4195205e275ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -92,7 +92,7 @@ select array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays -- !query 6 schema -struct +struct -- !query 6 output true false true false true false true false true false true false true false true false true false true false diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 4884f9d491f2d..609d283da555c 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -284,6 +284,6 @@ struct -- !query 34 SELECT CAST(interval 3 month 1 hour AS string) -- !query 34 schema -struct +struct -- !query 34 output 3 months 1 hours diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index a40f5acb6d538..b2c6b878f4715 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -134,7 +134,7 @@ struct -- !query 15 select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' -- !query 15 schema -struct +struct -- !query 15 output 2078 hours 48 minutes 47.654322 seconds @@ -142,6 +142,6 @@ struct +struct -- !query 16 output -2078 hours -48 minutes -47.654322 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out index 57fe8a3f4fcc6..0e053940df38f 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out index 9e40f52151475..2633522bb1d08 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out index 340496e404326..faf27bda1549f 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index d0f2ff4b44d01..6d09f62a6dcec 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -5,7 +5,7 @@ -- !query 0 select interval '1 day' > interval '23 hour' -- !query 0 schema -struct<(1 days > 23 hours):boolean> +struct<(INTERVAL '1 days' > INTERVAL '23 hours'):boolean> -- !query 0 output true @@ -13,7 +13,7 @@ true -- !query 1 select interval '-1 day' >= interval '-23 hour' -- !query 1 schema -struct<(-1 days >= -23 hours):boolean> +struct<(INTERVAL '-1 days' >= INTERVAL '-23 hours'):boolean> -- !query 1 output false @@ -21,7 +21,7 @@ false -- !query 2 select interval '-1 day' > null -- !query 2 schema -struct<(-1 days > CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '-1 days' > CAST(NULL AS INTERVAL)):boolean> -- !query 2 output NULL @@ -29,7 +29,7 @@ NULL -- !query 3 select null > interval '-1 day' -- !query 3 schema -struct<(CAST(NULL AS INTERVAL) > -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) > INTERVAL '-1 days'):boolean> -- !query 3 output NULL @@ -37,7 +37,7 @@ NULL -- !query 4 select interval '1 minutes' < interval '1 hour' -- !query 4 schema -struct<(1 minutes < 1 hours):boolean> +struct<(INTERVAL '1 minutes' < INTERVAL '1 hours'):boolean> -- !query 4 output true @@ -45,7 +45,7 @@ true -- !query 5 select interval '-1 day' <= interval '-23 hour' -- !query 5 schema -struct<(-1 days <= -23 hours):boolean> +struct<(INTERVAL '-1 days' <= INTERVAL '-23 hours'):boolean> -- !query 5 output true @@ -53,7 +53,7 @@ true -- !query 6 select interval '1 year' = interval '360 days' -- !query 6 schema -struct<(1 years = 360 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '360 days'):boolean> -- !query 6 output true @@ -61,7 +61,7 @@ true -- !query 7 select interval '1 year 2 month' = interval '420 days' -- !query 7 schema -struct<(1 years 2 months = 420 days):boolean> +struct<(INTERVAL '1 years 2 months' = INTERVAL '420 days'):boolean> -- !query 7 output true @@ -69,7 +69,7 @@ true -- !query 8 select interval '1 year' = interval '365 days' -- !query 8 schema -struct<(1 years = 365 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '365 days'):boolean> -- !query 8 output false @@ -77,7 +77,7 @@ false -- !query 9 select interval '1 month' = interval '30 days' -- !query 9 schema -struct<(1 months = 30 days):boolean> +struct<(INTERVAL '1 months' = INTERVAL '30 days'):boolean> -- !query 9 output true @@ -85,7 +85,7 @@ true -- !query 10 select interval '1 minutes' = interval '1 hour' -- !query 10 schema -struct<(1 minutes = 1 hours):boolean> +struct<(INTERVAL '1 minutes' = INTERVAL '1 hours'):boolean> -- !query 10 output false @@ -93,7 +93,7 @@ false -- !query 11 select interval '1 minutes' = null -- !query 11 schema -struct<(1 minutes = CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' = CAST(NULL AS INTERVAL)):boolean> -- !query 11 output NULL @@ -101,7 +101,7 @@ NULL -- !query 12 select null = interval '-1 day' -- !query 12 schema -struct<(CAST(NULL AS INTERVAL) = -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) = INTERVAL '-1 days'):boolean> -- !query 12 output NULL @@ -109,7 +109,7 @@ NULL -- !query 13 select interval '1 minutes' <=> null -- !query 13 schema -struct<(1 minutes <=> CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' <=> CAST(NULL AS INTERVAL)):boolean> -- !query 13 output false @@ -117,7 +117,7 @@ false -- !query 14 select null <=> interval '1 minutes' -- !query 14 schema -struct<(CAST(NULL AS INTERVAL) <=> 1 minutes):boolean> +struct<(CAST(NULL AS INTERVAL) <=> INTERVAL '1 minutes'):boolean> -- !query 14 output false @@ -125,7 +125,7 @@ false -- !query 15 select INTERVAL '9 years 1 months -1 weeks -4 days -10 hours -46 minutes' > interval '1 minutes' -- !query 15 schema -struct<(9 years 1 months -11 days -10 hours -46 minutes > 1 minutes):boolean> +struct<(INTERVAL '9 years 1 months -11 days -10 hours -46 minutes' > INTERVAL '1 minutes'):boolean> -- !query 15 output true @@ -143,7 +143,7 @@ struct -- !query 17 select interval '1 month 120 days' > interval '2 month' -- !query 17 schema -struct<(1 months 120 days > 2 months):boolean> +struct<(INTERVAL '1 months 120 days' > INTERVAL '2 months'):boolean> -- !query 17 output true @@ -151,7 +151,7 @@ true -- !query 18 select interval '1 month 30 days' = interval '2 month' -- !query 18 schema -struct<(1 months 30 days = 2 months):boolean> +struct<(INTERVAL '1 months 30 days' = INTERVAL '2 months'):boolean> -- !query 18 output true @@ -159,7 +159,7 @@ true -- !query 19 select interval '1 month 29 days 40 hours' > interval '2 month' -- !query 19 schema -struct<(1 months 29 days 40 hours > 2 months):boolean> +struct<(INTERVAL '1 months 29 days 40 hours' > INTERVAL '2 months'):boolean> -- !query 19 output true @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -191,7 +191,7 @@ struct +struct -- !query 23 output 6 months 21 days 0.000005 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours @@ -207,7 +207,7 @@ struct +struct -- !query 25 output NULL @@ -215,7 +215,7 @@ NULL -- !query 26 select interval '2 seconds' / null -- !query 26 schema -struct +struct -- !query 26 output NULL @@ -223,7 +223,7 @@ NULL -- !query 27 select interval '2 seconds' * null -- !query 27 schema -struct +struct -- !query 27 output NULL @@ -231,7 +231,7 @@ NULL -- !query 28 select null * interval '2 seconds' -- !query 28 schema -struct +struct -- !query 28 output NULL @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 30 output 1 months -1 days 1 seconds @@ -255,7 +255,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct<-1 months 1 days -1 seconds:interval> -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 32 output -1 months 1 days -1 seconds @@ -407,7 +407,7 @@ NULL -- !query 50 select justify_days(interval '1 month 59 day 25 hour') -- !query 50 schema -struct +struct -- !query 50 output 2 months 29 days 25 hours @@ -415,7 +415,7 @@ struct -- !query 51 select justify_hours(interval '1 month 59 day 25 hour') -- !query 51 schema -struct +struct -- !query 51 output 1 months 60 days 1 hours @@ -423,7 +423,7 @@ struct -- !query 52 select justify_interval(interval '1 month 59 day 25 hour') -- !query 52 schema -struct +struct -- !query 52 output 3 months 1 hours @@ -431,7 +431,7 @@ struct -- !query 53 select justify_days(interval '1 month -59 day 25 hour') -- !query 53 schema -struct +struct -- !query 53 output -29 days 25 hours @@ -439,7 +439,7 @@ struct -- !query 54 select justify_hours(interval '1 month -59 day 25 hour') -- !query 54 schema -struct +struct -- !query 54 output 1 months -57 days -23 hours @@ -447,7 +447,7 @@ struct -- !query 55 select justify_interval(interval '1 month -59 day 25 hour') -- !query 55 schema -struct +struct -- !query 55 output -27 days -23 hours @@ -455,7 +455,7 @@ struct -- !query 56 select justify_days(interval '1 month 59 day -25 hour') -- !query 56 schema -struct +struct -- !query 56 output 2 months 29 days -25 hours @@ -463,7 +463,7 @@ struct -- !query 57 select justify_hours(interval '1 month 59 day -25 hour') -- !query 57 schema -struct +struct -- !query 57 output 1 months 57 days 23 hours @@ -471,7 +471,7 @@ struct -- !query 58 select justify_interval(interval '1 month 59 day -25 hour') -- !query 58 schema -struct +struct -- !query 58 output 2 months 27 days 23 hours @@ -479,7 +479,7 @@ struct -- !query 59 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 59 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> +struct -- !query 59 output 13.123456 seconds -13.123456 seconds @@ -487,7 +487,7 @@ struct<13.123456 seconds:interval,-13.123456 seconds:interval> -- !query 60 select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond -- !query 60 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +struct -- !query 60 output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds @@ -495,7 +495,7 @@ struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> -- !query 61 select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 61 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +struct -- !query 61 output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds @@ -503,7 +503,7 @@ struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interva -- !query 62 select interval '0 0:0:0.1' day to second -- !query 62 schema -struct<0.1 seconds:interval> +struct -- !query 62 output 0.1 seconds @@ -511,7 +511,7 @@ struct<0.1 seconds:interval> -- !query 63 select interval '10-9' year to month -- !query 63 schema -struct<10 years 9 months:interval> +struct -- !query 63 output 10 years 9 months @@ -519,7 +519,7 @@ struct<10 years 9 months:interval> -- !query 64 select interval '20 15:40:32.99899999' day to hour -- !query 64 schema -struct<20 days 15 hours:interval> +struct -- !query 64 output 20 days 15 hours @@ -527,7 +527,7 @@ struct<20 days 15 hours:interval> -- !query 65 select interval '20 15:40:32.99899999' day to minute -- !query 65 schema -struct<20 days 15 hours 40 minutes:interval> +struct -- !query 65 output 20 days 15 hours 40 minutes @@ -535,7 +535,7 @@ struct<20 days 15 hours 40 minutes:interval> -- !query 66 select interval '20 15:40:32.99899999' day to second -- !query 66 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 66 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -543,7 +543,7 @@ struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> -- !query 67 select interval '15:40:32.99899999' hour to minute -- !query 67 schema -struct<15 hours 40 minutes:interval> +struct -- !query 67 output 15 hours 40 minutes @@ -551,7 +551,7 @@ struct<15 hours 40 minutes:interval> -- !query 68 select interval '15:40.99899999' hour to second -- !query 68 schema -struct<15 minutes 40.998999 seconds:interval> +struct -- !query 68 output 15 minutes 40.998999 seconds @@ -559,7 +559,7 @@ struct<15 minutes 40.998999 seconds:interval> -- !query 69 select interval '15:40' hour to second -- !query 69 schema -struct<15 hours 40 minutes:interval> +struct -- !query 69 output 15 hours 40 minutes @@ -567,7 +567,7 @@ struct<15 hours 40 minutes:interval> -- !query 70 select interval '15:40:32.99899999' hour to second -- !query 70 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 70 output 15 hours 40 minutes 32.998999 seconds @@ -575,7 +575,7 @@ struct<15 hours 40 minutes 32.998999 seconds:interval> -- !query 71 select interval '20 40:32.99899999' minute to second -- !query 71 schema -struct<20 days 40 minutes 32.998999 seconds:interval> +struct -- !query 71 output 20 days 40 minutes 32.998999 seconds @@ -583,7 +583,7 @@ struct<20 days 40 minutes 32.998999 seconds:interval> -- !query 72 select interval '40:32.99899999' minute to second -- !query 72 schema -struct<40 minutes 32.998999 seconds:interval> +struct -- !query 72 output 40 minutes 32.998999 seconds @@ -591,7 +591,7 @@ struct<40 minutes 32.998999 seconds:interval> -- !query 73 select interval '40:32' minute to second -- !query 73 schema -struct<40 minutes 32 seconds:interval> +struct -- !query 73 output 40 minutes 32 seconds @@ -621,7 +621,7 @@ select interval 10 nanoseconds -- !query 76 select map(1, interval 1 day, 2, interval 3 week) -- !query 76 schema -struct> +struct> -- !query 76 output {1:1 days,2:21 days} @@ -629,7 +629,7 @@ struct> -- !query 77 select interval 'interval 3 year 1 hour' -- !query 77 schema -struct<3 years 1 hours:interval> +struct -- !query 77 output 3 years 1 hours @@ -637,7 +637,7 @@ struct<3 years 1 hours:interval> -- !query 78 select interval '3 year 1 hour' -- !query 78 schema -struct<3 years 1 hours:interval> +struct -- !query 78 output 3 years 1 hours @@ -1024,7 +1024,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1051,7 +1051,7 @@ select interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic -- !query 114 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> -- !query 114 output 5 years 5 months -1 years -1 months @@ -1067,7 +1067,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 @@ -1094,6 +1094,6 @@ select interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 117 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> -- !query 117 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index e43e88c8c72a8..6ee7148755c34 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -301,7 +301,7 @@ select date 'mar 11 2016' -- !query 32 select tImEstAmp '2016-03-11 20:54:00.000' -- !query 32 schema -struct +struct -- !query 32 output 2016-03-11 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index d981ed15e37f4..4eaeee6030d66 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -29,7 +29,7 @@ struct<1 month -1 hour:interval> -- !query 3 SELECT interval '999' second -- !query 3 schema -struct<16 minutes 39 seconds:interval> +struct -- !query 3 output 16 minutes 39 seconds @@ -37,7 +37,7 @@ struct<16 minutes 39 seconds:interval> -- !query 4 SELECT interval '999' minute -- !query 4 schema -struct<16 hours 39 minutes:interval> +struct -- !query 4 output 16 hours 39 minutes @@ -45,7 +45,7 @@ struct<16 hours 39 minutes:interval> -- !query 5 SELECT interval '999' hour -- !query 5 schema -struct<999 hours:interval> +struct -- !query 5 output 999 hours @@ -53,7 +53,7 @@ struct<999 hours:interval> -- !query 6 SELECT interval '999' day -- !query 6 schema -struct<999 days:interval> +struct -- !query 6 output 999 days @@ -61,7 +61,7 @@ struct<999 days:interval> -- !query 7 SELECT interval '999' month -- !query 7 schema -struct<83 years 3 months:interval> +struct -- !query 7 output 83 years 3 months @@ -69,7 +69,7 @@ struct<83 years 3 months:interval> -- !query 8 SELECT interval '1' year -- !query 8 schema -struct<1 years:interval> +struct -- !query 8 output 1 years @@ -77,7 +77,7 @@ struct<1 years:interval> -- !query 9 SELECT interval '2' month -- !query 9 schema -struct<2 months:interval> +struct -- !query 9 output 2 months @@ -85,7 +85,7 @@ struct<2 months:interval> -- !query 10 SELECT interval '3' day -- !query 10 schema -struct<3 days:interval> +struct -- !query 10 output 3 days @@ -93,7 +93,7 @@ struct<3 days:interval> -- !query 11 SELECT interval '4' hour -- !query 11 schema -struct<4 hours:interval> +struct -- !query 11 output 4 hours @@ -101,7 +101,7 @@ struct<4 hours:interval> -- !query 12 SELECT interval '5' minute -- !query 12 schema -struct<5 minutes:interval> +struct -- !query 12 output 5 minutes @@ -109,7 +109,7 @@ struct<5 minutes:interval> -- !query 13 SELECT interval '6' second -- !query 13 schema -struct<6 seconds:interval> +struct -- !query 13 output 6 seconds @@ -117,7 +117,7 @@ struct<6 seconds:interval> -- !query 14 SELECT interval '1-2' year to month -- !query 14 schema -struct<1 years 2 months:interval> +struct -- !query 14 output 1 years 2 months @@ -125,7 +125,7 @@ struct<1 years 2 months:interval> -- !query 15 SELECT interval '1 2:03' day to hour -- !query 15 schema -struct<1 days 2 hours:interval> +struct -- !query 15 output 1 days 2 hours @@ -133,7 +133,7 @@ struct<1 days 2 hours:interval> -- !query 16 SELECT interval '1 2:03:04' day to hour -- !query 16 schema -struct<1 days 2 hours:interval> +struct -- !query 16 output 1 days 2 hours @@ -141,7 +141,7 @@ struct<1 days 2 hours:interval> -- !query 17 SELECT interval '1 2:03' day to minute -- !query 17 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 17 output 1 days 2 hours 3 minutes @@ -149,7 +149,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 18 SELECT interval '1 2:03:04' day to minute -- !query 18 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 18 output 1 days 2 hours 3 minutes @@ -157,7 +157,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 19 SELECT interval '1 2:03' day to second -- !query 19 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 19 output 1 days 2 hours 3 minutes @@ -165,7 +165,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 20 SELECT interval '1 2:03:04' day to second -- !query 20 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 20 output 1 days 2 hours 3 minutes 4 seconds @@ -173,7 +173,7 @@ struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 21 SELECT interval '1 2:03' hour to minute -- !query 21 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 21 output 1 days 2 hours 3 minutes @@ -181,7 +181,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 22 SELECT interval '1 2:03:04' hour to minute -- !query 22 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 22 output 1 days 2 hours 3 minutes @@ -189,7 +189,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 23 SELECT interval '1 2:03' hour to second -- !query 23 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 23 output 1 days 2 hours 3 minutes @@ -197,7 +197,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 24 SELECT interval '1 2:03:04' hour to second -- !query 24 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 24 output 1 days 2 hours 3 minutes 4 seconds @@ -205,7 +205,7 @@ struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 25 SELECT interval '1 2:03' minute to second -- !query 25 schema -struct<1 days 2 minutes 3 seconds:interval> +struct -- !query 25 output 1 days 2 minutes 3 seconds @@ -213,7 +213,7 @@ struct<1 days 2 minutes 3 seconds:interval> -- !query 26 SELECT interval '1 2:03:04' minute to second -- !query 26 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 26 output 1 days 2 hours 3 minutes 4 seconds @@ -232,7 +232,7 @@ SELECT interval '1 day -1 hours', interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 28 schema -struct<1 days -1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval> +struct -- !query 28 output +1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index fbb4dd3b5efd1..9a11ef13cdf1f 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -64,6 +64,8 @@ create temporary view t3 as select * from values struct<> -- !query 2 output + + -- !query 3 create temporary view s1 as select * from values (1), (3), (5), (7), (9) @@ -73,6 +75,7 @@ struct<> -- !query 3 output + -- !query 4 create temporary view s2 as select * from values (1), (3), (4), (6), (9) @@ -82,6 +85,7 @@ struct<> -- !query 4 output + -- !query 5 create temporary view s3 as select * from values (3), (4), (6), (9) @@ -91,6 +95,7 @@ struct<> -- !query 5 output + -- !query 6 SELECT t1a, t1b, t1c, t3a, t3b, t3c FROM t1 natural JOIN t3 @@ -307,7 +312,7 @@ GROUP BY t1b HAVING t1b > 8 -- !query 14 schema struct --- !query 11 output +-- !query 14 output 1 10 @@ -378,7 +383,6 @@ val1b 8 16 1 8 16 val1b 8 16 1 NULL 16 - -- !query 17 SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id @@ -548,6 +552,7 @@ NULL 4 NULL 6 NULL 9 + -- !query 30 SELECT s1.id, s2.id AS id2 FROM s1 FULL OUTER JOIN s2 @@ -573,12 +578,16 @@ DROP VIEW s1 struct<> -- !query 31 output + + -- !query 32 DROP VIEW s2 -- !query 32 schema struct<> -- !query 32 output + + -- !query 33 DROP VIEW s3 -- !query 33 schema diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out index c3d5fad0870bc..4e4f592a137fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out @@ -77,7 +77,7 @@ struct -- !query 9 SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') -- !query 9 schema -struct +struct -- !query 9 output 2016-11-15 20:54:00, 2016-11-12 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 462ad63aaaf2b..8608fa494d6cd 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + 2 days)' (tinyint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 -- !query 2 @@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + 2 days)' (smallint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 -- !query 3 @@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) + 2 days)' (int and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) + INTERVAL '2 days')' (int and interval).; line 1 pos 7 -- !query 4 @@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + 2 days)' (bigint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 -- !query 5 @@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + 2 days)' (float and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + INTERVAL '2 days')' (float and interval).; line 1 pos 7 -- !query 6 @@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + 2 days)' (double and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' (double and interval).; line 1 pos 7 -- !query 7 @@ -70,13 +70,13 @@ select cast(1 as decimal(10, 0)) + interval 2 day struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + 2 days)' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 -- !query 8 select cast('2017-12-11' as string) + interval 2 day -- !query 8 schema -struct +struct -- !query 8 output 2017-12-13 00:00:00 @@ -84,7 +84,7 @@ struct +struct -- !query 9 output 2017-12-13 09:30:00 @@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) + 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + 2 days)' (binary and interval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + INTERVAL '2 days')' (binary and interval).; line 1 pos 7 -- !query 11 @@ -104,13 +104,13 @@ select cast(1 as boolean) + interval 2 day struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + 2 days)' (boolean and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 -- !query 12 select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day -- !query 12 schema -struct +struct -- !query 12 output 2017-12-13 09:30:00 @@ -118,7 +118,7 @@ struct +struct -- !query 13 output 2017-12-13 @@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint) struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 -- !query 15 @@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint) struct<> -- !query 15 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 -- !query 16 @@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int) struct<> -- !query 16 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS INT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS INT))' (interval and int).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS INT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS INT))' (interval and int).; line 1 pos 7 -- !query 17 @@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint) struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 -- !query 18 @@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float) struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 -- !query 19 @@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double) struct<> -- !query 19 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(2 days + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 -- !query 20 @@ -183,13 +183,13 @@ select interval 2 day + cast(1 as decimal(10, 0)) struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(2 days + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 -- !query 21 select interval 2 day + cast('2017-12-11' as string) -- !query 21 schema -struct +struct -- !query 21 output 2017-12-13 00:00:00 @@ -197,7 +197,7 @@ struct +struct -- !query 22 output 2017-12-13 09:30:00 @@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary) struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(2 days + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 -- !query 24 @@ -217,13 +217,13 @@ select interval 2 day + cast(1 as boolean) struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(2 days + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 -- !query 25 select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp) -- !query 25 schema -struct +struct -- !query 25 output 2017-12-13 09:30:00 @@ -231,7 +231,7 @@ struct +struct -- !query 26 output 2017-12-13 @@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - 2 days)' (tinyint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 -- !query 28 @@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day struct<> -- !query 28 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - 2 days)' (smallint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 -- !query 29 @@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day struct<> -- !query 29 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) - 2 days)' (int and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) - INTERVAL '2 days')' (int and interval).; line 1 pos 7 -- !query 30 @@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day struct<> -- !query 30 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - 2 days)' (bigint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 -- !query 31 @@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day struct<> -- !query 31 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - 2 days)' (float and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - INTERVAL '2 days')' (float and interval).; line 1 pos 7 -- !query 32 @@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - 2 days)' (double and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' (double and interval).; line 1 pos 7 -- !query 33 @@ -296,13 +296,13 @@ select cast(1 as decimal(10, 0)) - interval 2 day struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - 2 days)' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 -- !query 34 select cast('2017-12-11' as string) - interval 2 day -- !query 34 schema -struct +struct -- !query 34 output 2017-12-09 00:00:00 @@ -310,7 +310,7 @@ struct +struct -- !query 35 output 2017-12-09 09:30:00 @@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) - 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - 2 days)' (binary and interval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - INTERVAL '2 days')' (binary and interval).; line 1 pos 7 -- !query 37 @@ -330,13 +330,13 @@ select cast(1 as boolean) - interval 2 day struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - 2 days)' (boolean and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 -- !query 38 select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day -- !query 38 schema -struct +struct -- !query 38 output 2017-12-09 09:30:00 @@ -344,6 +344,6 @@ struct +struct -- !query 39 output 2017-12-09 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index b5ce121d2bc6a..7fed9e147608b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -154,7 +154,7 @@ SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY udf(cate), val_timestamp -- !query 9 schema -struct +struct -- !query 9 output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 1698ac081be95..f3b9c252b8330 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -154,7 +154,7 @@ SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY v RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY cate, val_timestamp -- !query 9 schema -struct +struct -- !query 9 output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 From ffc97530371433bc0221e06d8c1d11af8d92bd94 Mon Sep 17 00:00:00 2001 From: "wangguangxin.cn" Date: Tue, 19 Nov 2019 16:10:22 +0800 Subject: [PATCH 41/83] [SPARK-29918][SQL] RecordBinaryComparator should check endianness when compared by long MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR try to make sure the comparison results of `compared by 8 bytes at a time` and `compared by bytes wise` in RecordBinaryComparator is *consistent*, by reverse long bytes if it is little-endian and using Long.compareUnsigned. ### Why are the changes needed? If the architecture supports unaligned or the offset is 8 bytes aligned, `RecordBinaryComparator` compare 8 bytes at a time by reading 8 bytes as a long. Related code is ``` if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) { while (i <= leftLen - 8) { final long v1 = Platform.getLong(leftObj, leftOff + i); final long v2 = Platform.getLong(rightObj, rightOff + i); if (v1 != v2) { return v1 > v2 ? 1 : -1; } i += 8; } } ``` Otherwise, it will compare bytes by bytes.  Related code is ``` while (i < leftLen) { final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; if (v1 != v2) { return v1 > v2 ? 1 : -1; } i += 1; } ``` However, on little-endian machine,  the result of *compared by a long value* and *compared bytes by bytes* maybe different. For two same records, its offsets may vary in the first run and second run, which will lead to compare them using long comparison or byte-by-byte comparison, the result maybe different. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add new test cases in RecordBinaryComparatorSuite Closes #26548 from WangGuangxin/binary_comparator. Authored-by: wangguangxin.cn Signed-off-by: Wenchen Fan --- .../sql/execution/RecordBinaryComparator.java | 30 ++++++++---- .../sort/RecordBinaryComparatorSuite.java | 47 ++++++++++++++++++- 2 files changed, 67 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java b/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java index 40c2cc806e87a..1f243406c77e0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java @@ -20,8 +20,13 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.util.collection.unsafe.sort.RecordComparator; +import java.nio.ByteOrder; + public final class RecordBinaryComparator extends RecordComparator { + private static final boolean LITTLE_ENDIAN = + ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN); + @Override public int compare( Object leftObj, long leftOff, int leftLen, Object rightObj, long rightOff, int rightLen) { @@ -38,10 +43,10 @@ public int compare( // check if stars align and we can get both offsets to be aligned if ((leftOff % 8) == (rightOff % 8)) { while ((leftOff + i) % 8 != 0 && i < leftLen) { - final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; - final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; + final int v1 = Platform.getByte(leftObj, leftOff + i); + final int v2 = Platform.getByte(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + return (v1 & 0xff) > (v2 & 0xff) ? 1 : -1; } i += 1; } @@ -49,10 +54,17 @@ public int compare( // for architectures that support unaligned accesses, chew it up 8 bytes at a time if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) { while (i <= leftLen - 8) { - final long v1 = Platform.getLong(leftObj, leftOff + i); - final long v2 = Platform.getLong(rightObj, rightOff + i); + long v1 = Platform.getLong(leftObj, leftOff + i); + long v2 = Platform.getLong(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + if (LITTLE_ENDIAN) { + // if read as little-endian, we have to reverse bytes so that the long comparison result + // is equivalent to byte-by-byte comparison result. + // See discussion in https://github.com/apache/spark/pull/26548#issuecomment-554645859 + v1 = Long.reverseBytes(v1); + v2 = Long.reverseBytes(v2); + } + return Long.compareUnsigned(v1, v2); } i += 8; } @@ -60,10 +72,10 @@ public int compare( // this will finish off the unaligned comparisons, or do the entire aligned comparison // whichever is needed. while (i < leftLen) { - final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; - final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; + final int v1 = Platform.getByte(leftObj, leftOff + i); + final int v2 = Platform.getByte(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + return (v1 & 0xff) > (v2 & 0xff) ? 1 : -1; } i += 1; } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 92dabc79d2bff..68f984ae0c1e3 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -33,6 +33,7 @@ import org.apache.spark.util.collection.unsafe.sort.*; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -273,7 +274,7 @@ public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + assert(compare(0, 1) > 0); } @Test @@ -321,4 +322,48 @@ public void testBinaryComparatorWhenOnlyTheLastColumnDiffers() throws Exception assert(compare(0, 1) < 0); } + + @Test + public void testCompareLongsAsLittleEndian() { + long arrayOffset = 12; + + long[] arr1 = new long[2]; + Platform.putLong(arr1, arrayOffset, 0x0100000000000000L); + long[] arr2 = new long[2]; + Platform.putLong(arr2, arrayOffset + 4, 0x0000000000000001L); + // leftBaseOffset is not aligned while rightBaseOffset is aligned, + // it will start by comparing long + int result1 = binaryComparator.compare(arr1, arrayOffset, 8, arr2, arrayOffset + 4, 8); + + long[] arr3 = new long[2]; + Platform.putLong(arr3, arrayOffset, 0x0100000000000000L); + long[] arr4 = new long[2]; + Platform.putLong(arr4, arrayOffset, 0x0000000000000001L); + // both left and right offset is not aligned, it will start with byte-by-byte comparison + int result2 = binaryComparator.compare(arr3, arrayOffset, 8, arr4, arrayOffset, 8); + + Assert.assertEquals(result1, result2); + } + + @Test + public void testCompareLongsAsUnsigned() { + long arrayOffset = 12; + + long[] arr1 = new long[2]; + Platform.putLong(arr1, arrayOffset + 4, 0xa000000000000000L); + long[] arr2 = new long[2]; + Platform.putLong(arr2, arrayOffset + 4, 0x0000000000000000L); + // both leftBaseOffset and rightBaseOffset are aligned, so it will start by comparing long + int result1 = binaryComparator.compare(arr1, arrayOffset + 4, 8, arr2, arrayOffset + 4, 8); + + long[] arr3 = new long[2]; + Platform.putLong(arr3, arrayOffset, 0xa000000000000000L); + long[] arr4 = new long[2]; + Platform.putLong(arr4, arrayOffset, 0x0000000000000000L); + // both leftBaseOffset and rightBaseOffset are not aligned, + // so it will start with byte-by-byte comparison + int result2 = binaryComparator.compare(arr3, arrayOffset, 8, arr4, arrayOffset, 8); + + Assert.assertEquals(result1, result2); + } } From a8d98833b88e7a366734c18acec36236357a41e4 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 19 Nov 2019 19:18:08 +0800 Subject: [PATCH 42/83] [SPARK-29893] improve the local shuffle reader performance by changing the reading task number from 1 to multi ### What changes were proposed in this pull request? This PR update the local reader task number from 1 to multi `partitionStartIndices.length`. ### Why are the changes needed? Improve the performance of local shuffle reader. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UTs Closes #26516 from JkSelf/improveLocalShuffleReader. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 3 +- .../adaptive/AdaptiveSparkPlanExec.scala | 13 +- .../adaptive/LocalShuffledRowRDD.scala | 52 ++++++-- .../adaptive/OptimizeLocalShuffleReader.scala | 114 ++++++++++-------- .../exchange/ShuffleExchangeExec.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 61 ++++++++++ 6 files changed, 170 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 873efa76468ed..3c6c181f9428c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -701,8 +701,7 @@ private[spark] class MapOutputTrackerMaster( if (shuffleStatus != null) { shuffleStatus.withMapStatuses { statuses => if (mapId >= 0 && mapId < statuses.length) { - Seq( ExecutorCacheTaskLocation(statuses(mapId).location.host, - statuses(mapId).location.executorId).toString) + Seq(statuses(mapId).location.host) } else { Nil } 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 8d4731f34ddd6..02d8f185a7825 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 @@ -82,7 +82,6 @@ case class AdaptiveSparkPlanExec( // plan should reach a final status of query stages (i.e., no more addition or removal of // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( - OptimizeLocalShuffleReader(conf), ensureRequirements ) @@ -90,16 +89,10 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, subqueryCache), - - // When adding local shuffle readers in 'OptimizeLocalShuffleReader`, we revert all the local - // readers if additional shuffles are introduced. This may be too conservative: maybe there is - // only one local reader that introduces shuffle, and we can still keep other local readers. - // Here we re-execute this rule with the sub-plan-tree of a query stage, to make sure necessary - // local readers are added before executing the query stage. - // This rule must be executed before `ReduceNumShufflePartitions`, as local shuffle readers - // can't change number of partitions. - OptimizeLocalShuffleReader(conf), ReduceNumShufflePartitions(conf), + // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' + // in 'ReduceNumShufflePartitions' rule. So it must be after 'ReduceNumShufflePartitions' rule. + OptimizeLocalShuffleReader(conf), ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, session.sessionState.columnarRules), CollapseCodegenStages(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 5fccb5ce65783..44254593f8895 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -17,20 +17,24 @@ package org.apache.spark.sql.execution.adaptive +import scala.collection.mutable.ArrayBuffer + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} - /** - * The [[Partition]] used by [[LocalShuffledRowRDD]]. A pre-shuffle partition - * (identified by `preShufflePartitionIndex`) contains a range of post-shuffle partitions - * (`startPostShufflePartitionIndex` to `endPostShufflePartitionIndex - 1`, inclusive). + * The [[Partition]] used by [[LocalShuffledRowRDD]]. + * @param mapIndex the index of mapper. + * @param startPartition the start partition ID in mapIndex mapper. + * @param endPartition the end partition ID in mapIndex mapper. */ private final class LocalShuffledRowRDDPartition( - val preShufflePartitionIndex: Int) extends Partition { - override val index: Int = preShufflePartitionIndex + override val index: Int, + val mapIndex: Int, + val startPartition: Int, + val endPartition: Int) extends Partition { } /** @@ -52,7 +56,8 @@ private final class LocalShuffledRowRDDPartition( */ class LocalShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], - metrics: Map[String, SQLMetric]) + metrics: Map[String, SQLMetric], + advisoryParallelism : Option[Int] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { private[this] val numReducers = dependency.partitioner.numPartitions @@ -60,11 +65,33 @@ class LocalShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) + /** + * To equally divide n elements into m buckets, basically each bucket should have n/m elements, + * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns + * a sequence with length numBuckets and each value represents the start index of each bucket. + */ + private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { + val elementsPerBucket = numElements / numBuckets + val remaining = numElements % numBuckets + val splitPoint = (elementsPerBucket + 1) * remaining + (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ + (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) + } + override def getPartitions: Array[Partition] = { + val partitionStartIndices: Array[Int] = { + val expectedParallelism = advisoryParallelism.getOrElse(numReducers) + // TODO split by data size in the future. + equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray + } - Array.tabulate[Partition](numMappers) { i => - new LocalShuffledRowRDDPartition(i) + val partitions = ArrayBuffer[LocalShuffledRowRDDPartition]() + for (mapIndex <- 0 until numMappers) { + (partitionStartIndices :+ numReducers).sliding(2, 1).foreach { case Array(start, end) => + partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) + } } + partitions.toArray } override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -74,17 +101,16 @@ class LocalShuffledRowRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val localRowPartition = split.asInstanceOf[LocalShuffledRowRDDPartition] - val mapIndex = localRowPartition.index + val mapIndex = localRowPartition.mapIndex val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( dependency.shuffleHandle, mapIndex, - 0, - numReducers, + localRowPartition.startPartition, + localRowPartition.endPartition, context, sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 87d745bf976ab..a57cfae6d0991 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -27,49 +27,47 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExcha import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.internal.SQLConf -object BroadcastJoinWithShuffleLeft { - def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { - case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) => - Some((join.left.asInstanceOf[QueryStageExec], join.buildSide)) - case _ => None - } -} - -object BroadcastJoinWithShuffleRight { - def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { - case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) => - Some((join.right.asInstanceOf[QueryStageExec], join.buildSide)) - case _ => None - } -} - /** - * A rule to optimize the shuffle reader to local reader as far as possible - * when converting the 'SortMergeJoinExec' to 'BroadcastHashJoinExec' in runtime. - * - * This rule can be divided into two steps: - * Step1: Add the local reader in probe side and then check whether additional - * shuffle introduced. If introduced, we will revert all the local - * reader in probe side. - * Step2: Add the local reader in build side and will not check whether - * additional shuffle introduced. Because the build side will not introduce - * additional shuffle. + * A rule to optimize the shuffle reader to local reader iff no additional shuffles + * will be introduced: + * 1. if the input plan is a shuffle, add local reader directly as we can never introduce + * extra shuffles in this case. + * 2. otherwise, add local reader to the probe side of broadcast hash join and + * then run `EnsureRequirements` to check whether additional shuffle introduced. + * If introduced, we will revert all the local readers. */ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { + import OptimizeLocalShuffleReader._ - override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { - return plan - } - // Add local reader in probe side. - val withProbeSideLocalReader = plan.transformDown { + def withProbeSideLocalReader(plan: SparkPlan): SparkPlan = { + plan.transformDown { case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildRight) => - val localReader = LocalShuffleReaderExec(shuffleStage) + val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildLeft) => - val localReader = LocalShuffleReaderExec(shuffleStage) + val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } + } + + def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { + plan match { + case c: CoalescedShuffleReaderExec => + LocalShuffleReaderExec(c.child, Some(c.partitionStartIndices.length)) + case q: QueryStageExec => LocalShuffleReaderExec(q) + } + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + return plan + } + + val optimizedPlan = plan match { + case s: SparkPlan if canUseLocalShuffleReader(s) => + createLocalReader(s) + case s: SparkPlan => withProbeSideLocalReader(s) + } def numExchanges(plan: SparkPlan): Int = { plan.collect { @@ -77,26 +75,38 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { }.length } // Check whether additional shuffle introduced. If introduced, revert the local reader. - val numExchangeBefore = numExchanges(EnsureRequirements(conf).apply(plan)) - val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(withProbeSideLocalReader)) - val optimizedPlan = if (numExchangeAfter > numExchangeBefore) { - logDebug("OptimizeLocalShuffleReader rule is not applied in the probe side due" + + if (numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) > 0) { + logDebug("OptimizeLocalShuffleReader rule is not applied due" + " to additional shuffles will be introduced.") plan } else { - withProbeSideLocalReader + optimizedPlan } - // Add the local reader in build side and and do not need to check whether - // additional shuffle introduced. - optimizedPlan.transformDown { - case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildLeft) => - val localReader = LocalShuffleReaderExec(shuffleStage) - join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) - case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildRight) => - val localReader = LocalShuffleReaderExec(shuffleStage) - join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) + } +} + +object OptimizeLocalShuffleReader { + + object BroadcastJoinWithShuffleLeft { + def unapply(plan: SparkPlan): Option[(SparkPlan, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if canUseLocalShuffleReader(join.left) => + Some((join.left, join.buildSide)) + case _ => None + } + } + + object BroadcastJoinWithShuffleRight { + def unapply(plan: SparkPlan): Option[(SparkPlan, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if canUseLocalShuffleReader(join.right) => + Some((join.right, join.buildSide)) + case _ => None } } + + def canUseLocalShuffleReader(plan: SparkPlan): Boolean = { + ShuffleQueryStageExec.isShuffleQueryStageExec(plan) || + plan.isInstanceOf[CoalescedShuffleReaderExec] + } } /** @@ -107,7 +117,9 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the * shuffle exchange node during canonicalization. */ -case class LocalShuffleReaderExec(child: SparkPlan) extends UnaryExecNode { +case class LocalShuffleReaderExec( + child: SparkPlan, + advisoryParallelism: Option[Int] = None) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -124,9 +136,9 @@ case class LocalShuffleReaderExec(child: SparkPlan) extends UnaryExecNode { if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createLocalShuffleRDD() + stage.plan.createLocalShuffleRDD(advisoryParallelism) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createLocalShuffleRDD() + stage.plan.createLocalShuffleRDD(advisoryParallelism) } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 2f94c522712b1..baca974a6450b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -83,8 +83,9 @@ case class ShuffleExchangeExec( new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } - def createLocalShuffleRDD(): LocalShuffledRowRDD = { - new LocalShuffledRowRDD(shuffleDependency, readMetrics) + def createLocalShuffleRDD( + advisoryParallelism: Option[Int] = None): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics, advisoryParallelism) } /** 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 f6b6b23b29140..84bd99a2a8635 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 @@ -103,6 +103,67 @@ class AdaptiveQueryExecSuite } } + test("Reuse the parallelism of CoalescedShuffleReaderExec in LocalShuffleReaderExec") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val localReaders = collect(adaptivePlan) { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length == 2) + // The pre-shuffle partition size is [0, 0, 0, 72, 0] + // And the partitionStartIndices is [0, 3, 4] + assert(localReaders(0).advisoryParallelism.get == 3) + // The pre-shuffle partition size is [0, 72, 0, 72, 126] + // And the partitionStartIndices is [0, 1, 2, 3, 4] + assert(localReaders(1).advisoryParallelism.get == 5) + + val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] + val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // the final parallelism is + // math.max(1, advisoryParallelism / numMappers): math.max(1, 3/2) = 1 + // and the partitions length is 1 * numMappers = 2 + assert(localShuffleRDD0.getPartitions.length == 2) + // the final parallelism is + // math.max(1, advisoryParallelism / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + test("Reuse the default parallelism in LocalShuffleReaderExec") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val localReaders = collect(adaptivePlan) { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length == 2) + val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] + val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + test("Scalar subquery") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", From 79ed4ae2dbdbc73c4225fbabeac5a1ddb10c65d1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 19 Nov 2019 21:01:26 +0800 Subject: [PATCH 43/83] [SPARK-29926][SQL] Fix weird interval string whose value is only a dangling decimal point ### What changes were proposed in this pull request? Currently, we support to parse '1. second' to 1s or even '. second' to 0s. ```sql -- !query 118 select interval '1. seconds' -- !query 118 schema struct<1 seconds:interval> -- !query 118 output 1 seconds -- !query 119 select interval '. seconds' -- !query 119 schema struct<0 seconds:interval> -- !query 119 output 0 seconds ``` ```sql postgres=# select interval '1. second'; ERROR: invalid input syntax for type interval: "1. second" LINE 1: select interval '1. second'; postgres=# select interval '. second'; ERROR: invalid input syntax for type interval: ". second" LINE 1: select interval '. second'; ``` We fix this by fixing the new interval parser's VALUE_FRACTIONAL_PART state With further digging, we found that 1. is valid in python, r, scala, and presto and so on... so this PR ONLY forbid the invalid interval value in the form of '. seconds'. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? yes, now we treat '. second' .... as invalid intervals ### How was this patch tested? add ut Closes #26573 from yaooqinn/SPARK-29926. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/util/IntervalUtils.scala | 10 +++++++--- .../spark/sql/catalyst/util/IntervalUtilsSuite.scala | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 61bd9242f3e24..9418d8eec3376 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -479,7 +479,9 @@ object IntervalUtils { var days: Int = 0 var microseconds: Long = 0 var fractionScale: Int = 0 + val initialFractionScale = (NANOS_PER_SECOND / 10).toInt var fraction: Int = 0 + var pointPrefixed: Boolean = false def trimToNextState(b: Byte, next: ParseState): Unit = { b match { @@ -519,6 +521,7 @@ object IntervalUtils { // We preset the scale to an invalid value to track fraction presence in the UNIT_BEGIN // state. If we meet '.', the scale become valid for the VALUE_FRACTIONAL_PART state. fractionScale = -1 + pointPrefixed = false b match { case '-' => isNegative = true @@ -530,7 +533,8 @@ object IntervalUtils { isNegative = false case '.' => isNegative = false - fractionScale = (NANOS_PER_SECOND / 10).toInt + fractionScale = initialFractionScale + pointPrefixed = true i += 1 state = VALUE_FRACTIONAL_PART case _ => throwIAE( s"unrecognized number '$currentWord'") @@ -546,7 +550,7 @@ object IntervalUtils { } case ' ' => state = TRIM_BEFORE_UNIT case '.' => - fractionScale = (NANOS_PER_SECOND / 10).toInt + fractionScale = initialFractionScale state = VALUE_FRACTIONAL_PART case _ => throwIAE(s"invalid value '$currentWord'") } @@ -556,7 +560,7 @@ object IntervalUtils { case _ if '0' <= b && b <= '9' && fractionScale > 0 => fraction += (b - '0') * fractionScale fractionScale /= 10 - case ' ' => + case ' ' if !pointPrefixed || fractionScale < initialFractionScale => fraction /= NANOS_PER_MICROS.toInt state = TRIM_BEFORE_UNIT case _ if '0' <= b && b <= '9' => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 73a2adbaec1db..ee3db0391ed00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -115,7 +115,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString("2234567890 days", "integer overflow") checkFromInvalidString("\n", "Error parsing '\n' to interval") checkFromInvalidString("\t", "Error parsing '\t' to interval") - + checkFromInvalidString(". seconds", "invalid value '.'") } test("string to interval: seconds with fractional part") { From 6fb8b8606544f26dc2d9719a2d009eb5aea65ba2 Mon Sep 17 00:00:00 2001 From: Jobit Mathew Date: Tue, 19 Nov 2019 21:30:38 +0800 Subject: [PATCH 44/83] [SPARK-29913][SQL] Improve Exception in postgreCastToBoolean ### What changes were proposed in this pull request? Exception improvement. ### Why are the changes needed? After selecting pgSQL dialect, queries which are failing because of wrong syntax will give long exception stack trace. For example, `explain select cast ("abc" as boolean);` Current output: > ERROR SparkSQLDriver: Failed in [explain select cast ("abc" as boolean)] > java.lang.IllegalArgumentException: invalid input syntax for type boolean: abc > at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$2(PostgreCastToBoolean.scala:51) > at org.apache.spark.sql.catalyst.expressions.CastBase.buildCast(Cast.scala:277) > at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$1(PostgreCastToBoolean.scala:44) > at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:773) > at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:460) > at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52) > at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:286) > at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72) > at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:286) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:291) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327) > at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291) > at org.apache.spark.sql.catalyst.plans.QueryPlan. > . > . > . ### Does this PR introduce any user-facing change? Yes. After this PR, output for above query will be: > == Physical Plan == > org.apache.spark.sql.AnalysisException: invalid input syntax for type boolean: abc; > > Time taken: 0.044 seconds, Fetched 1 row(s) > 19/11/15 15:38:57 INFO SparkSQLCLIDriver: Time taken: 0.044 seconds, Fetched 1 row(s) ### How was this patch tested? Updated existing test cases. Closes #26546 from jobitmathew/pgsqlexception. Authored-by: Jobit Mathew Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f9394473d06e0..258f9cea05b82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -141,6 +141,7 @@ class QueryExecution( ExplainUtils.processPlan(executedPlan, concat.append) } catch { case e: AnalysisException => concat.append(e.toString) + case e: IllegalArgumentException => concat.append(e.toString) } } else { QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) From 3d2a6f464f98c04f011509733a15972fd2b6701d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2019 10:39:38 -0800 Subject: [PATCH 45/83] [SPARK-29906][SQL] AQE should not introduce extra shuffle for outermost limit ### What changes were proposed in this pull request? `AdaptiveSparkPlanExec` should forward `executeCollect` and `executeTake` to the underlying physical plan. ### Why are the changes needed? some physical plan has optimization in `executeCollect` and `executeTake`. For example, `CollectLimitExec` won't do shuffle for outermost limit. ### Does this PR introduce any user-facing change? no ### How was this patch tested? a new test This closes #26560 Closes #26576 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 23 ++++++++++++------- .../adaptive/AdaptiveQueryExecSuite.scala | 21 +++++++++++++++++ 2 files changed, 36 insertions(+), 8 deletions(-) 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 02d8f185a7825..b4eea620b93a6 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 @@ -126,10 +126,8 @@ case class AdaptiveSparkPlanExec( override def doCanonicalize(): SparkPlan = initialPlan.canonicalized - override def doExecute(): RDD[InternalRow] = lock.synchronized { - if (isFinalPlan) { - currentPhysicalPlan.execute() - } else { + private def getFinalPhysicalPlan(): SparkPlan = lock.synchronized { + if (!isFinalPlan) { // Make sure we only update Spark UI if this plan's `QueryExecution` object matches the one // retrieved by the `sparkContext`'s current execution ID. Note that sub-queries do not have // their own execution IDs and therefore rely on the main query to update UI. @@ -210,12 +208,21 @@ case class AdaptiveSparkPlanExec( // Run the final plan when there's no more unfinished stages. currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) isFinalPlan = true - - val ret = currentPhysicalPlan.execute() logDebug(s"Final plan: $currentPhysicalPlan") - executionId.foreach(onUpdatePlan) - ret } + currentPhysicalPlan + } + + override def executeCollect(): Array[InternalRow] = { + getFinalPhysicalPlan().executeCollect() + } + + override def executeTake(n: Int): Array[InternalRow] = { + getFinalPhysicalPlan().executeTake(n) + } + + override def doExecute(): RDD[InternalRow] = { + getFinalPhysicalPlan().execute() } override def verboseString(maxFields: Int): String = simpleString(maxFields) 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 84bd99a2a8635..d5be45b0cdda0 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.exchange.Exchange @@ -534,4 +535,24 @@ class AdaptiveQueryExecSuite } } } + + test("SPARK-29906: AQE should not introduce extra shuffle for outermost limit") { + var numStages = 0 + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + numStages = jobStart.stageInfos.length + } + } + try { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + spark.sparkContext.addSparkListener(listener) + spark.range(0, 100, 1, numPartitions = 10).take(1) + spark.sparkContext.listenerBus.waitUntilEmpty() + // Should be only one stage since there is no shuffle. + assert(numStages == 1) + } + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } } From e804ed5e330c7dc6cd46812b520dd6b610a584b6 Mon Sep 17 00:00:00 2001 From: John Bauer Date: Tue, 19 Nov 2019 14:15:00 -0800 Subject: [PATCH 46/83] [SPARK-29691][ML][PYTHON] ensure Param objects are valid in fit, transform modify Param._copyValues to check valid Param objects supplied as extra ### What changes were proposed in this pull request? Estimator.fit() and Model.transform() accept a dictionary of extra parameters whose values are used to overwrite those supplied at initialization or by default. Additionally, the ParamGridBuilder.addGrid accepts a parameter and list of values. The keys are presumed to be valid Param objects. This change adds a check that only Param objects are supplied as keys. ### Why are the changes needed? Param objects are created by and bound to an instance of Params (Estimator, Model, or Transformer). They may be obtained from their parent as attributes, or by name through getParam. The documentation does not state that keys must be valid Param objects, nor describe how one may be obtained. The current behavior is to silently ignore keys which are not valid Param objects. ### Does this PR introduce any user-facing change? If the user does not pass in a Param object as required for keys in `extra` for Estimator.fit() and Model.transform(), and `param` for ParamGridBuilder.addGrid, an error will be raised indicating it is an invalid object. ### How was this patch tested? Added method test_copy_param_extras_check to test_param.py. Tested with Python 3.7 Closes #26527 from JohnHBauer/paramExtra. Authored-by: John Bauer Signed-off-by: Bryan Cutler --- python/pyspark/ml/param/__init__.py | 12 ++++++++++-- python/pyspark/ml/tests/test_param.py | 4 ++++ python/pyspark/ml/tests/test_tuning.py | 9 +++++++++ python/pyspark/ml/tuning.py | 8 +++++++- 4 files changed, 30 insertions(+), 3 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 57ad1e6dfb3e6..fe61f9f0fffd6 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -484,8 +484,16 @@ def _copyValues(self, to, extra=None): :return: the target instance with param values copied """ paramMap = self._paramMap.copy() - if extra is not None: - paramMap.update(extra) + if isinstance(extra, dict): + for param, value in extra.items(): + if isinstance(param, Param): + paramMap[param] = value + else: + raise TypeError("Expecting a valid instance of Param, but received: {}" + .format(param)) + elif extra is not None: + raise TypeError("Expecting a dict, but received an object of type {}." + .format(type(extra))) for param in self.params: # copy default params if param in self._defaultParamMap and to.hasParam(param.name): diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 75cd903b5d6d7..777b4930ce8c9 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -307,6 +307,10 @@ def test_copy_param_extras(self): copied_no_extra[k] = v self.assertEqual(tp._paramMap, copied_no_extra) self.assertEqual(tp._defaultParamMap, tp_copy._defaultParamMap) + with self.assertRaises(TypeError): + tp.copy(extra={"unknown_parameter": None}) + with self.assertRaises(TypeError): + tp.copy(extra=["must be a dict"]) def test_logistic_regression_check_thresholds(self): self.assertIsInstance( diff --git a/python/pyspark/ml/tests/test_tuning.py b/python/pyspark/ml/tests/test_tuning.py index 176e99d052d30..9d8ba37c60da4 100644 --- a/python/pyspark/ml/tests/test_tuning.py +++ b/python/pyspark/ml/tests/test_tuning.py @@ -63,6 +63,15 @@ def _fit(self, dataset): return model +class ParamGridBuilderTests(SparkSessionTestCase): + + def test_addGrid(self): + with self.assertRaises(TypeError): + grid = (ParamGridBuilder() + .addGrid("must be an instance of Param", ["not", "string"]) + .build()) + + class CrossValidatorTests(SparkSessionTestCase): def test_copy(self): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 16c376296c20d..5eb8ae44d3d66 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -88,8 +88,14 @@ def __init__(self): def addGrid(self, param, values): """ Sets the given parameters in this grid to fixed values. + + param must be an instance of Param associated with an instance of Params + (such as Estimator or Transformer). """ - self._param_grid[param] = values + if isinstance(param, Param): + self._param_grid[param] = values + else: + raise TypeError("param must be an instance of Param") return self From e753aa30e659706c3fa3414bf38566a79e0af8d6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 19 Nov 2019 15:56:50 -0800 Subject: [PATCH 47/83] [SPARK-29964][BUILD] lintr github workflows failed due to buggy GnuPG ### What changes were proposed in this pull request? Linter (R) github workflows failed sometimes like: https://github.com/apache/spark/pull/26509/checks?check_run_id=310718016 Failed message: ``` Executing: /tmp/apt-key-gpghome.8r74rQNEjj/gpg.1.sh --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 gpg: connecting dirmngr at '/tmp/apt-key-gpghome.8r74rQNEjj/S.dirmngr' failed: IPC connect call failed gpg: keyserver receive failed: No dirmngr ##[error]Process completed with exit code 2. ``` It is due to a buggy GnuPG. Context: https://github.com/sbt/website/pull/825 https://github.com/sbt/sbt/issues/4261 https://github.com/microsoft/WSL/issues/3286 ### Why are the changes needed? Make lint-r github workflows work. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass github workflows. Closes #26602 from viirya/SPARK-29964. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d2b7dca3684f0..2cd48153b190c 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -84,7 +84,7 @@ jobs: - name: install R run: | echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list - sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add sudo apt-get update sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: install R packages From 40b8a08b8b7f71685c5703605fd8a8b101de32f7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 20 Nov 2019 10:34:25 +0900 Subject: [PATCH 48/83] [SPARK-29963][SQL][TESTS] Check formatting timestamps up to microsecond precision by JSON/CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to add tests from the commit https://github.com/apache/spark/commit/47cb1f359af62383e24198dbbaa0b4503348cd04 for Spark 2.4 that check formatting of timestamp strings for various seconds fractions. ### Why are the changes needed? To make sure that current behavior is the same as in Spark 2.4 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`. Closes #26601 from MaxGekk/format-timestamp-micros-tests. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/util/TimestampFormatterSuite.scala | 40 +++++++++++++++++++ .../apache/spark/sql/JsonFunctionsSuite.scala | 7 ++++ .../execution/datasources/csv/CSVSuite.scala | 15 +++++++ 3 files changed, 62 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 79e433e88437b..082849c88669a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -193,4 +193,44 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers } } } + + test("formatting timestamp strings up to microsecond precision") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + def check(pattern: String, input: String, expected: String): Unit = { + val formatter = TimestampFormatter(pattern, zoneId) + val timestamp = DateTimeUtils.stringToTimestamp( + UTF8String.fromString(input), zoneId).get + val actual = formatter.format(timestamp) + assert(actual === expected) + } + + check( + "yyyy-MM-dd HH:mm:ss.SSSSSSS", "2019-10-14T09:39:07.123456", + "2019-10-14 09:39:07.1234560") + check( + "yyyy-MM-dd HH:mm:ss.SSSSSS", "1960-01-01T09:39:07.123456", + "1960-01-01 09:39:07.123456") + check( + "yyyy-MM-dd HH:mm:ss.SSSSS", "0001-10-14T09:39:07.1", + "0001-10-14 09:39:07.10000") + check( + "yyyy-MM-dd HH:mm:ss.SSSS", "9999-12-31T23:59:59.999", + "9999-12-31 23:59:59.9990") + check( + "yyyy-MM-dd HH:mm:ss.SSS", "1970-01-01T00:00:00.0101", + "1970-01-01 00:00:00.010") + check( + "yyyy-MM-dd HH:mm:ss.SS", "2019-10-14T09:39:07.09", + "2019-10-14 09:39:07.09") + check( + "yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07.2", + "2019-10-14 09:39:07.2") + check( + "yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07", + "2019-10-14 09:39:07.0") + check( + "yyyy-MM-dd HH:mm:ss", "2019-10-14T09:39:07.123456", + "2019-10-14 09:39:07") + } + } } 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 af80fa3d9327a..f3483151d26da 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 @@ -637,4 +637,11 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { df.select(from_json($"value", schema, options)), Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456")))) } + + test("to_json - timestamp in micros") { + val s = "2019-11-18 11:56:00.123456" + val df = Seq(java.sql.Timestamp.valueOf(s)).toDF("t").select( + to_json(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS"))) + checkAnswer(df, Row(s"""{"t":"$s"}""")) + } } 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 dac305944dd0d..a6c3a51858aed 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 @@ -2180,4 +2180,19 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { checkAnswer(readback, Row(Timestamp.valueOf(t))) } } + + test("Roundtrip in reading and writing timestamps in microsecond precision") { + withTempPath { path => + val timestamp = Timestamp.valueOf("2019-11-18 11:56:00.123456") + Seq(timestamp).toDF("t") + .write + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + val readback = spark.read + .schema("t timestamp") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + checkAnswer(readback, Row(timestamp)) + } + } } From 9e58b10c8e1e066e3daae06f57e48f369a24e9ec Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 20 Nov 2019 11:08:04 +0900 Subject: [PATCH 49/83] [SPARK-29945][SQL] do not handle negative sign specially in the parser ### What changes were proposed in this pull request? Remove the special handling of the negative sign in the parser (interval literal and type constructor) ### Why are the changes needed? The negative sign is an operator (UnaryMinus). We don't need to handle it specially, which is kind of doing constant folding at parser side. ### Does this PR introduce any user-facing change? The error message becomes a little different. Now it reports type mismatch for the `-` operator. ### How was this patch tested? existing tests Closes #26578 from cloud-fan/interval. Authored-by: Wenchen Fan Signed-off-by: Takeshi Yamamuro --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 31 +++------ .../parser/ExpressionParserSuite.scala | 5 +- .../resources/sql-tests/inputs/literals.sql | 5 +- .../sql-tests/results/ansi/interval.sql.out | 12 ++-- .../sql-tests/results/ansi/literals.sql.out | 65 ++++++++++--------- .../results/interval-display-iso_8601.sql.out | 2 +- .../interval-display-sql_standard.sql.out | 2 +- .../results/interval-display.sql.out | 2 +- .../sql-tests/results/interval.sql.out | 12 ++-- .../sql-tests/results/literals.sql.out | 65 ++++++++++--------- .../results/postgreSQL/interval.sql.out | 2 +- 12 files changed, 98 insertions(+), 109 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 41f8e3552977d..4f81dc47394a5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -764,7 +764,7 @@ primaryExpression constant : NULL #nullLiteral | interval #intervalLiteral - | negativeSign=MINUS? identifier STRING #typeConstructor + | identifier STRING #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral | STRING+ #stringLiteral @@ -787,7 +787,7 @@ booleanValue ; interval - : negativeSign=MINUS? INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? + : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5b9c204fcd4ae..e0476879f0e80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1850,7 +1850,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) - val isNegative = ctx.negativeSign != null def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { @@ -1859,9 +1858,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } try { valueType match { - case "DATE" if !isNegative => + case "DATE" => toLiteral(stringToDate(_, getZoneId(SQLConf.get.sessionLocalTimeZone)), DateType) - case "TIMESTAMP" if !isNegative => + case "TIMESTAMP" => val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => @@ -1873,9 +1872,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ex.setStackTrace(e.getStackTrace) throw ex } - val signedInterval = if (isNegative) IntervalUtils.negate(interval) else interval - Literal(signedInterval, CalendarIntervalType) - case "X" if !isNegative => + Literal(interval, CalendarIntervalType) + case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) case "INTEGER" => @@ -1887,10 +1885,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ex.setStackTrace(e.getStackTrace) throw ex } - Literal(if (isNegative) -i else i, IntegerType) + Literal(i, IntegerType) case other => - val negativeSign: String = if (isNegative) "-" else "" - throw new ParseException(s"Literals of type '$negativeSign$other' are currently not" + + throw new ParseException(s"Literals of type '$other' are currently not" + " supported.", ctx) } } catch { @@ -2020,14 +2017,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } - private def applyNegativeSign(sign: Token, interval: CalendarInterval): CalendarInterval = { - if (sign != null) { - IntervalUtils.negate(interval) - } else { - interval - } - } - /** * Create a [[CalendarInterval]] literal expression. Two syntaxes are supported: * - multiple unit value pairs, for instance: interval 2 months 2 days. @@ -2041,10 +2030,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", innerCtx.unitToUnitInterval) } - val interval = applyNegativeSign( - ctx.negativeSign, - visitMultiUnitsInterval(innerCtx.multiUnitsInterval)) - Literal(interval, CalendarIntervalType) + Literal(visitMultiUnitsInterval(innerCtx.multiUnitsInterval), CalendarIntervalType) } else if (ctx.errorCapturingUnitToUnitInterval != null) { val innerCtx = ctx.errorCapturingUnitToUnitInterval if (innerCtx.error1 != null || innerCtx.error2 != null) { @@ -2053,8 +2039,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", errorCtx) } - val interval = applyNegativeSign(ctx.negativeSign, visitUnitToUnitInterval(innerCtx.body)) - Literal(interval, CalendarIntervalType) + Literal(visitUnitToUnitInterval(innerCtx.body), CalendarIntervalType) } else { throw new ParseException("at least one time unit should be given for interval literal", ctx) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index a7dbd4e4e3459..371b702722a69 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -442,7 +442,7 @@ class ExpressionParserSuite extends AnalysisTest { intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") assertEqual( "-interval '3 month 1 hour'", - Literal(IntervalUtils.stringToInterval("interval -3 month -1 hour"))) + UnaryMinus(Literal(IntervalUtils.stringToInterval("interval 3 month 1 hour")))) // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) @@ -611,8 +611,7 @@ class ExpressionParserSuite extends AnalysisTest { def checkIntervals(intervalValue: String, expected: Literal): Unit = { Seq( "" -> expected, - "-" -> expected.copy( - value = IntervalUtils.negate(expected.value.asInstanceOf[CalendarInterval])) + "-" -> UnaryMinus(expected) ).foreach { case (sign, expectedLiteral) => assertEqual(s"${sign}interval $intervalValue", expectedLiteral) diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index d1dff7bc94686..61b02d86bb51b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -107,7 +107,10 @@ select integer '2147483648'; -- awareness of the negative/positive sign before type select -integer '7'; +select +integer '7'; +select +date '1999-01-01'; +select +timestamp '1999-01-01'; +-- can't negate date/timestamp/binary select -date '1999-01-01'; select -timestamp '1999-01-01'; select -x'2379ACFe'; -select +integer '7'; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 11c70d4987eec..bceb6bd1d2ea9 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 30 output 1 months -1 days 1 seconds @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1056,7 +1056,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1099,7 +1099,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 6ee7148755c34..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query 0 @@ -419,56 +419,57 @@ select integer '2147483648' -- !query 43 select -integer '7' -- !query 43 schema -struct<-7:int> +struct<(- 7):int> -- !query 43 output -7 -- !query 44 -select -date '1999-01-01' +select +integer '7' -- !query 44 schema -struct<> +struct<7:int> -- !query 44 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-DATE' are currently not supported.(line 1, pos 7) - -== SQL == -select -date '1999-01-01' --------^^^ +7 -- !query 45 -select -timestamp '1999-01-01' +select +date '1999-01-01' -- !query 45 schema -struct<> +struct -- !query 45 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) - -== SQL == -select -timestamp '1999-01-01' --------^^^ +1999-01-01 -- !query 46 -select -x'2379ACFe' +select +timestamp '1999-01-01' -- !query 46 schema -struct<> +struct -- !query 46 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-X' are currently not supported.(line 1, pos 7) - -== SQL == -select -x'2379ACFe' --------^^^ +1999-01-01 00:00:00 -- !query 47 -select +integer '7' +select -date '1999-01-01' -- !query 47 schema -struct<7:int> +struct<> -- !query 47 output -7 +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 48 +select -timestamp '1999-01-01' +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 49 +select -x'2379ACFe' +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out index 0e053940df38f..2da098f17e168 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out index 2633522bb1d08..7ef2f9c51218e 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out index faf27bda1549f..a292fe3c83028 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 6d09f62a6dcec..bda5fc6d8b8e9 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 30 output 1 months -1 days 1 seconds @@ -1024,7 +1024,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1067,7 +1067,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 6ee7148755c34..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query 0 @@ -419,56 +419,57 @@ select integer '2147483648' -- !query 43 select -integer '7' -- !query 43 schema -struct<-7:int> +struct<(- 7):int> -- !query 43 output -7 -- !query 44 -select -date '1999-01-01' +select +integer '7' -- !query 44 schema -struct<> +struct<7:int> -- !query 44 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-DATE' are currently not supported.(line 1, pos 7) - -== SQL == -select -date '1999-01-01' --------^^^ +7 -- !query 45 -select -timestamp '1999-01-01' +select +date '1999-01-01' -- !query 45 schema -struct<> +struct -- !query 45 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) - -== SQL == -select -timestamp '1999-01-01' --------^^^ +1999-01-01 -- !query 46 -select -x'2379ACFe' +select +timestamp '1999-01-01' -- !query 46 schema -struct<> +struct -- !query 46 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-X' are currently not supported.(line 1, pos 7) - -== SQL == -select -x'2379ACFe' --------^^^ +1999-01-01 00:00:00 -- !query 47 -select +integer '7' +select -date '1999-01-01' -- !query 47 schema -struct<7:int> +struct<> -- !query 47 output -7 +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 48 +select -timestamp '1999-01-01' +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 49 +select -x'2379ACFe' +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 4eaeee6030d66..f23415a108ff0 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -232,7 +232,7 @@ SELECT interval '1 day -1 hours', interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 28 schema -struct +struct -- !query 28 output +1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 From 5a70af7a6c271e12a1f8b508dcede804e30a1b75 Mon Sep 17 00:00:00 2001 From: Nikita Konda Date: Tue, 19 Nov 2019 20:01:42 -0800 Subject: [PATCH 50/83] [SPARK-29029][SQL] Use AttributeMap in PhysicalOperation.collectProjectsAndFilters ### What changes were proposed in this pull request? This PR fixes the issue of substituting aliases while collecting filters in `PhysicalOperation.collectProjectsAndFilters`. When the `AttributeReference` in alias map differs from the `AttributeReference` in filter condition only in qualifier, it does not substitute alias and throws exception saying `key videoid#47L not found` in the following scenario. ``` [1] Project [userid#0] +- [2] Filter (isnotnull(videoid#47L) && NOT (videoid#47L = 30)) +- [3] Project [factorial(videoid#1) AS videoid#47L, userid#0] +- [4] Filter (isnotnull(avebitrate#2) && (avebitrate#2 < 10)) +- [5] Relation[userid#0,videoid#1,avebitrate#2] ``` ### Why are the changes needed? We need to use `AttributeMap` where the key is `AttributeReference`'s `ExprId` instead of `Map[Attribute, Expression]` while collecting and substituting aliases in `PhysicalOperation.collectProjectsAndFilters`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New unit tests were added in `TestPhysicalOperation` which reproduces the bug Closes #25761 from nikitagkonda/SPARK-29029-use-attributemap-for-aliasmap-in-physicaloperation. Authored-by: Nikita Konda Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/planning/patterns.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 51d2a73ea97b7..c2a12eda19137 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -56,7 +56,7 @@ object PhysicalOperation extends PredicateHelper { * }}} */ private def collectProjectsAndFilters(plan: LogicalPlan): - (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression]) = + (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, AttributeMap[Expression]) = plan match { case Project(fields, child) if fields.forall(_.deterministic) => val (_, filters, other, aliases) = collectProjectsAndFilters(child) @@ -72,14 +72,15 @@ object PhysicalOperation extends PredicateHelper { collectProjectsAndFilters(h.child) case other => - (None, Nil, other, Map.empty) + (None, Nil, other, AttributeMap(Seq())) } - private def collectAliases(fields: Seq[Expression]): Map[Attribute, Expression] = fields.collect { - case a @ Alias(child, _) => a.toAttribute -> child - }.toMap + private def collectAliases(fields: Seq[Expression]): AttributeMap[Expression] = + AttributeMap(fields.collect { + case a: Alias => (a.toAttribute, a.child) + }) - private def substitute(aliases: Map[Attribute, Expression])(expr: Expression): Expression = { + private def substitute(aliases: AttributeMap[Expression])(expr: Expression): Expression = { expr.transform { case a @ Alias(ref: AttributeReference, name) => aliases.get(ref) From 0032d85153e34b9ac69598b7dff530094ed0f640 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 20 Nov 2019 21:13:51 +0800 Subject: [PATCH 51/83] [SPARK-29968][SQL] Remove the Predicate code from SparkPlan ### What changes were proposed in this pull request? This is to refactor Predicate code; it mainly removed `newPredicate` from `SparkPlan`. Modifications are listed below; - Move `Predicate` from `o.a.s.sqlcatalyst.expressions.codegen.GeneratePredicate.scala` to `o.a.s.sqlcatalyst.expressions.predicates.scala` - To resolve the name conflict, rename `o.a.s.sqlcatalyst.expressions.codegen.Predicate` to `o.a.s.sqlcatalyst.expressions.BasePredicate` - Extend `CodeGeneratorWithInterpretedFallback ` for `BasePredicate` This comes from the cloud-fan suggestion: https://github.com/apache/spark/pull/26420#discussion_r348005497 ### Why are the changes needed? For better code/test coverage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26604 from maropu/RefactorPredicate. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../catalog/ExternalCatalogUtils.scala | 5 +- .../sql/catalyst/expressions/Projection.scala | 4 +- .../codegen/GeneratePredicate.scala | 22 ++------- .../sql/catalyst/expressions/predicates.scala | 47 +++++++++++++++++-- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../catalyst/expressions/PredicateSuite.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 25 +--------- .../execution/basicPhysicalOperators.scala | 3 +- .../columnar/InMemoryTableScanExec.scala | 2 +- .../PartitioningAwareFileIndex.scala | 2 +- .../joins/BroadcastNestedLoopJoinExec.scala | 4 +- .../joins/CartesianProductExec.scala | 5 +- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../StreamingSymmetricHashJoinExec.scala | 17 +++---- .../streaming/statefulOperators.scala | 10 ++-- .../sql/sources/SimpleTextRelation.scala | 4 +- 18 files changed, 78 insertions(+), 82 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 4cff162c116a4..ae3b75dc3334b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI -import java.util.Locale import org.apache.hadoop.fs.Path import org.apache.hadoop.util.Shell @@ -26,7 +25,7 @@ import org.apache.hadoop.util.Shell import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} object ExternalCatalogUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since catalyst doesn't @@ -148,7 +147,7 @@ object ExternalCatalogUtils { } val boundPredicate = - InterpretedPredicate.create(predicates.reduce(And).transform { + Predicate.createInterpreted(predicates.reduce(And).transform { case att: AttributeReference => val index = partitionSchema.indexWhere(_.name == att.name) BoundReference(index, partitionSchema(index).dataType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 300f075d32763..b4a85e3e50bec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -89,14 +89,14 @@ object MutableProjection } /** - * Returns an MutableProjection for given sequence of bound Expressions. + * Returns a MutableProjection for given sequence of bound Expressions. */ def create(exprs: Seq[Expression]): MutableProjection = { createObject(exprs) } /** - * Returns an MutableProjection for given sequence of Expressions, which will be bound to + * Returns a MutableProjection for given sequence of Expressions, which will be bound to * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): MutableProjection = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index e0fabad6d089a..6ba646d360d2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -20,31 +20,17 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -/** - * Interface for generated predicate - */ -abstract class Predicate { - def eval(r: InternalRow): Boolean - - /** - * Initializes internal states given the current partition index. - * This is used by nondeterministic expressions to set initial states. - * The default implementation does nothing. - */ - def initialize(partitionIndex: Int): Unit = {} -} - /** * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[InternalRow]]. */ -object GeneratePredicate extends CodeGenerator[Expression, Predicate] { +object GeneratePredicate extends CodeGenerator[Expression, BasePredicate] { protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) - protected def create(predicate: Expression): Predicate = { + protected def create(predicate: Expression): BasePredicate = { val ctx = newCodeGenContext() val eval = predicate.genCode(ctx) @@ -53,7 +39,7 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] { return new SpecificPredicate(references); } - class SpecificPredicate extends ${classOf[Predicate].getName} { + class SpecificPredicate extends ${classOf[BasePredicate].getName} { private final Object[] references; ${ctx.declareMutableStates()} @@ -79,6 +65,6 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] { logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val (clazz, _) = CodeGenerator.compile(code) - clazz.generate(ctx.references.toArray).asInstanceOf[Predicate] + clazz.generate(ctx.references.toArray).asInstanceOf[BasePredicate] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 4c0998412f729..bcd442ad3cc35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -21,8 +21,9 @@ import scala.collection.immutable.TreeSet import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral, GenerateSafeProjection, GenerateUnsafeProjection, Predicate => BasePredicate} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.TypeUtils @@ -30,11 +31,18 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -object InterpretedPredicate { - def create(expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate = - create(BindReferences.bindReference(expression, inputSchema)) +/** + * A base class for generated/interpreted predicate + */ +abstract class BasePredicate { + def eval(r: InternalRow): Boolean - def create(expression: Expression): InterpretedPredicate = new InterpretedPredicate(expression) + /** + * Initializes internal states given the current partition index. + * This is used by nondeterministic expressions to set initial states. + * The default implementation does nothing. + */ + def initialize(partitionIndex: Int): Unit = {} } case class InterpretedPredicate(expression: Expression) extends BasePredicate { @@ -56,6 +64,35 @@ trait Predicate extends Expression { override def dataType: DataType = BooleanType } +/** + * The factory object for `BasePredicate`. + */ +object Predicate extends CodeGeneratorWithInterpretedFallback[Expression, BasePredicate] { + + override protected def createCodeGeneratedObject(in: Expression): BasePredicate = { + GeneratePredicate.generate(in) + } + + override protected def createInterpretedObject(in: Expression): BasePredicate = { + InterpretedPredicate(in) + } + + def createInterpreted(e: Expression): InterpretedPredicate = InterpretedPredicate(e) + + /** + * Returns a BasePredicate for an Expression, which will be bound to `inputSchema`. + */ + def create(e: Expression, inputSchema: Seq[Attribute]): BasePredicate = { + createObject(bindReference(e, inputSchema)) + } + + /** + * Returns a BasePredicate for a given bound Expression. + */ + def create(e: Expression): BasePredicate = { + createObject(e) + } +} trait PredicateHelper { protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 473f846c9313b..9d0bd358aa24c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1506,7 +1506,7 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] { case Filter(condition, LocalRelation(output, data, isStreaming)) if !hasUnevaluableExpr(condition) => - val predicate = InterpretedPredicate.create(condition, output) + val predicate = Predicate.create(condition, output) predicate.initialize(0) LocalRelation(output, data.filter(row => predicate.eval(row)), isStreaming) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 52cdd988caa2e..67a41e7cc2767 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -510,7 +510,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Interpreted Predicate should initialize nondeterministic expressions") { - val interpreted = InterpretedPredicate.create(LessThan(Rand(7), Literal(1.0))) + val interpreted = Predicate.create(LessThan(Rand(7), Literal(1.0))) interpreted.initialize(0) assert(interpreted.eval(new UnsafeRow())) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index b0fe4b741479f..88f5673aa9a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -230,7 +230,7 @@ case class FileSourceScanExec( // call the file index for the files matching all filters except dynamic partition filters val predicate = dynamicPartitionFilters.reduce(And) val partitionColumns = relation.partitionSchema - val boundPredicate = newPredicate(predicate.transform { + val boundPredicate = Predicate.create(predicate.transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 125f76282e3df..738af995376e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -21,7 +21,6 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.InternalCompilerException @@ -33,7 +32,7 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => GenPredicate, _} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ @@ -471,28 +470,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ MutableProjection.create(expressions, inputSchema) } - private def genInterpretedPredicate( - expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate = { - val str = expression.toString - val logMessage = if (str.length > 256) { - str.substring(0, 256 - 3) + "..." - } else { - str - } - logWarning(s"Codegen disabled for this expression:\n $logMessage") - InterpretedPredicate.create(expression, inputSchema) - } - - protected def newPredicate( - expression: Expression, inputSchema: Seq[Attribute]): GenPredicate = { - try { - GeneratePredicate.generate(expression, inputSchema) - } catch { - case _ @ (_: InternalCompilerException | _: CompileException) if codeGenFallBack => - genInterpretedPredicate(expression, inputSchema) - } - } - protected def newOrdering( order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[InternalRow] = { GenerateOrdering.generate(order, inputSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 3ed42f359c0a4..e128d59dca6ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{LongType, StructType} @@ -227,7 +226,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") child.execute().mapPartitionsWithIndexInternal { (index, iter) => - val predicate = newPredicate(condition, child.output) + val predicate = Predicate.create(condition, child.output) predicate.initialize(0) iter.filter { row => val r = predicate.eval(row) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 8d13cfb93d270..f03c2586048bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -310,7 +310,7 @@ case class InMemoryTableScanExec( val buffers = relation.cacheBuilder.cachedColumnBuffers buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = newPredicate( + val partitionFilter = Predicate.create( partitionFilters.reduceOption(And).getOrElse(Literal(true)), schema) partitionFilter.initialize(index) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 3adec2f790730..21ddeb6491155 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -171,7 +171,7 @@ abstract class PartitioningAwareFileIndex( if (partitionPruningPredicates.nonEmpty) { val predicate = partitionPruningPredicates.reduce(expressions.And) - val boundPredicate = InterpretedPredicate.create(predicate.transform { + val boundPredicate = Predicate.createInterpreted(predicate.transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index f526a19876670..5517c0dcdb188 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.{BitSet, CompactBuffer} case class BroadcastNestedLoopJoinExec( @@ -84,7 +82,7 @@ case class BroadcastNestedLoopJoinExec( @transient private lazy val boundCondition = { if (condition.isDefined) { - newPredicate(condition.get, streamed.output ++ broadcast.output).eval _ + Predicate.create(condition.get, streamed.output ++ broadcast.output).eval _ } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 88d98530991c9..29645a736548c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark._ import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Predicate, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils, ExternalAppendOnlyUnsafeRowArray, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.CompletionIterator @@ -93,7 +92,7 @@ case class CartesianProductExec( pair.mapPartitionsWithIndexInternal { (index, iter) => val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema) val filtered = if (condition.isDefined) { - val boundCondition = newPredicate(condition.get, left.output ++ right.output) + val boundCondition = Predicate.create(condition.get, left.output ++ right.output) boundCondition.initialize(index) val joined = new JoinedRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index e8938cb22e890..137f0b87a2f3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -99,7 +99,7 @@ trait HashJoin { UnsafeProjection.create(streamedKeys) @transient private[this] lazy val boundCondition = if (condition.isDefined) { - newPredicate(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 26fb0e5ffb1af..cd3c596435a21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -168,7 +168,7 @@ case class SortMergeJoinExec( left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => val boundCondition: (InternalRow) => Boolean = { condition.map { cond => - newPredicate(cond, left.output ++ right.output).eval _ + Predicate.create(cond, left.output ++ right.output).eval _ }.getOrElse { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 6bb4dc1672900..f1bfe97610fed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, JoinedRow, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, JoinedRow, Literal, Predicate, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -233,8 +233,9 @@ case class StreamingSymmetricHashJoinExec( val joinedRow = new JoinedRow + val inputSchema = left.output ++ right.output val postJoinFilter = - newPredicate(condition.bothSides.getOrElse(Literal(true)), left.output ++ right.output).eval _ + Predicate.create(condition.bothSides.getOrElse(Literal(true)), inputSchema).eval _ val leftSideJoiner = new OneSideHashJoiner( LeftSide, left.output, leftKeys, leftInputIter, condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left) @@ -417,7 +418,7 @@ case class StreamingSymmetricHashJoinExec( // Filter the joined rows based on the given condition. val preJoinFilter = - newPredicate(preJoinFilterExpr.getOrElse(Literal(true)), inputAttributes).eval _ + Predicate.create(preJoinFilterExpr.getOrElse(Literal(true)), inputAttributes).eval _ private val joinStateManager = new SymmetricHashJoinStateManager( joinSide, inputAttributes, joinKeys, stateInfo, storeConf, hadoopConfBcast.value.value, @@ -428,16 +429,16 @@ case class StreamingSymmetricHashJoinExec( case Some(JoinStateKeyWatermarkPredicate(expr)) => // inputSchema can be empty as expr should only have BoundReferences and does not require // the schema to generated predicate. See [[StreamingSymmetricHashJoinHelper]]. - newPredicate(expr, Seq.empty).eval _ + Predicate.create(expr, Seq.empty).eval _ case _ => - newPredicate(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate + Predicate.create(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate } private[this] val stateValueWatermarkPredicateFunc = stateWatermarkPredicate match { case Some(JoinStateValueWatermarkPredicate(expr)) => - newPredicate(expr, inputAttributes).eval _ + Predicate.create(expr, inputAttributes).eval _ case _ => - newPredicate(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate + Predicate.create(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate } private[this] var updatedStateRowsCount = 0 @@ -457,7 +458,7 @@ case class StreamingSymmetricHashJoinExec( val nonLateRows = WatermarkSupport.watermarkExpression(watermarkAttribute, eventTimeWatermark) match { case Some(watermarkExpr) => - val predicate = newPredicate(watermarkExpr, inputAttributes) + val predicate = Predicate.create(watermarkExpr, inputAttributes) inputIter.filter { row => !predicate.eval(row) } case None => inputIter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index d689a6f3c9819..01b309c3cf345 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, Predicate} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ @@ -156,17 +156,17 @@ trait WatermarkSupport extends UnaryExecNode { } /** Predicate based on keys that matches data older than the watermark */ - lazy val watermarkPredicateForKeys: Option[Predicate] = watermarkExpression.flatMap { e => + lazy val watermarkPredicateForKeys: Option[BasePredicate] = watermarkExpression.flatMap { e => if (keyExpressions.exists(_.metadata.contains(EventTimeWatermark.delayKey))) { - Some(newPredicate(e, keyExpressions)) + Some(Predicate.create(e, keyExpressions)) } else { None } } /** Predicate based on the child output that matches data older than the watermark. */ - lazy val watermarkPredicateForData: Option[Predicate] = - watermarkExpression.map(newPredicate(_, child.output)) + lazy val watermarkPredicateForData: Option[BasePredicate] = + watermarkExpression.map(Predicate.create(_, child.output)) protected def removeKeysOlderThanWatermark(store: StateStore): Unit = { if (watermarkPredicateForKeys.nonEmpty) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 60a4638f610b3..d1b97b2852fbc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.sql.{sources, SparkSession} import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedProjection, JoinedRow, Literal, Predicate} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.{DataType, StructType} @@ -88,7 +88,7 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { val attribute = inputAttributes.find(_.name == column).get expressions.GreaterThan(attribute, literal) }.reduceOption(expressions.And).getOrElse(Literal(true)) - InterpretedPredicate.create(filterCondition, inputAttributes) + Predicate.create(filterCondition, inputAttributes) } // Uses a simple projection to simulate column pruning From 56a65b971d4f1b43c37a5664a3a4e2e4fcad1c14 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 20 Nov 2019 08:20:16 -0600 Subject: [PATCH 52/83] [SPARK-18409][ML] LSH approxNearestNeighbors should use approxQuantile instead of sort ### What changes were proposed in this pull request? ```LSHModel.approxNearestNeighbors``` sorts the full dataset on the hashDistance in order to find a threshold. This PR uses approxQuantile instead. ### Why are the changes needed? To improve performance. ### Does this PR introduce any user-facing change? Yes. Changed ```LSH``` to make it extend ```HasRelativeError``` ```LSH``` and ```LSHModel``` have new APIs ```setRelativeError/getRelativeError``` ### How was this patch tested? Existing tests. Also added a couple doc test in python to test newly added ```getRelativeError``` Closes #26415 from huaxingao/spark-18409. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../org/apache/spark/ml/feature/LSH.scala | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index b20852383a6ff..4885d03220e95 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -112,7 +112,9 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] numNearestNeighbors: Int, singleProbe: Boolean, distCol: String): Dataset[_] = { - require(numNearestNeighbors > 0, "The number of nearest neighbors cannot be less than 1") + val count = dataset.count() + require(numNearestNeighbors > 0 && numNearestNeighbors <= count, "The number of" + + " nearest neighbors cannot be less than 1 or greater than the number of elements in dataset") // Get Hash Value of the key val keyHash = hashFunction(key) val modelDataset: DataFrame = if (!dataset.columns.contains($(outputCol))) { @@ -137,14 +139,21 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] val hashDistUDF = udf((x: Seq[Vector]) => hashDistance(x, keyHash), DataTypes.DoubleType) val hashDistCol = hashDistUDF(col($(outputCol))) - // Compute threshold to get exact k elements. - // TODO: SPARK-18409: Use approxQuantile to get the threshold - val modelDatasetSortedByHash = modelDataset.sort(hashDistCol).limit(numNearestNeighbors) - val thresholdDataset = modelDatasetSortedByHash.select(max(hashDistCol)) - val hashThreshold = thresholdDataset.take(1).head.getDouble(0) - - // Filter the dataset where the hash value is less than the threshold. - modelDataset.filter(hashDistCol <= hashThreshold) + // Compute threshold to get around k elements. + // To guarantee to have enough neighbors in one pass, we need (p - err) * N >= M + // so we pick quantile p = M / N + err + // M: the number of nearest neighbors; N: the number of elements in dataset + val relativeError = 0.05 + val approxQuantile = numNearestNeighbors.toDouble / count + relativeError + val modelDatasetWithDist = modelDataset.withColumn(distCol, hashDistCol) + if (approxQuantile >= 1) { + modelDatasetWithDist + } else { + val hashThreshold = modelDatasetWithDist.stat + .approxQuantile(distCol, Array(approxQuantile), relativeError) + // Filter the dataset where the hash value is less than the threshold. + modelDatasetWithDist.filter(hashDistCol <= hashThreshold(0)) + } } // Get the top k nearest neighbor by their distance to the key @@ -169,11 +178,11 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * to show the distance between each row and the key. */ def approxNearestNeighbors( - dataset: Dataset[_], - key: Vector, - numNearestNeighbors: Int, - distCol: String): Dataset[_] = { - approxNearestNeighbors(dataset, key, numNearestNeighbors, true, distCol) + dataset: Dataset[_], + key: Vector, + numNearestNeighbors: Int, + distCol: String): Dataset[_] = { + approxNearestNeighbors(dataset, key, numNearestNeighbors, true, distCol) } /** From b5df40bd87656ee71210db1008ce8b89b5a777d1 Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 20 Nov 2019 23:20:33 +0800 Subject: [PATCH 53/83] [SPARK-29894][SQL][WEBUI] Add Codegen Stage Id to Spark plan graphs in Web UI SQL Tab ### What changes were proposed in this pull request? The Web UI SQL Tab provides information on the executed SQL using plan graphs and by reporting SQL execution plans. Both sources provide useful information. Physical execution plans report Codegen Stage Ids. This PR adds Codegen Stage Ids to the plan graphs. ### Why are the changes needed? It is useful to have Codegen Stage Id information also reported in plan graphs, this allows to more easily match physical plans and graphs with metrics when troubleshooting SQL execution. Example snippet to show the proposed change: ![](https://issues.apache.org/jira/secure/attachment/12985837/snippet__plan_graph_with_Codegen_Stage_Id_Annotated.png) Example of the current state: ![](https://issues.apache.org/jira/secure/attachment/12985838/snippet_plan_graph_before_patch.png) Physical plan: ![](https://issues.apache.org/jira/secure/attachment/12985932/Physical_plan_Annotated.png) ### Does this PR introduce any user-facing change? This PR adds Codegen Stage Id information to SQL plan graphs in the Web UI/SQL Tab. ### How was this patch tested? Added a test + manually tested Closes #26519 from LucaCanali/addCodegenStageIdtoWEBUIGraphs. Authored-by: Luca Canali Signed-off-by: Wenchen Fan --- docs/img/webui-sql-dag.png | Bin 343391 -> 74288 bytes docs/web-ui.md | 14 +++++++++----- .../spark/sql/execution/SparkPlanInfo.scala | 11 +++++++++-- .../sql/execution/ui/SparkPlanGraph.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 7 +++++++ 6 files changed, 27 insertions(+), 9 deletions(-) diff --git a/docs/img/webui-sql-dag.png b/docs/img/webui-sql-dag.png index 4ca21092e8b397fec9657f64c74b39bb3862b53a..1c83c176da325fae542248fe3fdb343b0ca1fa5a 100644 GIT binary patch literal 74288 zcmd?RS5%Wx*De~GAV?Ddr78-jG?6O3_m1=`O%MU;HH0G41VltoI!N!mC!jRxJxG8^ zuOXB`AR%PO|D20+#@XlU`}e-t7mSrRV`aUo&1XKd5pSTUPEE-|2><}7H8oU>005!| z0Du%uaf9$p!N%4HLPO+hq^<<08E4xiJdnOt)Kvrk>Qkt$UXc-=Z@$y8@C5*9e*OC+ z8uTi20s!#EnktGfgKQ6Qpv*f^|Lv5#)VFLSsYO9?AFb-EdRqXz_Uj+t78qVVYqTZd zi45X$gSsBPes#yxu~CCl`B`56JSR&kdHql-YKSA&)gE$^d0@tcxW#(QsarOTJ*j89 zDwTZjeszZ7Mp+W$fHxODuD(AE*)CtHI4PCi#wI`KP5ZT_;dQq+FYZ5Xm9 zbYoY0aM10f^lB`9_B@w$=Y#ZNi3jKKZ-180X1qIpQXh(jz5S94x(@z+k%|c!pQGC8 zMzA;eoITxp<5k4h*k$WNS*tsJu_`n**=rbbF3o^?3JNq~6);~^Y+b&-X^90}+9Ux+ zZTB2{!I-sE)YT9}6L!Yk^^%vPa7wfLzB3P`KIIUDWAwu&Klpype(h6o`ctk0=NLGp zY$vuh+@-|;e={jSYdkgG>-6Ge$9>?d8SOXAEqx8?NWsVh`TT2k ziV9|o1QjTl@koXykABLB{Sp-y-|DhPzmxs;W18w*pfS1ysM>nWI=GvR-l=8%kSKb6*}0Qj%Po6aYjkY; zs#u;p_~`ViH+HA&tZwm~lcPjgzta8ht}^Y62XvP#Dt?fo5OSS+O}Mc+Z^^z6atP3W zuB}a(EEqhQJq5oA;qLEuq@7BGPjM>DU0r2hl71vHfo&k&VR|i&%X3FR>CWU28(V8T zBIVB7d3P6}%kEpdwQ`!Y@L&>~ax+7n$w{0tN@gjWi2+qzYxNa)XzYi7E%mq*KfARd zwR0#BPFjL=d9%poOkLapfUlY5#Cp6%K;MYv9-*|0p$1NG!&4 z_s98pz;Q>sX%$S3mYt>acLOUw`Q3Q`!=n$DNh_^N_;W#)a8*!w@L9hhOGj&pC+?Y$ zS#6eFi1gVqmP`txROjEXsOMr6E{n>xrOPnOD=o6SGuY%ww(R6lddk!cU)}z^d@;be z3a`z-EKzp=wK2WM)fO-zF)c09sd$voQ7 zH$%XM-odGARCY%ljkWs)k+rubKhQQC?lWSfd$TKHW$VdAQmB58<01RwHAhjIt>x5G z52EeGcDYgQCi%KjyU=tVu$&6|sIAsEqc(mB9PFHV*w8Y@blz96+@xq{mecyS2pb6c zX~D51H-^rAH<1(MA$j)~cla?^_i6{LYi}siWYm7&`p`*#QxDoej4=9gK3rRd)(H?H zTAr79al%&Jy-w5MXeku#@i7;Qq-KURrifR8(+@*79?N12={pleFO^b19KsYJ`1tUr zdXADcs5y_OEhZYu5bihmAkO9y-JNd8I(S1q%O~qp%46F8ck&_fNow z97-67>1AQ)*^aUw7N|ej2ClIM9`kQ{UH~l%DR^ONl)Alq!|T zfZ6VDiAPqy)ZCJ@U0Bp%d+YQ9QuP(1?57p;bh@vx;>eBf{Ly;iCbi3az<+s~%IdeSY}0uNPA-aU zt)PeTaqrATDc{}Nbk~dAGgM63vCV^?Uod2$PTV~W{DL;0G@iY>TNCu) zE^g77h{-QLSL$srn_O2yN6kF4^!KS9-o4&G!+pzC#uyOswA}a8jF?{%ujCFfuF@9@ zN>9iGMW`jw+0;jb$OXpbUJu+&0To7GtohO~&ZL4Mn;X8!KyIcu@-W|8c?&Jzt3V_$d_da^p0047_uOrV+Wzs)p zou_kAbNV8)*{wJWI^}<)8ClE4`fx6)b;>s?2;BK7AKoJ+SFH#UFzww(;h)Tecw{m5ztDk6c@;7SE-BNp%V@0&=}q_R z6glLT8V*h~^B)$#qWCkHp6*@J&ZF%@6LOOb7HR^}ilqd^wn9}gJ_=CYrrtC?lg=<# zRN}_im2$DoO88Gwo9&{-&HN{a88O{G`#iZnO2J#Um%ibUv=wO^`m2>^%V(Te8!tNF zQWX74wf=LrN0EWWcX$&jUonN6pA&3cuNY>=hIpby5yHlf)< zj60`7>ehZpO5adBO^MnTnMe95(~Zg7TTh-!ammMeV$l-jv`!{HYayNb*0nqNR}b?F zxn~$X0Sb@&nHH!-ciX+;3eYZ-?$9&xcsB6oc%yeJE_Z6fw;uynSE!rTlo5V2v1XIi zN=pHpbn{N8_woh3ieMan)B{1!MiHf?=z@tQExwX_4{Qe9=e82NGCQ1QUmsd~kzUTc zUMJ)0e!fag*U6<`-oJZQ_T_D6WN`f*nXyu0-$m(N@z?zA8vS06nIi{fvpb_rgE|K({Uh~(-`2-3iNnj?%jJWA81G+os-fM0aDh-TXczXBwJk> zQXshOet)WA$lNx^IIybdErlTejvQzvuBWg za!u1i6A=eJPonBO#0^-0`fYjLLCH3^Q!Pt-BzBieHtXxpmX@hd53(2j;*TiLxpHvH z0**FKlL`EKF)y^smRVvq%9ez#ek;$LBOg)AW5`%QhGT)YQ^C?_^3(dT>q?1NYS7rL zd$~{F9h(5hNnFQ*`!FBaV~J};CMx%yYWnVc0C_o^V7}K``~Vh8Af||9iDt06O!q!o z=7trUcG+{YuE?#uRz(>+b!b9Ab(4JoS{%1WJB=C}hc z@3+}JJVk>#yx>qhGHRUPXIlX)|1Te|<%@=?pKt)Q;1|Tp%`@$D#fb~}K8X)*u2Aug zv!cK$@b;498c-h`^d*yAj^G&8sW)`P{5KRatYD>9{P zn)(29e3V@F0gal(<)K~#fl&Ow$iY1MM>XlR)djG?1FHJJ+mpkg*x{0I*1py3*(Qa9 z9HCL8ceczRr=+;!*B_Ls0!osumjw<Wt2s@y>(99QXgrS!WKaLYHYX%!Uj8;O zZk^5Y-8Qouo?YA8tR&(T=NV)T%BL9y%{@Yv5szo123rjv0+imaHrkz#FG3UV`d);| zx4^VS%!xq4SI}GnH(U+CC{;bNTU&L+XY3j$`Xk>7uBBMg^5!jV^ITA@OJiNfcp-lo zaDp9p_f9wG$vb%(TBtsZSQBi#$|PRWgB0*m6&RgOY8HNEEPAv}KniO}B=|xjB~a&i z#0zL*RyAm`H=DI=CU;~v|FgyDrM=KHWZ<=|MN{=3T{L``64sB@`SDk<>vKmy*tZs- zWU}nzK18z6G$=`+=GfxsHx9LlNUt)Qnn&0PGC*fnvuE*VcM-7ccl`c7nJwPnjX^cp z7e96pty8ZhgeW{tv(dl=xn?yG#o7s3)9M6gN$x%$+ZiL3E21XW#=H6}+HaEcLjO1+ zeKytHnhbo)01$yOVvg)S=^*p_r7mG`UGFG*5-dBwq_nRaIFmsO`gRC${kyV6*cNEAC&y;i#naTRX&BDm^|`v(^g#f2XGA)*>hTI+IU<^G z_i%!Q;O&KK^t+>pc}7xWR# z&%foi0#?pc_GmpQNv)ODq{S9yZeGiOD+SuI7c93fXbL~yIN@{q%}eg;w4uq;7D!Tk z^_Li(eRIot462+{#D{Q{FQo*GjFtKI!h*_#)}nn@G@R|@-HVgpB8+@kDkJFe{7 zsquXa`_gNZ10sN^NS!@VnXuV7_4k4?dRX8Lr-C(6rZB-9R3>J&<@$S0&-@!r6_yjV zS@p1m2p~o#=s<7<-6tUoNNE4p`icHi&y*8a-FixsacC#DV}J68?dJ%y54(cbxUE+e zYtSjDc7y_tTaXC4-$wK*;d*$&mo^wvb;TWy&9Q8izEL7H##iP6b`0Hlfdc}<5TrIj zX|V*$mwrBF9c6%3UbEwDNFsu3Ur4NO$G{vH_dS%~Nq0lT7vwxfn+xn_4&=!vytrzl z70vT9*4_6X5~-W%S`lRy{1B|W8thp0#wm(~z!UaVS_T#*jLE^TNS8GtZr7xo8^atO zMa4^Ou`6@>d~Z|#`XAmcv! zKVa%M&ae{6_W}KV+8_vdnRKfp&vf6*OHW=JxM!lm3`;$hFYYA)#-hF@`U4<~$Zs3@K-71h(203ga$bui30~VFy9s8^OLQ zr&hEXXuHe6L;pq#)Xdha(8LO47kn2;{#Ih=%Hvw-3Q{YikNakQF75Db*_O5|c}ZP` zHGAD`&>ks&Ax&jxr|E#odu!GQC&TlU(MB&aEo~DG1(2Z%=92UGAJpLhqqm>4M6(yK zX`VU3I<{V&iDHK=-=xFKN?jn>p*DCF|8jyGQ$z+dN5i^R{cs~Owz@GCt_ zX3Xy^w`NxjbMh-(U~zvTB6mnPF~9t=;MKa$GHqJXadK z={Ik?TVY>+HGuNQ^Vm}E_D;XQ&~__osXTkVwU<@CUt;+E@p0_m>mqAbe;L^aA!-jM z8lBGs|FA8YcCt?a1wU=W8{}$jwP{n95oa{dOr-$;TO%fOf8|#T0&HQ9`a?6gre2Di zM?(m472LU$5nXppZGrUK$5<1!cFJxVbtY%DWOvlq$CcLJ(-*+c9H0pIm#D_UW-IPA z-o&r2q=`j+9m3iLE7k$~^VZ5966uL|67#mVu7{0Z-ZAoSd}yk@b66kiRxMylqxB3) z@bumtdL(-$R#?OOOC>!cQ4VC571Bd-TYGFLvG?|)l+_Y(@AsQ!>mJ1|fn-;#N7oG47k+td*CA!l+siw0%o>W+`|@hm%7M2G2fOp$skELA2tK&Np}0?5@2i9u zSVzlpCM?hbL^5Xu7Pzxg3~iU_sHeHA=$s61hF)nd$Qj+rmUl?9c%8=-bezONH3O;l z)Uds9*>AD%R5y1<-j>LBLeTAdU8kjt`^3?_Rws zrG>LWs?#4cycO~}co{bjew?aB}SzkYU+lvA>Cli`&05g%OTrDHL}8IQ0vz! zQ>2}bsU9=_VEweZE-XRMuHabp#Lb36?$5p)4uWLJjZH>r23!e2%K9~}I$D}p_r|N< z(+upkq0RRsUbeI(FKc_|wI;H*A`4%~WWT^$x7rDkJim1u@-cs~ka2 z8ShdJ(J%T?O@qHlOW?jdW^MiMJd3&z@IFh%k#z=gKd6~{mR%oq*^yOq+-A1y3U9iL z@2Hy~w$7?)!z`7`2(qkk-;`$dVSC|pYv#1>F9kQGFT6ke=h9R!h2c`q*LU?Du}jZ3 zqKo2N4Q$D3(GEA4$7q$@kjxQ04{Rdm1fS^JkIt>B<-;-FgI_Ohl*2K{ZPCkTM$M*u z)Nz9yS3|7%NxuAFvCxj)5!S@6DM}7bKb!9pz$fd%3r=mOHTzuRwL=QI_nX=p5<9M1 zdmZEWZO0d!Z(S+M9F%MvPtdI2rum*LA`nGp70=hXLNdU!+59Rh?bcohb+>7>iUo~u zTd&ddQ&dvY$TgL$?dDHipk?+IOpiC2DB_P&2Qd~bD7G?4B(iu`Ht>d2qJ74R}xU-wnW{>IUihlPoPG@WNoW3U3W`WIt!quU@6{8`DPy5&Avm zJ73y9w8zXfzamddlIN;{mbstpHP2!-y>phlFy?XV1SEVzE%b$q#^eCZVT;-qGH$Z{#=P8}w#_@@uQ#-jwPUT*-e@vc3Ymm(t4C=F#OPuA1Ho3|9akAXi4vEU#eLzwo<7vb2lACluO6|;tOw|T@ zC}s4C&${yY<4*G5k-^Wf zIdvEfgv4{pdlC0^sPsR~Aqnv`z^B`P$X@*W(0KPh4W2!rJdl*#*tmbzU3hx{O82eN zq-jvV*2b<)w%kmX@QO*g*@39^IsZ3ydkJJ!hJCXsPk|-!gjZ*JJKe9do*t+GNKyF4U?DMv&E3a0aRk}C5-TreERaF#EU)gH|u!aoVoWOSD?r{PF|fY zCd`m|JISC##X+TnLdZhr(=WD%mP{~wSq13L5mQDoIdoLVh|~9q{HMoGo{9#rb*o5cV6n#T#K~EvAp7<5%?C0oQQNX9@tY4+Yju0yS@hX> zjodNjTr?AOV7Hd}4xM3Y-wtYfCN8AieD~^+ZkQQlZpd*W|E~-o9_m+K-(&ilxAXDj zR_Nh_P4Vtu?$HCk;>^f5k=|sL{<^g+BHiX42-&2OP2NIx&<1C zUal7KZy8RL2sk z8J!@ewLFm1UdqDz6jKcYqRw=;A?@Xfo;9)gAAZ(&nYkwN3RC_YPv-`)G=U_=nT3Cg zTjAKVab&B98e|334?niE6>7SQw2Y0M^oW0eM9KR=$ zAbC$JYJp}YWb?^WIgj>A!nG59)a*?^;`SuaYM#%^(Ivd9kRXS8_C}&>t?11A4@;s? z?B*18=i7Ea(^^>3Cfa_D)LaQDL(GNNR4Jm|0Dz;J=iHP3 zx9p73s}lAPAbRgV4OKsCvIyg~TAFQ#h6~+} z!tB7d{s@P#-Iwq_cYFY2a+EL4YdSaF1QX;rahB|BzyGgw zOtcFM%qU{uq{wF8V+uOWz!;PA*C@*u#&BK&a+WG=LC5>@g>$KEgKHSj?uuPrDd<$M z1DQ85bM7$eu6u9_8F0&5m_Xydxn5%7yG^=a&u&v8azKQS5z~VAQ!Q^3Mz=PDi~3NL z<*cb%S(7O(fXGc}i<;{ax)O=~TCU8i@s!<*tlC0&E9DKL>*Y$De9l-;;8b)x&9fD6fryY99pR=>$+Vw>LV zQlnaeF^LIXDtLq!NNEHvUu*5>F@Vp#-r;;Z(n?%=rPQ>Z;fJIfup?}q=5mxxg9WoZ zv~K7n<1N%uMiU)sQm!R}J+QGpImRAgJpIQTDeTOhwYHnWT}n8S$$wGPQlG*J zcs~@bNePz(Y9kLMoMe1$uCZc#cy7{!` z(8bdCuI6fqtMlJw~Vs-zqz?| zp>$t(X1DdvJGf54`av9AMld~-He8)oBVVxgbA($nJ zuwSO&!`dHH!{zt;&eYuE!D-jV{ES}3DQ{=j(Da4!i>JC6*e7q;j6NlO_+VEi%tQly zHjkcBXVbP!I-8cardXyWWUGkY;A;yJkR5f{rlp60<N?vh*P95+6y`Im=I{vqq_zEMLn@#Pgw80g8sAdl(74` zZ6}z}>s-D4l=Swm*J-k2YF#uCyuXTgJ*t^ybTmSMGhLLQ%OD^(wsV7xVwy#JLY*j6z_e`IM2bPBvx~Gk%`H9*N56m zoe6f#AX=O0jfuFqA1jGqkOXpcu!n+5^n>{&RwVhUW;v(2c7luMWfm?DVHrD+rMS>uWLJfY|uK1TBk%iF(NDqB>m);aBYgL_24aj8t!J!(~im(+mUmHF$ zub})n6}O3PD&F{M`tq_x=I05xbA%*D2D^5?)5A+nn1a>E^sT>fT*fQO$)V`fup$}KAUc2_KVS8kGmxV32h_&|t655Ju9UR49t51BLUnBRxQuuQy7 z9WRoW6@*``v9Kn`T@SHVJ$8uW1LQw{V)ZX83Si$r<@3+a327Cn2;t8;D) z)J|t=tyRH<&|CZ+Ce|3W1#O-D3eUAD`bzCDstLjQB$a$6AoM+@|KPV-pCw9QElX^o%2d zV_z?Rq#wFRILQCbthY==HpaN)tYgG&qTm!mRmjQSp&A~z?+?-v+ClDML7Jf#o$7i) za5H5w2o&LGJ9j9}I6h;X*RuG0`J6O6`h#K%f+=JVi)q}7QNL)b-hb%S{pS`bQ^cR~ zE^$o$1(H=ruWUj|1`Isl!{8)#8GV1r$ScC>@W7)E@w4`?b@xI1`lxTEJHMIjiXcH7 zjw*Sbe;#TG$rx2;VQ)UQ4y2aVQARNr3M}RF>ff3oR}J;Vq3otpP?jPj$M=sjchGYn z4G2ygmo9TX;x@RAQvWrrhmi!vaD+GH(#j=~{nu1rHAR)9>Jr&ER9s?!1}+{(HTrv`@`y!z>P!2HGOWDx;`O+S$(d z`fgthY`G6Fl^u+mu05?Pb*{;p^Y^XH*bZ#zyqp1Pk?kH*zl+^BTm5ovYwiW#0#UxMP!QNJ@N&&KOcRn*m~K91oyc4M zcYsm~F=T%gUu?PXi~9RAt(|0cluv*{yN=DAFG*LEcfa62@9A!-u^)NEvM0tQWsIU5=Qw9 zLAV=3Tb1N=f5Kbl$?BfE(-JoC{7{6rHg|~s3qwiruYaqcPx7wpFc#M{NFAgmX?>+K(w4HUz~;#PJ%V$P z`+${-n#oT<369s6a_Wp@ch6aFjxxq>1eGZ^FOvoy2bKGXWC86U-}+_sr4Nps818UG zveY5nQOm2cFGUe^SOZVD?j>*7UimYcDV*U7m|WJeE*YxW*<=DY5sCQ?=5a8ZD7U7j z__t<5371H9((0_YF+Ci#Y_}Y@J~T)k&C<{e-@N(h6||OeU}_&z(p$DE*DBzpZ<+oz z{}jpjqCnN4R%kv4e~(*gc+X00*^WLhiRv7=DC1an-Qrt!oBWo4*73`0+0OFc{;N`m zDUYolK1R0U3=qGNfPBpDS!lidxh z5kZ=evvb!^?#F*ye`7tnXpC=XC@A%YGPVck_=#QAJ{g+K!Bz5O|hC(7V{SpINyV({zjhMTus}c9mE^b>v z;YX>|Vt$SB=gW9Azjaw#s+v(P2lF`po@6Pe6P20a`^+sR+3V%w-!;?TB(KM)A2?m> zPhE{cU-Ige7YxP&Pu(vI^pb)r))zDbh)DgHf7e^shC9TKzku4@cy~0H04W?~S$p+E z_}@$@pYH3vL;YY`_x-wC@lf&ft}~^qwbLMm3};lwVl|G*$>SH1v5i_|@UGgHm5duVh^Z8w*2R6`HZ+JQ%kI1J+$2vk?RC*s zLiknkr+S6pB1Wd62Tf4P$ zH4y=Um9Z3hZ|S3FlcQ_>A7Ld3_@zN|@3&iV>FH9?OO9nRp^D)6R=IgK5dE_xMV?!A z)fEbTa<&Jvp1NYtW8sjO;akBO@MrVUZLbc4R>nx1d=>TM_Vp~>SIDEC{eT**PE0={ zNGbPL6)pD0mk0rVj?U2;%}T^orx(7)Q)q&I&W*0v1k%%%V^`9d3wco@X%fmt{6Oc< z$zLPA>~(vIFxoK*LATwSq!;7Y;@)hL4z*j4rMIn*ex_y5PknFg_$w8{eN@05Cz`Cx zJ@riUnTs@mZBHw>)YVVV1P@rGSDsAzhuk{nC>$J)0yT9`<%75%=o@20rg9X=DZd5Cy#6C{Me9| zZA-EPK294lPm%`8{$rhHVtgq%%z^qZBjov``JhK>PD?_9MyRFhYzr4^`f#%g9z5Lnia zs9T~9rIkZbi;Mj{MITQ6hL+xgYd=rGB30jnXouW;cZEomfaOOpTZ9XQ*H|$Ab$D-? zGzeT5ks`JrAC}}AU+>Aw0-FeucvSW9?ABy^&KH-hfBKVnwe8+9Sn%PdwNMmgSXRx# z@6!XmzC+UA$NpyG<7{`VGu~$oPVrmRKmR8^lo}FS2@szp7f{o;DkHa%GOLtz%?uXa zJbXeRn{Ze78~=Bql6Akm*S$Zm!)46K%sg-AKEE>rlbpJeILpk-<$F^SsnII>VmJbb1ZCq;v}(S4i=1eb)e?XgI1w;cp8jv0@O86do^o3{*G zz5K@Q(H(2Ms`~WbfW!RSGwkRjIT{A-zbRUI)%Q)f0FCS|GWZ&i&X=LzxcDp7<2te$ z5aJ>mk_#erZ=>e#|ACM)@e?}Hga@yv?26wjRf}_pY8F*aj^17|(rx@CCQi7`F0!XP zTcOvV^h($zUs8R&jvt&`X&QaM{#W>~Bgh=a??g>w-AAxb_WrL15Jw>2gvbB3lAHh3 z{A|*>iNbdHZqL+CHMF)&X4#fP03shi7>UBXtIuWLHR6*Wo0>j#$zKiB?g;gKj6~A% zhCfra>)Q3f>7J5Al6^awPKm#sfap!J_QOYOAq6pl3=$fEip|e$R=CrN(w*0}^xfDo z0Y60T9$EfwWRJ@&(g2s^D~Qe%ZMd3WqhzUvnUaNKh)$Q3s(yUdu@zjS=`Zx1#~7(g zVUNQtCMRO5efjJ6JLuZ(TWgd?v9(XYEf*~7?$i-#q{LLuS2@FFR1v;bI~6Z5Q~;w7 zwF2OQ3rmuG`pBNB*~@SNzh*4oNq;s}abUkB6#tlDrR$PNC0PdAQhvpa?;{ zC;-ua#2^Ixs(wI@4BdiXZ10-aHse(wWB^evI`e1l7F}nzWql*A`zTR-XmeABJ|u2l!vSuUAmrOERxn^$^%CNJSKlcaZQQgy(~s3Ty2kuT#(B z^O5KYHpcw~Ah6|~oVPT2Yhp?#N8g1YK#Wjw#*$LcMIDF7VCD@guYOV(mc6%AT2bD=Yh<`dQhqq!Rjo;4 zI>Eo@QF>TJZtBx%J;nJ#qJZh{Xf&}^*~hwJ+IaGW=@(36p0aZL58b2*NmL|+H73uy zXY2)T)|GwVXDQ}&@mJhr^s=BhL!g-vj8Q>yGY zN9*j@ja?FxjyN%`KK;o81*m)E3Hhwt$Wj|qIJ#bz+03EJfX`s0#d2h6A_Vy2MrFJw zFCd?TaQL{tJ&%}}_KQ`bOUpkR>i=fy^< zuXmRFeI1%j_ORW4!tR8~HerL#^8dLgp*hld;YnY+n`2f`QaCbKEdqHiECoGN4dm}R zWgW^L%!Q~(tb(UdN5J*>ZC+u=CEtucPz5RwB=dh)RbysBcBx}`NknMUJyIeKOrZL1&5yBA1mgl%0=Lsyk2FO-Z8U zL=GFz%7<&B*14q`G9E*mcys0xiCk|GK-;P3r~CG9VE$8|{T-rxqT;gw&%GbUFQXW# zzEe=}m&q=&z#!X9P_nk|_qOJTi?ZE=HZ?1ppZ3e(4JJiZ&(V##Q#SLV5jkPv#B1f<8EIa+YRANxaI!B;P0XH-vROKR@2H4 zU(%UoFPgoRcm3f$hdD5ZEP-iQ=d#|Cd;xW*z@<7kj z#b9Y>wepk`QK$IRp#kE}A6bS~+Okc<6q#+iwTo?EZfqh2r~Y!e-xJB^-CK87^t9y; zqDrZNcN~rcH#vNQsSK-j`G2#8$MjdAN9ovw5sZU2!P!Cgs<-6|J@+_U7aJ>MwHlHN zaWGZ+3y%B4tLNUF@yZMycJLbTnaH~m-R2Y+a+kN0Hp38mPRyuZ(b5R`Ea{0o9Xc1Z z90fYJhvdEJ_;M0Fqvvg%zO;i;bV-*?0Rs2SOhD8MB7}ljf2Fv7DFKCcam%umen%`5 z&^TAUUG?gl*M<)3W}og#bn{^XIQ)T@s+&oU-1z#BZcLy`BD8*}(m5gg4K4kB7QTd7 zlE7ehG}Ds6BKU!2fBH7Ej-reWMQh^`v860{X>e(fQ(D$T7TR(en-q zFwEk6smW>NP=NV86U*dd{W1Y@lLglN#RpMHeImkH8Vq>FXJWCZX9vrTYNIGX{1L3$ zUpKj8ijX2`sm9pp_jHD87T~ATW}SkYe73V*`_0<%A7bKmCXNl6&i`aQZGZ3Ra_|@c!_nwTBtj5tBDk6xssFaO`PO2S&tZ*xE1_DcAUI3JT&Hz~RIE8Iuh%LGX0}vM3Zs% z6Ye_QNJ#-=^GbKp^9XbjYFTzht>s5l#sn_m0qLPY*Ukww;SZ<0@F7$+<@bK-eB}Dj z6E%8S(Dnd)()a{ZenAp7Bbg1t#3G0BpmlcSXMD7expplbI<^wNpyWkQ`__I75IIIj z<;fhox@70uz~e9L9Zx_27lur7LV5)t!u&sp)40;D_~>SZ5dmCo{GSkRBCL_+JttV^ zfgK>?o8rh91xT%Qpd5}xAk>|zd)J>p)Lou|>||6RSN#)byoG7{ay)p*@x1f~ew5Mc z+CK!{K)bIG7`6UK3mE<%KD8cV172|5>O++F^=TR?(ioKp!hXBDK7*X79eX8);X6jG zc8sGzd~lLyY2V_BsR+6D{%d1q+dBuX`ZI8Lp#&&Jj!du-_(B$hp6X&HXd$Wm&w$%-FjB)Q`7A!CxBUI#4rHz{}8+a2E1Qu;YyHeRNaA^(mspUU$IA0!AlL7XK#_PA~v zq=p)!GWJ`a6KHBOafTBCB0{|1_$26opQ|GCU`nTwPt4OIbo6?DP1rqT!LQ!&-@IP~ zh!}`Ks)p&Wg)+d46mStedeIr-I;e93K1A6?w-Eza$ickF-SJ4n{BQ;P(%zUn{i)8C zDnj0y0!c5L2do~6A&ShoA&=!&Kj;l1*jOB4UTdLu$~gajB+O-KS2kmcd}~E%Pzys< zLf~rBW?_)+_t%9X#I2k?POD7ywq5=NkM9o>(trWq+yNS96qdji2$kzu=?qN}G(dMx z9{`vvQ?P$6Rm{a0+mDA~%k{4$eA*WJ)*kiwdy-S{{Od5JV9yGG##Gwo3ACBI%sy8Z z<^+eGI7O=U;ejK82H1dOmo^1Q4vFJnsrmiuaC~kK7@;C0fe?YSLR)?8_A@5dP;6-z z%i(aT-oa7CYSz)ihR~OTTclD|$^q!H^)NKZ8D#xS_3i+Z#Oh zaL(_2?gXt7YLZ!Q*wN^`vypXZpuaI*67TW_V@{WJwCh-iDDuHQJk<{tn4Y}OROh?V z-x8G5tn9b(Q_(Wj(R(cZ{A}hvGBhksD$t))?mgjzstI;}GCd2SJb=LTuZ zc(&ZQ;QwImt%B-`x^=;gTL=~)xVt2{TX1)GcL**)0|bZQF2UX1-8HzoyKG!~Z(2u{j?vp&1G|r@!3>waCD=l8yhsTtiP6S6{nGJuO7fzJKXY5Smv{Z9&*6I z>qf@NlNu%aZKpHix^T@{TjQ;A94%Pay5vpqzyQcvprIQ`rhY1O3ko+#t7tn}$vEJY zFP!g9JlZ-skPmntV{^A7?}=rIpOtD}UbC_f2yn{}!k#kCON|ChF&Yj^?GNdboO6d1 z11&`JyZ_EHiDAv9fuPEF3E$7%92QAc4g!T-gP%d50wiF_^y?+`%9f^#M}LA=REc9h zeI|_NU9KE3(B$M4l+v(9aV}+AppN*_Xx?Nl2$bnXV7gepM zMF_3E7O*1w9b5eEZf0|ndoF5~d$~b9q`~#%9BaOW4;F8UNbIuy+2j2)9DjnybD*mI zQ!o*A18x#02s9g=NaOFL?{@C>#HMvOkL5XYa*K^DSIsKg>2_v%i`SXcnT$|1*T(!M z*L`8Mi;^fIkxFdR)|BH*Ae!IoL!mF(Q(6+^@b<}gm%Iu0$D>Z0XWGTlBv8UX>_@YeMond1L+Ob&5Xq}UP2+x48mIRYtS<-#m2cy&LUXdsnU!(laQ!HRGEGl6|JcFqJ$z zD-INV5^r<+m0E!{Ij~P*T}j-Tepd9)mBG{by^bmY9_YK^0lj%12?y87q6VVhjHdOv z{DmXJn?9}x=jMB{Z-SN9`6b5Rebdr5WT6JZ9((^dRS-EGWE##kY?G3P-cCPf2+Kd?=kr&KWoc0Awr>AAJKu?2lmqaU zpFyjDpvzwjt6s76t!xNs+FRv9IpinmRrQ`(>$h3BZ-IsjG$ThovCq+G=$I#&C$XJ7 zPHZ(y;;rSc(Kh(Aqt~&V6E~0`Ie%51T6WToj8kbHPJ-@N2Vg8+RDw5pRoqPgw1+5 z^UWjm!@+SCcxSKHO#<^Dr*T=-=qF3hQ5yP!*EWhxk8SDG8zysCN#9oUgQ_DP?_5cr zQwFtyL;9e4;QH7Ee-I&m4&xNfFCn}Ifqa0`1MP6FOZU0TPQ^`=UXs#8bMsM}nX1Zi zbfn{2CZ<|E-d9#33PZ_i%)=KS(@YBuugV2@RIl9qu)9`l*!Q)95Wi ztv!76sOud2$&Nm(9q7nB*9o$s*-75{5k2@T;wo8zJkF5d;Gic4kinCn0ZUN7eDiUh zwpvIwo~pGr;uS-SLC)Qdqi1x}))(A-I9?7lQ;2Fh^|y5o)sIoJ^V%Os%;bm;|Dd@t zjM@_AWzzTMPXJtVkLhadKE<=PR)=NgjREZ)OY3hpm+%G)_3B7>c+OHlXIOv&%}$p` zVHjB|b&|bOV?dvDxtLw$On+~PPJ1rC6;YcgIyaGfH@BAQAdN$gLDiMtKDvXRNzZ%e zfRp54_RUtg&Lbos->9o%HO2|!NjaeRm4se{3SC`0?%I%QK~&e8LA}Enr?jycV|kKS zFAEz*8SV(;h<1HfkK<#*KRMn<)Wx#RBX;JY{-%>fHtT2)@Jx>rSG~-}?Z{xSk%EK3 z7y@@)^`?`4SQ3^ZPFvlO?tE)$7Oe1q1NN5-b;Z8x#vVs3tQ*ZUJ-t1IR86rn{O{z6 z4qda`EQ(> z+g~krI~ePeUJ(SW#AVt`gbr&qPCx&CV$L3mI3o@F>}UBO!T8BUhJ2^k==$L*t+WNHobH0GTrDK`RrffB^Z0$(;IEpClExh8L(W9g@DQuQ zbscs$uFY9aD#tU58`(#X|Gk7Ls#l2m&Mdy{CT&-4!YSYOr!g*UunJTGh(oX9c+a3w&QjRSvP>=0*ZRE z4FcpZWupA2;8pdS>hUW6O%WU)h>?a)n^Rl&TFL~pM*TBzS<+gkm)8=X%Hrk2_Xb9H zkQo{3B7KZ9>67K}LYW~OSu~CV@)N|i87q}bnYsttoNX(Nmp-iUVOkT@dxUTB{v!`$ zcPxZh9#&gv1>0tBVM0*gr12v@TFww(o+du|cEy{Box9S!mGQ3rFH4-&ouU@ z2~MHOr|)B@f{`=&>{J#uNvIZl{It2wvfM%f2v2ONTnXY-1;6QY*?!13TcSdMWMQD$ zSyV_Nd{YJP_bua0)`;6k6<`Z5S&aJPO)=2}$s;yZf_6t?(oOVA#IwmPv)^iBhOu96 z!h{Jwqu*K+zd46N%IU>Ot~2O4XrEbvnn&T z70hrD2j6V_@jE)28%hKB@YxI5L&Od%9qFgb;^n>yb&lZwop>?5y50?$AKJx)!3{6} z3-S|#fLzIQAN(Xe>%xhR-jrL$ryjvPkVSh*t!PgxiC-Z8@>W98!L*3PyN~-xLQN3T z2b78mZ!^$jCr#n3NUpgMj(d??RozW&vyks>`^&gBFn^R2!O@HLt;yE-E@=And>drJ;NG!(Z#TY9v6c4U+Wgy5eO&nd|jFclo(^9Ll#ZEu2$9P?A&#`Hsj zP*fzJi^p8t%YAgS^FiId{{-h3FW#2pd*!{*$zu2zSAX4QBRw#0MS9HFy%7< zsaCxq6DH3Lwke?YGq&8h2(JT~pO+E3__LWHX zgxEJJhTpg<(5QV-&$Rg{>)qL>$IYZr)3>y>+I=qe8|?R zoq{{AGlCB^@TlLter_70KPSxiaCTO&5X$=w*4ugalg!pTceRK$`OZ1CWZ^u!gMR?B zKkEhVEJgB{XL!EBh=V8k<1ciEQ9FJ=BHDB(xoYoo=iTm%qm(NzGJV@Koz|QZ85LcMB|Y?P$B*j@w~t-ySy&TQ97iuiY^NZ%rdmz!Mp@aB6w**WHp*3W z&AFd-J8BQ7(f-^j<)g6^3n#=>vOU5;Qn66N_l`kXfGDr zWaoPfE~oj5Nw)#NrB(P$ZRoSd{9<~vdpElxrNU2%}ojy)QvqU zzmB70BDoHhH)DOtS~!It!8os971;jx3Lui5X8afp7?|yaN-hTA9sEbQA)5QV=V9-B zYzfl7aHoA-x4j_oV(BkT9B);SCpEU!NMWUEX$}j`Up`8Ntw#^D*{u4Z-1axqs4ni; zDRA7C#LU;|*72R+#S+Agq}k{s!4*>_hHzk%2&`H!w8b~(zC7J(CKaQC?{haF$Dpi( z<_Nw*qb=Y;rD6b`9T}2c>ki)lZFKzm$X(nZR9J!K<7UH?_KK~Ul*WUvHXvR98 zZNoU)`*_jr+Y8;KG0!odg%Gu{WH#9e7R;1R9V|=z{GGA2&W(-|%lzmO4I+c3lBzwm zL;D6zR-?CDaQ=Fv|?<{5vw1s`QLem*FW>H#hgC+8s62SSyK z?QG5vQCsAsU1$J1V;7zsugc}EVM3#I-z#${XH@KFI-(%{y5pXc(e}Ed%N)|zY*=@J zn#ENVbl(|jQLnKeR3{7jQq*u(a6ZJui1W}dMD3`ZUX*3e>nTHPCEz(0H(0$}!~b!g z>F4O2UHnNiy_UMo&Ct5CXBwBjpQO@LkZFx%Lz?c2xA=H+RmGOoYW(1eQAA-F|G_53 z%z-qe1qoaq%r;L)kuh@*!&4|x*RFdWjvgxaD%moQd7FHg8f9W9Osi@k*zM2NZ7|^h zKF3Vm?8e6wC$ic4N!+(h!{8%V-Wv;{n)hSN zOp|w-9*)SVV@6&VvJzTZNYZ<#nxIMJ`F@hXt~0dfBJ(gQhse45OXwfDeZ8vkVy_V< z?_fAA`jEb7m2%zKfqaU!x$}6EJ<^XfVtW2VuYS170o$ngqzU^}#R+NrU`Adhj~YP} zQNe%|IC=o2;Gx%OFe~#c>A)@Ijk2<;87Zb$)vla=hN%CgTa*^^qqSKHi3AfF@HZG^k*Rv(7@&+-mplnLC|v)IIIoGTQtI6u)p3@?+d zW$ZsUA?&4CHx5DLo+O=28V;Qe%Dz=B+jYn@pZ&>bM-FJmWLC~!62+QIq12*A>Y==)l%t_ zw|=gHh9JC(?M$N36TnzNZjHdA3Za%Qv|lV$)c*nrBIY5kViSrSbX;yJ;&O9{3wN?J zidzb`?t+EeDZ2e(D!GDz+RCRxI&MN?)rc`xC^@zbKUM)(pFe~^09kUHCz$H3ykZCi zAf`aI_c8j+WlB&BtUl3_r+IMCnW+dS*|7mW0*i`4#Xaq27|2O5d?R^$*9?|IPSXiv$8zg^-2wM(d6mJ z+(15SQ9Inaa|9c4jeRl(YvO<~PzjS*3Q-mwTfV4-(tp>avW+&Ysfru0TV?Vlk%yTk z`VlhEA$3Qd*%R;+mYD4KaPeE$&y49=V#^=o2bQNgX;L=R@DZ4yqy>_5xS3*e3vdoF z1_hA*8C&0xM=E;3QlXsDYO}TYDOq>iiIg|JHz_op%UGc7X8$gBytl5DuuB~U6%m9U zB2-n_dzWn{r?{iyp9_9u-6z{6`-TCmh>?YYs;@|bH-JzeQa&~~tPh3Q5dO~?b`c%9 z--kR9+*&zfpcyGt_7Xq>q4q5j?_t{(_gt$l<>KaiFu<5vZV$q;!;fH##{0)1rQwiB zZh@7=#VQ~Zc%Y>cW5=SVifd2e zOk*ssvEy}&Dj4SMbJEVh3vWX3i0!)%KZ_UGyBSY}GR+K4{)O>{hHpd0Wq_vr^i--c zt-j=JC_ZZa>_fVxau8y}8)1Frd4d1Ve)1$Bg!m@N6lhwPc`vkP|kj+J;XrVI3HrnA+&yP2TCM8|A3K&Sa*#nm(b`j z0Z}MVNqnQM`!?0*?`uyW+;`5SR5w{6$erk#`ToL!=mj)d>`Gow1G$ z$(OCIvA~66SOeI9?6f>CuY>Y&yuC_(8c?S!>rLQ!!|+K53cW4PN=G};9B*r;~4{9R?EU+#-VtM~!GggIGY6<26S;%;vW zRf5@VuVDRR#RYnkG5Fswe3=)`9 z9CVV7&FojhnB}Y3DoTE;oTf#aj$dCuZjz((zHy#eeq4@q9z=chr)G*-0fZjcLL}S^ z6^LDj(z?G~*>gFgyCuo6L2#f<9?w{vG4RLzOd*xls2}F%6jeyoJ}rh1{qqd$PkbOTPJT9 z^QF)`LyMxiHl|Ju0kJIvGO~g0K@Wb;IDbZ<`95Yb8X@A@Y<)=?CA8WZ2HCyo(IU@XU`y83lO) zB&9Kx->03x917!7ASB&TjlCr!C1hd<=onKk#8ejoW(NEJ%uDOQ#8DxMs6n#=nFWAc zjflBtL7%K%5g-SjT8op#P&Gd{yVP||s6B6&yVT(afm@!xbJzw1!;L8bFz=rnHoh6E z-BCo7YZ;O&QFTw|9hISS>qxA$-A5m!vEkEsrPledw- zcfLOlwd%l}9Bzn3sV>_xL@T){o;(iXkNfQ2U#Az@R=KeC3}dF--jwz z$#xfm(CRwN-eqo5hNEslAEEDFQQhoN-4QYYX30znHAk`_hNJHS!5<|fq}=5& zbcEApZ~m+%7{3^WIIlR`wiMcyd@AVJUPV3kwd@r-O1Q#hWwP?n$uRe6e%LC1GSd4) zz7x#GE$%`pZ@W00{qTkArbg9}%)!fvQ{OPrX-DH#I?-CEBI)^N`>)YT0v}L)!QJI3 z&P79o^qWr`Z*6kG1JAN;1qRMo0{NzW>Vg2(RCbia69G02B--?d@EdWFCzbHIO_F4 zDE7$#T0v98=FuZCe9Z^VZfeLW2HvzA1~DSN6qkieE#W}ma_gHxYe>LlL2)|ls-65V z8tl2j?$MV^;Z)y`wtJQuG>2X0;OUK>C3rLUz=T3-HACrs@@6i!`7}GHaUpL|Qb5Rho z%#QZhKmE)*5IEf;g=Y!Y=*$}x=;=Wgg@5`@A?N7cHPhG;jO##^41nk`@nz-gW8>a8f6IX;xJ&S`iVYuo6KjVl(*N59g42@^Qy-0@-BOj`y- zufxNijYNh#2eYOAA*NPaBF$b2{RYi^`CS?-3&W_Zc*PbQeif_eUwya;VAX1@6-0;9fQ2}hrjwB66u8`N(s&*&t}|JMM!^8QMjwo;~xeL zqRSZJ6=ggbXoVVsMe_D6!M00_6`ULejtUB@1tr6WPtnG}xZ!|t$J2ea$TZi4`E+Kb z#K=6losg1oy=6LYla$@&Al_@&4F@Z(`dBTJlRxm#L%R^zyQa|YPx`G+~Nh|{D zX+XH=W-{l|PeyylC{2c5m5s2`$J#Wo>3GcGZm|OR>{D;Y8-?kDuY1@K>ZI~d4UI_$66F7QWJuTaGWrXF$4uAEmPO#Pox zbW;@gYtdp&T=3FiGG$Y8TviPxn8r632cK{iLUQ?}bU&Jhm+6Pf?0&8c zil9QBciB6YfBBBaJ~2Le>R@`6a?qE0GXR4)Qt`g&U1EgBy`!cg028wfCZahumio)Zn!xR>%%Du-6Z;o;EQJdr2=K3VR=cl+AYefw)-}_9;&f=M zo%h6;t=AB2-)e%$!20Az4A7%;`C~jw-Do*hHX{PAgUVmO&xD4rL<@N%;C_xGOD-3N zBE~}1P{5?*!K6YgN=Bw(m$La=<#Tes83RpN+kgm}m#P2>Qm_X|GEdUKq(3>ovzCMX zNDEg{GrSSh=s_$H@HS>eN6=YowKFB5M-$Jt+8UW2G59Ie^LzsvM}WrNc;Vnyiwnb1 z|IJ&%WLdO!EH#X{b3^hO%pOoqS^508EH_sC)9t5-O&P&!_c8mkPwr2P=^I0?-&^C^ zIEXv8i_skQNnVK^da=w>QJ3TCC!I@k^POE#AW(_v` z6d#$J=8Hv3(xL5pq?4^Qj^)tEfyQizt8jMq@jgmNfBuygu9hF=#Ih8hA6$Fg;OW>R z1wlP&dAc7HS(W%QuksVWZ9UeE*M-XC%?O*LUjLf63vF@9-Gp1Z(f7=&wdTU*`oqLX zeU&`@OqS`m7M*qM%~WsbH&UyS#9R>#{)lu0Np}d41vhaO3&aol*^zN=J#>2U$E{>z zFJ?bQP$B&qh6$#N8yck1ITsH#@Z4)XWv9N9!&{R7 zBRZm5u@if^5L;ES;&jT6wdVC#UuRw8&x_Jyq?lLbG3KyO z_%5(ydreVy;KcUzk~m{76B28;)~f^mzDnRh5`K4i^Z1M2>|!a;{hWF{0cdI#pqc`4 z4wQ$_IYQIwQS+^q82h*@b>f(p+T*ZQ6Lz_+(Gy2a4)c>Gi5E%n@`IQ|+aAil40{8j zxI+NJd|LEE`0LctP;572X9(kH5cjiOw0;aT!$-@Vu?Tt8B2rH2OA>*7u1{%S2Kr-E za)p+?fCt8MxD!Pw9)b{?oju7ptIT}wuga{)WbBj=0#YIYL{?tjWhVs%9F?Cz3i7{C zeQ|;LVBnH9BS9n}6cX6|>_hD90+Gi5ZErAk7Sp8&CE{?~0phQ2U#XPkj|uMp#4@0) zk?ZX2IUwH(E~`;P$cbI^);!F;reqG2AF#G*LmvV1e`2z~?TXK23B3qj8pDtz@ze{6 z6gL*4z8si0k;He|xc|hpX;$rQbYuKI$~rDo*n44)b{^*9Rn?#x5I+n(Aj1I*j0V^8ICCh9f!`52!scComf z<$QHRf1hl05T3reS703Q#_P);e_lP&pXg0T=;4MWHI6}Yc7%>2$!7l(=NTE zzE3~LafS$_GC3%pYER7jJp_!FK9*d&d2bhRm(3#<2tGRLTCH7?9Dm|ayAda7@okUZ z+`W}~U>^;Y8n|GfB({GumoX~_CXN;GDs(BM)+UvQKWJcuKLLC3N8EmzA<)|OGaWS# z{OOEw@^=w}HrI2c=$Qyl{i3Z0|0n(W8(hNE1iyGf1rl}&abB6uAD6Wdp98Lu*7Bd2 zWO#7G6DcdO;8xk{3H6uPE03G`H9l)yjO&ZQ1FM@FB<((zWbz~Fu^jww5o;%bNudoy z2GhGvrGITk#`L$l4c(@AZKxbZnYDaL_Pv}H+<=4c;oxaZ9feN7|D&R+CH4_}j)GGAXCvGO#zBawcdd>u|i5EOto2-dOngsl_#X zngx2EL<#Iz2LwK95f$Grzf;od{HoYC^VUkahkZbQp53xi3zO0_L(?Mk5xh*Ex@N<$ zRFsLcm7W(%uCxu0IVkB8yvoPKciAdLQV&F9$}awH_zDxh7s$-Bf{i#@Mv`=W)^mwq zOZs*^H^|HMh?H*o-?t4d(r!S7#Sg!cZ~=fuKZ#le{c(F3V;f;v zs+B6MZ^m7#yw0~Hh1PA(Id*1HexUODW}sxZH&3;WxJH&eVVTOr{P$X=tE@g4&vj+_ zOp44v($<_C7)sm4-?je?<;lg{=!Bx3XYW9lKRz#F3~j>hNApF%?2hlDG4^$Z-SmXt zTf-ZcpwmB$r?4bKMx}Bcms9_IXyD_>mzppeILak;`%)73_qWZ=rs1Q2%f|ZEy`szJ zeo?#;^p!%jx~>Q|BqMD2%SL&){0GPkEPGYsL92}x;#P-Z#9jN6hNr*0Us6T9v3=i9 z-()I{lm>=6X--n728#z4z#feQ)`hC>WQe-r$&=lva5-EdcZa+yC=&m{V5IaI6qJF! zn}i-hbEQ_eIVj3)I2&$KR#MG@saqEIiid@e%=bXnh$yS*Rh*(wUP^=c&CvujLAvAf zLBb|WYx<{;VYxk`g|f3h$o^RbkCVC|rr=d&)@3DMQwJl^qGs$k*3M0f@msqOf zEeE|NU;4WezM3qj6N?qSD{}*woMAs^u0x#A~~ z8(zO7d=q;X_9(CD;~Ydvn+EP?qPX>*w>j@W6?=dC*gbzi|XncdDnoAI&L{Z#VIFvUz~_eGU% zS+V(-iovfZ+f^n$(L?^Fe!S#zYDx(YjuXwcdWc{~vk$zk5y2;But0;5PEpXLC6Dh` zQwP6=j}ngcVgHs86TbWBNd~Qwf06fm7m(`c2gC-kE4$N9vh z&7y_M9oJdVp2an~%%_?GQEbEn#8sL-H}n~oo`y1+bndm72u9-+@fhQ;{&H8biYlvA zlgMl|2>%57umH&@M>LtVXI(Gw!!bdvFV!ORtBckxzpPj>XCf=$)&UG~EW-JRX=+%R zxPMtSJ%D6dNk9+&S$in&nVmgRn$ytkuhg@QndHA1F(~MUU(g7e9>bL&NZp<*$GCwd zbLKxlnk_2azbcj4(lz0uDyJq@T@pfPFZ#~2;!Yl=byUgN7OcD|np;+NAR(Z5|5uEW zmhQQey2_GXj6x#PiBny%tLkuS2Fg&|Q~XH^CGXkq7dwyJq7ifm`u7E$e}_Bg=yjud zMlZXbnx0O|R5EqLtX1iIFqugn-gt5^6nLTBr5rlOs|gs8ih<|z6V|x08o>zlKT%r4 zaiClXxrfv`07s#N!6XYbz=;7eWpQ=ZzuksQKMBMC^Xo2banhh2Oov%X8`Z08PFUMe z0lC&LK$uMAhmEucLYQN5q~mV2NvAmF+cbNtKN~6GrJq4l8ZQ?}%D2%1DLMD`IfORt zelR@9>#m|m|1n@CVU_{gixyG*Uz{6_9WjRt_Qc$GHK}e{Rx~- zAI)Vsk=%5&kKFBwz2%>Dccb)xK`fA9WHWWy{9pOP!p#OL0x7tEJ*aA znZ3yi4U+fw$!LPOcc`yvg1<3T5Tmj{+c;l!d?CRT27PgDjANaO3f6Dzfluh2+v!ALLW$Vy2~tOb3L*v9sH)QonDs zx>ddqfu>f+sw0RmqC|GgQ>|2JbGB_-u966h<-|C=NI7vo3>QV>3Gz2Nr* zyLLqCF7ohFAbHlEk-xi?<7u{*OTecx`be}Mii=rGxGUb|#Oe|%pCed8~*wX}HO zSzh>)MyIQd2XFs*)*yjBUyl1+7Idi?W?9T9rrbJi2p*rFJKujC^iOtNcpdP7m;c+^ zmZ4E^rJCp0@z3{$jK>Z*o&L-6mhrGS$S;P3MGxfXmv8*tXd25B$f{`G$F;JzU-;*2 z1KbKAm)#wrch2^mWw%3l#sz&}KJVLHqPZ*(XdG~V`f)q%L~{IHakS5M>wI2&!rQ}r z8{c@5eF_kGfO4(wC7XC+X2l}VOBWGp)P6w(c2Bm(aZg0_$Xi)9o7|A*L6wI zgA{~j5b3;v&FSx4|90lI#x?uz#w(CGDinEso1}P30un8Ge1QAaHmdCnFkoRk?7x1& z4t?7iRKB&YwEu&A(GlUOUSyj6CTO6%9o(bU{;plAxRSsC7~+1bf&^{39!EaO0{JHQ zZTF6z;Z;^buic+*q<9~On z0Gv$!?rNF|s7ZJ2qfgHgJHd#C{$aCpNho&ML~eYC@VAyriFU_HE9<0*n6r=r3ykY` zT~b+1P@pb)7)Wht$N__e#uHP3ql)aeZwkI{?K7{g&}0V$He}oH9ebW_4c@^oXHaW5 zv*u>D9Q6kevUq@o^!*o=#BuuKIaJ$ueSp%Gx@{fw-#>tZO=V?d9NQN4r*^CWzmV}N zmifx41#+X}j=lOxRD<|5?rerN0$$A;*k}tB?a72Wer|T6QY+N||w@ z5D- zbeMv~L#^)q8x}Cn#2_Ud`tNm?EI>$JhLxl9E5mW%JwTKOOq?@>I#EyTJfKbw=nOnV zgJu!oZIoxu7`!@)BDKKUsL}n}K~=W81p;%aKwgFdFRII=GLe(+Y<=S4sdSYQL^6V) zf%Q*E7(5PxX6YJ>W(3`ZiVg3h7P+OC<7^081)pKl+jpCHJ}rte&E>hR$97@BoHoDo z_Mar9`|juG)65Jj-9gb47sID=x+SsNRsM=9?~x0)zdo-#^?a?qc7KEqc2f~2U3=)I z>~ejH_MUkOs_bV5R~y?~`&w3mf4qrlB|>eJVW)CDJL_I=o3>UBx`th>o?VZ+fJ=;# z>>8t$V#0dYW?gczbU=d2z%&GqWU}%4tR<^{*iy5;1-(wa1WTMkXJ$*wyx>HHsw~sG zw;pP6sM{&&Tdl6Iu=t%Th9vgYN3AlozwlR@e}$3q=``D-7%_A3`T2QBMGAr!@N}rk zwj8ndml7(zODb~buKwL`im!6N_b~bLc$ZRGdo#xlObKr!wHXZGcKdpNiku4A*Xy1O z6rGJ0dH?Vaqe>`}&HU7fotXXVVe%vC9Q&dJ8HjYS{NCzbtD9a}-bqCdfT??)-$c}H zb)*7nB9kn>E25NNr&GDLPU4?oA2`RIQ<>&3Khuf}7=-ZSEp-H`E$ShF1ZMzF5VMu= zL;IHN`Pm%DyGOh>`9|KLhGNus1=F+jMFJ^5mYE{gH}c1tnxMOK#is6x^mWc*TiXDAC0QdpQq+?7T=*vQBcEHU+i$XI7l{< z0%2C?s2WHFO3&^YIP5l$>HB5|Z{|Hnwc6|e2QMZ=b>nk!B==lva#05h3e6tjN01=P zKi<-YfQD|O3AJ~v&XVQvFHMf2p)JnB{wn@nuyNE)0;Porym95t%4i;*D#8+ULZ2&^ zo;IVJW0v&_btZ{gN5+3*k?489IS*O+N;o?JQ=`_U+|*)4e!R7PZqPN0{RZaqF{gY! z%cth2`Ouu`29w0 zs|H-5<=_7+c|>+a%|n%X@>`)p0#9W?ajR1tUpfecivw?id6!%+OZOP4*@})y0e>g8cO-(oxe> zQG4?pgD!`X*JfK)BYYD!Iy8C(2f-ItpR^ZBj;td0hxk3hip>5|51qNXnCMWb%$~uT zoj(J5Ykt&h;1QQ3C{XIZk!ZB3MweUO=X^*EWn$}#(51ybqs*Cm^^`oDv`<)O3g{hw$OLSEOAA`qEVIu|Vi z^Wc#+PirG~?jfSf(Gl<<2G@} z?_wR$EH1pw%->`HQ^A7t_3RzH5~mjbZM4E^t?B^I9h#5F1Qv8#7ZP_`Y!DkPnG=b-s`TF^1T7*JAUr#c?MYPuu;KF!ii zOSB$eO9H2?kT#l9jk&D_v_++T)o#8?f zWxXv!*|XR&`}(fzCLOPGK8ISaIU-8Z9R+cB!j)O~o*bieU^)9H=X~e@lTM$X7|%Og zSFa3iA5|7P9UQ%PCrgUP5;!ewL~0h?pS9$HJtsC0CzBcu3{R}F{c!(Hh_W%K`jmOp zQ^{@0jx}&OawN3p<~&*U>4A@Jc4OWrj};j`q!&ae6H&W&F(dnRwDNV40yh`wBDhkV zyw-E>G8R|}K5kPDEtACQgnRr4`MQ-zspk)~~_xh8% zSh;#)&`5D65X&_K>ymD$;yd2-n(N5ldZ23dkwqkf$5)fDS*ATwbjSnsXwj46-(6P40$gWq1ee|p%-_0)#h9r_2mfi!L=c%@7XK(~;;^88jVaPy0 z{=_0KN!CDqX{kfp>u4D3cJX^%Q?r!Y*Vf4X1LK~SQpzm*SntAbCJsNT;{{c)jQWqD zbQ;I#Kt}$kM!PKD!5?!Q1+A%%ydOKHd1a9QBkqTr zZxNY;+WfZ!jK`g3VXR`C{Pd!zqDS1m$$2<8^zC_Bd}il3qnJT6z-;^U#luI1`8}P( zVqk$?(`KznKIr59cR$UDD}K!g;(`Pnwo^5*h^o5uZ1aqe5wlfBlI+yCk9IS881>cz z-C%wI<0YnF^7V?KyP0)Ufw`@B6|jQIXIjtZ6+sK|oc?L@J_9%WCd9j#vmdg=sp?)L zsot&6{J2U?trVkO-r)32Icf zqPWd8c-}*RAOUL_0v=hM5c!_u=TmwfFpl{4F0z>xT4%IH^v6KOQELuNn$MVggK3ZA!?S@(fWqtQ{J@BUE4hSbKQ}q&@CqjXSf9EE$HHm(^Ff4_);CWcG=*dv3qy#}=}X7wlrNj-U3P zYJ<|@PJq-}A>au>kbB71X>`EY{;`^?sLaiot9+=Vp#Q^UX)h^U0$^C^8|5~lbgX;@R z=(AEZPVMtr(z|OPtvMwNi>T&Gb?gk%w5~_kQ$JU%?cVM~b*+nNQnk|!wz`z0M)pfH zWBa?0Lj>2dEx6igETIBIehUsHSP$5VKGT>$**=Lq6iww_8xW*vAY9?l38S=n2VgJ- zk{4o2s8@F|RWi><*JV4Z94n7zYU+LZhDY&>DQ*HO9q|i(SFD9yt|O+j;jRb~x4Q)8 z+wX71gs=nqPW9>VMpV0vHFzF~kRrV_?-Eq9$%JdOofBZ{?=Zy1^`-m`MefK=>+m2! z7ho7zXFtzaI4l#E1{&H*e%ikVtyh!P~+s!@}OnEN8Z=D=U%0S~M!s*)U{0qc*v|um3;TddKj(x~Tnk$7<}xZmc$T zW81cEHfn6!Mq}Hy-Pml)=a2^h)+#du8r5=NR`res`u;Yi6m*sD9j{ zjY)F{%DN;n<85!y=5Nj8%z*n_LDCfp{CyR1>HRM|w1_aEQlJ*<&v!cDf~w&hYY+l| z#u>@Q9*0*}Q3@Ck+1$AJ_7U^isFKCb%Xp#Yz*Tv4)o3XSFlC~*k`~cbjDZ^kbV;v! zMAq-LOqdX*?xA4q+tyCpVxoFuyyl=s(E(A|tvtN+*>XYs3Lbsh83H2Db? z;l32c`s=94luWVoo_r=O?bl*qu3D%jqezxiHi#HFxA>ujqk)L%X91Yc``< zcBpvrgm}E+j$DXJ1sEyvd$CBhI)6(1HF0!z!p|mum=_@Yt5|W!INkrnMe?^`-q-+M z-yHr%^>9QLSsu1CC5Fo=qVBCnwDdDxA$v7+1_haNm;j8j+gEw3%-Faax zo}2WYe*s%f2`~>!P5y-VnM?d(o8!HN{_HyEKR+DozYMuMw)6m~2$Z$3oGnvP*U%tk zKyAWET|GEm!E6sJ^=rG-o%v96KXw30tvR$7O97-o^Y zC6&F5**ky1`|C_~Ak*ap7=n-iY0dr|eE|c)009UurNCscOMorUzyDwcHeGPn$%o6r zR~CZ~Zk?|OzZMzwmWa3|+S$G0mQ_wQ9X!{@2(!+0?54Md-}Y{4qcpea5GJ|EkKDqW z&Ms14MH)BjC-`|q+R7%c!k?umE!U!AIR3r%6tEc%+GmR&o~Z1l(dF9qxqRtfz*WN_ zZLr)^Y}Tco)<`5hbV?fYDA5JO<0x>#zWbwGOVE625$0$njQ3f)bFk{7{DFwU)`N>% z?;>INh^BEfLEN`JDJO*pX`g zJQ~q8$lyu~*){zbZrLu-mN;1_W6bS8?(|l^5HdQ8j39gK3j&$*5ZAZ}!bLbFwJ}bm z7M>+(Qo|Dz3WzSy-=ow31rdWQq4vpGluGyK@{2yp!?p5EDK%fW86^Sj!;X6lHZ?;X z#1o2RAg<0D5YB=Ds$$!^>3 zMP+YW>Exsso}}|Sg&F077sKGtqQM94k#%Z$hFFF@j?&uYn*aTjLHs}4K8UgyrTk}y zkrlMZy(Uk&!6{4{(1rdWLhc{C?y%YOb>XTdv@`7Gp3e{qE!48T9JL|^j`wPnHfSBT zT6SyNN&Ub#xb1?;kLuezrPGwfN~f$Q@s)eI4B*iP|_Oz{~c_ug<$-UUnuc_ zj?(7iXFnrNAu;>S`5GoHsdqk!2e)N+=J=@M;T0cV?=BSv@gqJg9c-?@ZQ;Dg)OZy8 z@N(GFQci^!tjW8UO8FC7JnMnrv$(?QFsPIt0FcdnFOj1!r%WmsQw}~pcDEmR*05^B z0Yw4OC_?u-g5OmYHO6mi^tjLMpTaNMqJk;c_1lvsKjXf2f1Q#$u$D3X`*RRx;VRR_ zz?QqS`#b`ACA4e5V^A5G@_Bspf5BJ>c*p_+N|Bj+cLZ9}?EKU0ms+;PJY_cCP4+!r zNat2P&HN{%OUaLKysF*9&S5l<_h1zl8@4ZPw}fI(t(XI+X8aP-8v1_IjAxnGW{-9P z1c7@((~YxzKf$9$@5&$GBCy`axzY=4pAlsGyg20+H?E&3e^z;UE|K41vu9kBKhH+O z{v3(AyOLaxYqS3*tr=Is?DRF>BbrmLCf#X_Gk{YoVM2~>kgD_vw4I4pW&SNH-ECcB zbAg{w8HiLoV(66PG;7H)uTn2Q`jt$Oam+|1A^HiRMX0%HaE-@Bp)c4zjFM(ktqtY0 z`t082_3+V#?ycoNOrM1{4PnLV>>A{4+qhkArP~OEfc@@Ks0Od9TPYW6SJ}-LbH{-WfmtWtiyU6U= z3?zQh{#4FJWhbEu`7t%(tc`Fm3Ld2n1#P`TdbY8ChDEevjOI$XQ%wCa_u<6!o2IB$ zP$}2TFA^mQu2FWwVi!7@nbmLJv$lWBloMnDR!+h;3<=?frIS~&YV2<=Q=BHnTEBWT z{&L=K$kuaw?J#=KVeN9xcC2*a<|jT5q68b22cHiIp8)|^CvA+hS!(=7$ezv~=1U=j z{P6L2C_%D&zb3fHq|`CCf<99BaQIV>|_ zE3>S4lTVi`yVvBZYa2dv`2JBn9*QJwbq{$6@bu57@E_luDiiqm(>CdYxI(DNieKAT zf`2@2USUfg2o*POMpvwSC;9+pu1s9hC`#4AZj=K!-gVeB`O~_QPVI*_`{Iw)^o{p& zyO^nGYpez&*>A>>K8BV`_lrPm*w|-qej195fUD?-@3J`02eZtnmTGaF1r5J`LZB^e zjuU@0+XxrCr#jtG-JX`9=6Izbs$9?t!-pv$Sz~jTZw;@qnnY1wL;VH~0@rd<-kH?6 z_^8tuH!$Ds;B(`i$X><^!*lbb?2Ptm&h^+do4TPZyHy#jzNbDXL-+O={sAWD(_6JXDb(X727d~j>)t`lW6^h!67h)b*lv2t`19`Ita`# zAvM$qZ*v+yHbrQ`)|TM@*edFgxhXTSM)~gz^Vq}j?}3PBO+gDZO!?xWdnHR`5H(Po zP|s|PzO737#yWD1VHJX6HsJ+bAqwjRGMjmY&saCbpavW+O9<{GHV%9V&7JbQ{J}@x z@L<$y>Iqz5sxFFML#;f4S+t6+s{RbP-3|qPdCdzBA8mZg&H)wTBRp<}<9`ijjY#}E zZI3aBUjK&wCUkIUS1Kge&dfP2vFzw$POEVHJTHo7+knzW0a$uLW;8-8%2=FVig*-R zE}!B%W3F9Pb})X-(?WzSJQ-)+@NDtfsX_I%ys$8OS<}&3$>Tma2-_4}sz`TOl#ZeL?Q-C;`f}i^>f|kW&2!?}HVhUD z(AJ=9h*vejI#x42(-Au&1KzLCkY4lYO-Qo#aKK`w``e zXMo>TWcO}|YHv@WTkfQ7w-8l)2H6x9_&N|njlvJe$F*N7BUzw2J`Y^bDPzr9qOky0D=Wx_$DP?^?eY#GVoGCzHR>Dup`QRaLf9|i$g{)i0pZ)6x?Ik+-F-$$c zY$LP3z_Qq^)~n{{Zcsqhveu$4;*iJn_;Uxg2Opz{Gp0|1v1BAqoR~S>?i#w3TFaD1 zGm{g+ZmerAe^@TwT#oE#IVwqKHhxQ7mG*=GWrs6ma4`m&9O%#x#^ZYtV@?I;RK`$6 zoP{63al<`>rLF=ejc=LKb|RNeO^gUgEMxQqgJAhs-!s7Fk8+}V67px#?eKyol<-q| zgeWXO#q2i-&nLLCfw+5vlZKdacy+DUCRP8=KFvp9Mff6x#F&)r9X`Y!^N@463dY~< zl&}paJh&7c!nKbogHl_8r4@-EbFg6V61{OBos?91X3y{$6g)o*WjcOC5jah_jdLEX zK$`HuDE)ec8fw37GM)U01TM;#$6<2^^qCd-8|@kwkA5l(JZrmC$tepR)l4+*kjF=+ z3Pi5#F=`blTK^2bPGNl{5iR2UC37VaX=Vm@e3F+Q4c`lQF)M^ck@x*>|IfXSlHVP3 zVXsG7)?d~+FK1TClj{CWzb?>h4$Mjur`QY2?pMY#*e4__bQmwtmxxEEvtbCw8%@V9 zL{k-Fv$Vm(_ zH5G?|W~=hwrN+1Ahzenk<7JtZ{77#WVwp~j43d_R_uLfv={NvQs1Z9*dMXzA zOAg;;);T{?AKJ+#x%9OtBP{V+Yl--U))}5aG9l}QJ0#tX_)HG<|5OspJNijQ(P#Ar z)R<+T{X8#szZ(`4#DL!K3OJTE{De1xYHwQLP_mv-fQvkVb*HKB5OPm|4L+b#Gid=m z;F%|&;Dl)jwF>%h8Rt)gSw21xT;<=;q=HHqSVfu_IN^b($XVJ8_d}rIr;!x?H)Rw4 zr4K5oB_#1TclTLBeTbk_IG}NRoEX3|E)!hFB||K!uuDcnZi)LlJ-2(#Dg$NA+rLd- zjGbHF17-ryLcjFqKz60Q!%G=k=M8q_0XCv>yn4aZwfBsORyjvC&5J{Fz!>-Yls4dr z+vGAAnEv_sQgGRXPB&VGVkzm1frrjAO#)Vd#v%jB!T zhl4h@)=gPc;%-dtA-FH_!kHqWF{4{49(wxAP3yYtL_8Od4TF|HLLgrc>@xl+5~S4I zN~g3(2D#7-q;_%cVBxz(Z$6)EsNt7f#=Lq}o)Hhw?udDyAgS9NxzavXzpPR`Y0i)2 zL3diqDfnrGkq^^dXG-1FO~LuORw=uDm~!b!tIZtaw#=z97i0z8Vwu2HHG*6OAdw|6 zMRcq>tqEsYn8mf6jAulvy8*0L6GbnR-x7jO;f*P!eg1b3mW%-SJokh;i+d=%$Kq`FyBq-ASc_z)I)Nn`Tv>C{da7)t@WAR>hXvcf}M8J zxR{)^-SmCSN?;a8D49<#kP7|=9i~_GRtN73(-1Y@dVuXP%C4*St13Z-20rOi?^P7e zKVdJ2Zr39pz}l=+@e#ped=GHH2F`knFWml8&M6!*ueR-8M_8M$PM$DM_<748$(}VR zW8}x9C-^DW&~w~}@f~!+oD8uY*auH_5VpHqlkQ_+U1y|$ryEF1Aj``ut9IqOZ-Z<5 zM*{rJmEpPDcpTKh{o+a)lGIUX+jG#&$h76l*vGrg9yF@0m_QUEr3Rg*<;mlD&kM!h z-9Z(g_(3(1sU*oAeix-7*3agGK{zZp29Vph}ADk-ZU(kgF(O~OMq9hQcZTX%9Y zu$~-#pqZ$$eV&)8W>6v+%b?NH6T)8`M3nbEyl7`vkFj#uQ7+Xti>jRK$BMH+DA_=N zm2aJK>wBx)EN~(!tWRc|{E+|S%j$IYF1)UTY=q=;()%2@{72FEcLK!x!%qi!`*CUQ<)fJ^OpnVC+-~uax}RH4rK>`Ua2{!Gv|J+HzFB0Ko(Q zrnhCOE!lV5c8wA*X79peC`j2ur>i>>6um&DB=q7*96qr$4#~Yn&1pvqyn-)}C{@$W zM3^YUtvRj>jE*TUxJhnkfy!b38rkuE3X^7IB}_$Q>Tx4`T|anBVj{j)0boMlIrsTJ zUJZ>kM)jAQ60_bVsa(_pmu7VOpcZZ6{REmp=fj-&<_qW>(4?utG|;fMglpe8Cx$^3Nlf<$@L^f+DQmFtKwqx+Jg*Yr2$Al2l&l^cY&Ys zqoT-$hiJT#fHw1nd`5D4)Fj`R)Am z(3R36CR@^=D~5Ygm=axm;n1NOS}sLlJtoUow@Js3NRz2YEbI9@D;6UDj{iL(~POIC8H`EKEZi_r-}LtmW`8KWj51+h#u0 zB%MoT`%(JddUB@;6#a6)iJz#K`jeO;7>WQr;M%{`Ww+s+3_$R~0mQRI!cztLlWmuN z_fslf%ZD~N+QTWV2C5@##ogwROn9Cy@+)dI{#suEG25UrBFLIHQ6gP$mrnH+1Bk7D zzq*fI865uu?Q!LeNh z4nJJK%`}W%{?@#3$aXGz-pfb|A%V<+&AUTNW(sKFqSpq}N2r+Umq9noCA zL$DXG)96rEzd5ph59qyW4R*y zzsrw3=Z8usS>;FAu=V^bDzjLLlCOHTe~*)hILTt@`F`s)^oR#sgL8#O z+L@X!h*$+!ZH63j3&R@TiA5xK6K3C!LnqD)4#hof3EKw{yYv{fGD@~uMc8|JrwbR) zJevy+?Zz$|u3|@zEUMHgovv>Y6ORHMWi)H6Gr!`w7@-Pzw8+?~RgOEk7nL@+DQw&O z4^3PRg?}cNnk@5Z%~}BR*OH>+AiNB*X!Ln}uzm1NB<`~yiNGBSlI}F`>o#y}H;HNP zH)nl6&b;iOOJ^?RwD?v!UnCP{BnXp*_XK>{#m_u*eBdOV0+VY5V18Y3Go{x&Sy@+2 zF64qfo9@9FG}_$`FPyP0!*oHjN)!IeD&tso2htkF(TqP*O1cV?gRfe@bFO1o`o$sw zt+$_){%T8-{?E_S`^O!(A8Xf}kNKYgbI?GJ3>tV+twdhao@ub*V@eAg)M9!PyFEK| zXQiNK-evhbUn|3jqb1FJ-0#hkJp+w08X}`gAtd$`y^t=tTJpL3mCvELrYVHP6>ra;4 z{wJufBY3=8eV2TzjkwxwNjJrq4R=tCy0*5(M`g~GbX5im%yqDE6(#9HP)X7Kdu-mZ zQ<}(RMXX);!w!RDm#n@Yzs8(7?i1wM{C95fQYy4Qzc?-kKCQL%w z*M-(BQGO}|>)6}=OP6bboaub*BZEg#l3zR>ifqu;geag9z^|f{3V{3Db+F3>Gvn)Tl3s&=?d9h>QuYe~2;t#acObj^4wu{|#QV}@X-HlOKs!+3G-NuQsq z+H%m12t5}9lOjIC9Az#ce*myH9w<7QeaIkbWLWF0*k0xcZS1lS@MZbLEpFVF9i#hr z$tso{@*<+jb%Pv|2qwM$w=UzTsw;o;HPtA07(Iq+_hpoK!m9-kP^ngq@HC&8DL*O8 zi!L}E-|%IX{)HHBHwML%wXnF4R*iCz)BR*xHZk7*9blT68|qWnlX_$6V{J4(h{!kP zt|uV){g;C6qGPRu?EXF8>hvFhM4^RhLKwA|6>B zax16oLGM2QDFdNg}KojW|@quc0 z$T9ALbXb6?(L|YADw@rgf9UsJPC}xt`ddAsh7yIcy@aqTITk;6UPqRT7<6FJ-{hP( z&fL()J<~C`VeY7}B2KDk6e48zDZwaQ+0P^;JCRPV6`>y_B$iB^yVFz7^~lQqvqNot zrUVJ}1Dh;VwzL4v+!v;GG=Q-TGX7;XkSXIQ?(Kip2rE3$SC57;Hz(qitaB zw&y1H;oP)&9Z%ZHE@CLDEkX@v)~g%6JYf?OIAwL^yO~1s+{=f&HAxB%)d7J2G8vv8_$boUzoUdbRyQa@m z_Uo|R_;>Aa_s6_$@ERh~^Ivg-0A@L@y+V&Au8S&En#wKqcPsKf> z@aML^NTNubM!LeW4r@a?iihGd5sd$j+)P%1z%?2i!-`(JxeYks-#wX4>xyk($i(Za zb4DZ?ds`R`4p@+YG2{qWAj1rB=ZR5Yw%7zdq+A@{@0%eJkGA=`Pb7__tug#OhUWs1 z(Gb8M9I%mWkZc_9QX*3$#lb{npDWd}v6%^vxTXiB-2HuO1a5-EO^PwV&SL{kcsmmE ze!42&aFuy9;&rwt3Sq|@k=|ee_*JJEwM&?i0j#gbkbm0%(z9XCr-TG9EV~_9*r+)@ zKI4H}2}3i!VL!vL?K6MOkjfF$=b=Y{Juh5QG6?v-HO4FC5Zx z!^-2vMn@LinH}syXR`jFQ1TXkmbs3mIW)ZmoigZjw|dLWFW%}<0qcI;rhES8_A}n? z!2_Pd76dNvZpH@B0G~`e&#`+>qe_ekMv3GVvHW4Q)}bFHe@UXbZEit)@D(zg_S@Kh zogp(^#Dtn$<-)l?kkuQS%&O$!tZL0e*WNmHXaK?aBN6O*CD}{RZS6=^!F`naY++|t z(FQo`(%EJIW?L6EU={_c4sb%d59#x%Sb6J54(;EB?}UWiBNZsqa*}7hTX!pn&s{m{ zBpm1U!rMoL#KH6pl>_{61fV%ZxUKCWNpW~Yjw}HWm=*2Esf=|0(% zcsXdhWEN?pC?`5Md5Jk=q}cdz^GqiJSo=Y+z-nDVp`tVTe6M>an-ChPMRDWom#cDGEw~Y@}BCJu*j zlKo(yVnA3J8VUc@<=um%h!PNJrF^Nk=g2G08|R`$23yHS2I*%bp68;2LR_xyezYZa z;{QgHAC^0Q2NmmNrx79p-YxilT_wplo&0VwDm|HW!hq1ua!g39G zpM^sKB;-T7f_u-%n9a-zHnjVgj9Mwe3jCsAJf}6Na`0^TbH|%Dl2iQ4Je5D4Gf~-{ zLN$h1+m{>bqDGEhoFo@toym!(8MHE3+$FBu1k}T1yLSJh+MOniw@gY|FcIJQ$ zZ0K;vy_Z|$F8I&v%y%!I>wRI@FIlf54}sPtXUT+U$PCpH9Hil?!@4mHE}N^gcYOiX zC@8=b2BtvWUkvkNFj{Qn$7f;xB#^yMjx-Qjn9K?G4vhH8-u5JnKTM~%co zfl_^dT8YVjwu@L~pkK5rMBH>sQKOEd2e1zV@qD1g-O-MNaSq_;?tu*htQ{Ynyg|)L z99ezzc2&m$8|=D`5u%ywVU*J)tfYZMwX!n`rwk-lzW@3s+z~E>00zB8{M&OUALaT1 zjjn;s{Wjq63>cQoQW0~)Yqz<>j>P;k>-?B-?A+DYHr7=%*T+>_x}IzN z{!PN=5^3N8u<%BEa7Q2d;H*@Vv?gy-rwAx(5lZL8Rg%M0WPy`)FejDwu`E@jU=N%X zT#w8keM;a`5oX~o_Ry0Gw?i-x_n;efndh;Sr!aqUXOzCG(V)%9yuQNl{*J(;t1I>? zkUbp?7be{HNDFM8Z;4?9T8O7=0Tvi=uxPtAk`lnQ5e!f_42YW(MI4jA5*m|$wHIc> z!@>g^8GKrbvf^vkfiH8mZ``+~OR=gG`f`~|^&e>0>R6tUdrnmvvWGvmSX8j#uFsVs z+~v0+K&BKZD_p2aGkkCg63<$8=SyKXobsXcU+qZ1{5Y7b<#oaZ4iJ2x53|&{8{ssjC zc*%J9xH=m+y`^KBpRlyQ-MOD!R%7T3d(BrrUyNLwtmIva{lC;Ze_( z(I?-WJI61sA1-XX9)L=)d%Js>MjXZJ?@&F{WWa}yUTMuU0+5Fs@Ru%wN6MbCmD5NM z_h$4;FHLJS%k@((KzcS3wz;&g>g$IHtZ(kX_Z7UI^-lJ%7|4>F{iaK5r`ZMhNa>TC zY1F1uS+}`=hN!nJr{Va*3RcZ$F?nJWma>*I>QA@UF|f8gTa>7W#>A$ssw}{dGe9qZ za_7!7($U{V!xjK|MQjBjv2J)~+3Iy}e#_@FVGOjc7+&!EOEr`Sd^pwS0GOBb9Ww$H z@QM7(MQgr^M`%ap?7TGmPWESp+z3^l7DiiGpl{=kW8**nbv~k92QiY$LZc<>uLjTuFa7ehA5>ACzRP7am zkHOIK#7>Ou!6<~s^;KH_9LVkL@OftJZap~8|JR!}=5~&=QkG~Bu|d&T-1GY6mh>UA zZMDs@G}tE3!D}oWs+YpEPJKjC8MkQ~rq>7!zE|?_W)TDfmxwo7u1QL1Y0&EoWjL32 z$@3q`oO`9KTm|zYvc%7P;FJ7U^3HG~F>=HHJ6)kGqj@D^=x%5OPBC2UT58j3olpRx zDlmje-~w?e&%O_Twg!ab1z1FC$+rO$ef-&{eL`0bKnzk_?=#w!7{#AQ3{3fCr}xho zJsenRXaq^dZ7gg%QM&u7j|EZ4b}DDE-d4~+pO z7atlpQFn81fS-PUzL-mLpn2u7B2thIg$tW}v9}rp2Emp^FYeLQz}hP}Va@-=faOXU zxKTnu9~6qdM30ootOAmNbyY#j4ZcpeRtz?1=RI%4_&ffWDb?hO&HBo@RJ@q+VC!K~ z7>d?)6-GJN9O5T8%Ck5-PqAZn^5*T|GE9yLED*fR|#3eJZt99=9>w2rCXdt~GM#Ad~IVH%&9I_r(ho zwn=XF-emK-aMU9U_q#-C=2{qPMJP!X|?X;@^1LiTJHuG}6a-rsXb#WBR0^6q6w zpD_|G%mxO!Tl}o0LkN*fXOKs9cofYridGigFI<$cUN+HQuvJIqApLW^i1Gk?wuiAx zuS5rG47BZCTG;tCCx{h@O^T|y1Lv=Aq~{^&!sjl4t>|@xlTwftQZ~sL8)4QfVKZm3 zRx{b%XyQUwh`$jIMxn2kO)$<}Gvu;S#Y`x7_`#-{(WwG3U*3@~fjza#Uj>3OzvfDD ztej3jriJqTehbh}Y8K4T>F|U`meXiw-K`u1@h!G~jN=7&Q{+jL|8y$3ak80?kfKzY_|=`yyI6>^o~TU5p;UqY;51 zM1*UYdkVVV=>q5YfZpC?R9fwFUKFW}!*M}LkrEHn5@gYCP{v}GOv2ocN}cmKl!dtK6I)3@dyUL$IlvU`Kf75^aA4R} zOQQr{()_T9Qien+B@_(>T7h} zh%T3048cH)fH7RxUAiU5+%4auLr(i{Z@uP0;hbE?^NT#|k6);Juec(x-jbRq-A2HEZhzHm?C* z=-UT#AdYQe7RL~|7z&U^(tA<+RSo}|4yd7!^0Dr$Siw@caB|P=AiL+10<~=qL(W!K zLw8Ea*A}dhv6~9LWGV4hSY+{(bj>MS>4QE;{&q-PvGatxkK@#g(FcZhWZ${Z$=l@@ z+oG-H2qRuY?25xf|8mAuXMR-vjgxeLz<^*QrN1pg5aKA3Zld7R__yV4Zrb@}imz!) zup&QtS?2U4gXv7su41DRu&6-9P_^asU1J3?4LkK*;B9psJ$1e$4Cm2by?Sk|H9F|z z-#(%Jv01ry7=KRvL0|vBrjHr`)bAquRPRMlW@JCbIxW>)SNyx=%>NLu3(NE^pp#7wx_Njj_DxK$@rv;L|d-X|GZ zcpF_W5%9Lpbuy=r*=~faFSuTOEV|HHFe{P5r=9y&>0BSV6midG^0p^A{An%zs1M># zr-jr2;?}&|>@c&&it%ENijUcIxSgfrWsPn!H(3*&a8VccnjA!)Pf2vtM4OK=M89Vo zV)Nh$UeZx(P;T*H^!yg|yt?eCs7>gChTQ$-h{i71~dMDOB$mk4|9qF<$s|8Sy04dtLZ z*P@@Lgp zeizN~HcG%|c=8!N&PqL5@T`RJ4koign{mM+M|*i+?BVTay$E`yOX)2$5)L*FjH<{~ zW5aWGpM$%dFgnXoeXmxuOd}Q68=9j+YaGO~irsk4KnMbjyYR1xKuv9#fGA1w7A#1G zR}4vU!BANFYXLeupGL}A*$-Nr;GBs4wCWUbF-9&4& z4raAZvn-ZE#WXs`c^R~Rr-+hJv_*Mm3AE+fn#;Nn_=c67xI?wsEVzP4KtIL#ZJfre zp4YpnEfCc#E7sm^I2zcx>>YND8k{`kN5|Rm^gR>b?As;(-gGli=0aM(V1E?kvf_WptXb?F`NuJH&OL_DBQVS}kV(6^q~$&e<^V-6E*p zH_Rsb9!$>4@qCEs>e^n*GxH88u~pX~0>>utrd`r^G(*oz(gL`)FPYO*ZedG5PCh_X zc?w7*#gD;UBu1V;A`}zKs-JN`rVpr;2=s#i7uXith)MA&$9YfF0m+iP8jp!4%@5nf zV`$Y|5Yt3`DWO9Fw$?d7%X!v6sgD#^29n4n(sH}n{*yzfiO(Ph(sH^=@CBl^M*=Pc zx_F8M{yralioNm;SZDRR+c}G>osIEKxJ%aoVs~Z06pN-^)VxT-j^<3zz0#-MpS^l0 z`EhaC$x(r_+oOE!g7SZ2k8yvPq&Q%;{O&YyK)P2Y9qzRVZTdHi`Hi?(Q^g*973k)p z^FUJuGBmu1NB0KFoX7OE6WHYjRaWfZ*<=`Q!+FTc!p6I+6kZ1Q8;L=TbDRZ#B;IDW$d83sfJ)-ilX#i1cD zxuxJ^6JND0eZVax0rf^M2RGrIAC9FDGi=9KYgB>IJvSD z<6XqUUa#ZNi;WzCHM(BZm$7i~7*C*pK31eor9Nza(=5pBz=7AF zsy!xr=fikSf;k}bG?))4|7W-vv)voDEhJMN2Cq_&wsbD|&a2^I*o#`~K&k?TxS+@8 zxS>%Tj6vj$Tlj}ugB=#P$YyeWU-~OT6*UzARCzzo2e5zzo7hO!*@Ck1yl^vRV=`y_ z-n5f9#Zgi#1scvO8Qak*($P;akHtS0cXy10{q6GTHbDLWh(>)GOL0G2`wWoz0ciwC zm!dr<2WMHAgOpfG0$lU8+_>HtY+KED@2a2U4Y5P;9)4U`~ z^|UN;ZnAqToR+yz>ki>2WS;nTOF|YcQ~NExJQ3}oo%{^CcXrFQQ!!*_7aq8m1yHS_ zC32P81DSLJO9dF$9J%P~WWssuVnuxIOin9px*T!6!2{7;@yr|P0XjUotjk-+Pz-1lV*#%1FwGN^^r4kip zsu1&f>oB_hZMRQjec^$^X9s}+xtx0-e;I#M$%kq7*Uew%!kAf_B#=gOacj-rVXZe& zgfy;%?*vCeU@_D{Nx--yl0E7)%STEB!7}lp-*DQkjrm|`lQ6w^E~Ant;2?9LfmM&h zuq%T(k8dh#OHlQr9g{!+f;^q^>;DX3i=H!r1c33Cw!S^fq=cc6-b>x2)Ne;pw>Y_y zir1^}03L>*SmRxm6YtJoH?A7IM%2RwS?s?KkmSopPDw4q95G&EI@m6vy3>e?3%TWQ zP(48B(SlPjgbmA?C4?YxuT%-?y4606Q2njKGiz-owXc$pV?u_dBEv1-NUkcUMF3(f z&17u1sz4YunDHM!vh3ynI{QM2V`Z*=Cn`4EBeTQ^qNLPM2Xk-Wyj37pS^7U~&M!t# zIdBF+Po!NL{i0<$h(YjJebINP#Z!%GTHK5Ase7O0*{|tub~+O2-6}MXi&RR%sxJlh zY0#o-J^iBZx(Vev>DtiWacasAyOOHv_H;c$wG3yabdz2i=5@BG7k_@KW-hx5fSa6h zfw|6QqFu3}`$Z=7j^v7Ht?W7E4n<3~a--TpYV~cv)u@5>vKFTL>?9)fD^#YOCXYJj zAY3GO>yB+iGug{bx6Zmq<@QzcGx3&uFzyn^4k6WJnNO4l4;1Qhz|qz2e@tY!%?G0; z=3Cc`J6zw%*5R_H<~QAa&&$@s2@P8lO1|iwGtf6Cpr@q&pB7cUigurKa=if(Ws{X} z*Jx^T%u>NE9k^E+t zEv-52LZ(yPYd$g^(RSou+clnjyz}&M=u{_+ePD2~hWeOranRkGc1Zwc0i2^SDJx<+ zetRWAIdjc3(x}vz1=Qx_XXqF`U~Al|=vUENb)Dixiv2QH$&QVb^VEn5bEmHeJ~a)D zXBf{dL}8&XnDUz7jKQ`+U7yK0xLG+k=wp!+$sY98Qc znXY~OPQZ>xdi0q=u65<2Ez;yvycu!!cf`dVMyw+X_+z?F!G`9|s_}IEku16geW}xq z>84?d^@MM7%eu<&u`bNk9p)yGxV}ck%MV0+6Z(z~HzL4EvViXZ_B4nUPtH0u9w$a- z+w!&6^nJuZrF?pTC?A{tBhrRom0c<;ULKy6Y5`fw<5R)~LPgJN&X{5$+P2lW*4E6g zTLCMbB9nc*+RVSykQD4W3uX>HVTvTsEP@oXFxifteV!7EVr(cL=MHzQ>zA|R=8p!i z_tK=dn({|IQ(jpz%z=`x)u^i$N){K$)Teulqovf&hMs)5>fdUy%{EC-MQxopGKy!5h``0?C;)` zd!jw7#?+;`5Tzudzy8Al8Z>GYqY zyH&o&T+t3-cUtdBbWLZ*3Nko)&qMCxfW>1ujx;?T(yg&Li)saRT;dumu7UBX6cAx$ zI&k?!^Yn765XWpf!l}`b>kp`_YiPrgKzqxqm|TBM2bO9MgqwHE*cmsL!7Cl#kyq$@ zJM}u%XQy9xpb<#`>ir81B=!j#Z75f;F(BHDB^g-;&|t6s4~!^i@Kd1FLKlBcklm-E zYtF>=;L@YTHUI^d(9J7KW0$(?H;q$T$7bSOi4?jEDgcDbXZOXGM zr@}L|^mjtBDo~r8_p$$Uw}1`1BG+eVwEF-4&On4zSfLH&KAKw~QhnBOu5N_pao&}S zJKg5#t$H8K(d^Tg9%Jx2wl~so+~!#EoiAvRa#s_jw1?5j=cu${q^*lH->nUqN7|hK zj``3yHfb!D;!Un?jB$F6=<_B`bpe`Xj_(UDPbONcIDFitK3IWU@rqLwcJ7D)QxS%&=WcRX|&| z90ZQAOG5Qiti@UI;=NIUfsdbz716xJxu;a!HApObbwbP)Ua6)Jv^lJT#y%S9k{EHa zz?SmB1--6IvM;)ymFuI|XWs!_J(BrzV05ZRQdHjzzE01!4%&+dHHqnJn*4|I{2~w@hnYu1CRi1!V8Lw9)qkOiQS|*N zGhX6Uo&-TD-ep`tq4}qhK#S+}Tr4xSLDg8O#-8Hd4K2xzXLaQS?141X zNXev`xXHXQ);=a=GRkyr3GU)m)4#uCWXKxR>RIUPiv7SqW8Zq%zyU(ZQr|nFq~C)k zE3aHUc694=nbZO2364ns{NF#lD1vZfzKP^O_6Wx=X=ST_N7AKS zY$3tCb^Qd@dB&-7+K?6N^r*|75(MZEAi4@@fQLMo#LdRraNnn10NbNfs zu5;1OhDe!pvB>J^f<7~4K$jG*S($y|rFu5YmH=K;9J0E8xPueQCdth?Ly?&ZT4vpSOgSTH0P_1Vf4-dRsFy-I?8 zx2CsI4R#BY>xKYT@NkkY!k;pl7_|E#Bxl9Xwiy-nE)Yyoz!E^gT2{f6fE8wm7um-ig|7yGcIJ90FUZmOL4g4v<_T^y5*um&&E(0q*Pk;Rp@4 z;AiuUU`+gzGyro6>{O_d+9`AT`RVb!wp#*HEVFGEc~a58b%y=fb+?~#NJdmF+F9mB zV^)(JDI~aOOoLHQMqsEwrkucWnyE=it9q2Gdc2&4lBA6=1i@*TQll9kQyASkisps^ zITZ~{LH}DE4ZB$#;wbYG(_=9eNe9xtgmU!%A?+=L;tJk%-@$@=a7%D^*O1^E+%0%; zcTbSu?he7-A-KB)celZ9aPG=~-(Ba{`EvKE^J!I4!}QGbTHUWa{d*K5p$1=3b!>q{ zchbvjYup-)(3|?x z&Y^vj%G!kh_cSv86&{qu05=DFZFAC24~UcC-zri^u_R({Mz8$jZ1vkL%Hn`D@0P6; zhYJm;pWZ#`7YYrcHrFE}q$#Y8}$0x}qq?f(maPyNktK%ZVVy)ZVy zKU79S!E#gCwx6+kZW;VOOvlgyeInWPLjQ-Hg(ZY(4prHG>);m;V$sbT)efy}tNmG- zmX8Ohr6isY$*04&*PWzShrOygBi8B8g#LkT&AJvQ6{np5FKJ|v1p^R95HjAY^s9*5 zncdJERA}!i4@Ubr6Vpwu2gW?0okRW9&CM1jG(3v%uwlapwNV0MMM=vLo*|b1{}TlV84Jo+5_0F$NIK2fw#~&O`tu z9`B)tJk13#(wo|q{McYDr2<~gIu)klSrn#QG>QQDIffbc2yviR*M^95(eqo=*=yc2 znSUw@Ug+D_WZ1sJ(&r}qh0o`n*JEwdJQ-cSPL97wg`C3HU#tKsH^awD!@G;!x6bxD zsM`G2vuA>U8aA{6QUdfzd`hTV6AIIxG@3{gm1eFySX7coFy#x&CDudu{VGaZO!&F!sa3uNY39 z2pz1i__P_gb^4OItq(r?mCpY;^R0Ti^8vorWXnO;Q<1iz-5Uo2qb7d!(Cf=&=<8kf zzQxmsxbqitFRpv}Iryg$BC)4z0mZV7710GfBtUKG?;5<~ZVPT1KWNP6%E`2#4zQ;57lIv3(%gnVIQTq++>7ETbadbyz@ONRm5dy}!tTyJ1T)UT8l_c%a_zMzF zROQB%Lmie&ndjD1SMgmRf57Fpdg@49hD^mT%hO1rs!ZahO_GeGDmypXCdk7qI7``2 zxY49UKU+~V#*Pll=S=5j2-$90gRJ@Ip)6T6!pt@Ln8uE8#z2y{e}`QDjhY%R^CZQf zCIlRShlB}0YroB!{Zgd%DI*2p~?-u z&qJ()@I5CUAYa7g*I>*weTK+l4`^rRCaY1DdPL}R6W%@ZsC|)bnp)mCoFUqePiZ!2 ze8}Ov5a0&a3chv0)q`cPW&|chE*v~lf-N8h-oKbRRt|qDPNU&!yXgpA%)n-Th4c(^ z-we4dd7kU!yBXYd6V1LwPlugf$k#?yzGXaOuSx=sWLMxIXT-dACg2K|SDpZ$p!WM! z_*uSDYGP`1vP8rgMz4}F)&JmZp;BLq*p1zIt;>~e=d)=n%6j@YL}8Z&c$wAWlZpB5 z(()pg+;lEyCot6&-@@9nq(K3jgBNvk?>IZRX(I#pw#~azUiQeb{bS<&L$5JwWtp0m zzX)eKN$oL(;|aX#!_IkF^IyHutO~oAbvwSn=p?4y#B;$?QtKJA(wi?DLwmBeN^f{a zZFGY=7XOx)-$t|Z#|$Fg673y7ncpHwwcDiIz0ct0?1_NDmoWq|T$>j!C7jn&7t_j_ zzx}H+Q1~>2ypgA9+WfTv&J)?^uG!;!%eVC}&MueVr|i92T1^wZX~VMbO@bc|C&Sgs zVD~55NRf3@|Gbs7HKtF}#u~X@Q}1lQ)O&^^FuIDDpPWSb-s3uTB2>T9czi%IbWmG>)WG{vKDqftkqLf7 z7YQMwYF#7NsStiPyn`98<1Q? z3~I}2!i8V)v}w7PwdL#nSd3?fHs8E{)$zoH)YMD`SvmBjPn&9NLv&hCl3xZYj<>p7 zMXwDXs}ubpgc_wngDR3VsxOPeXCGVj@5-#6ZZJAoO%^1jAGkM|$SuR#u9Dwy$pp;Q z-VLgw%Xy!M%)z{`lBm`Xhm8nD&TdN=wuNgh5b-mD0ak|;5Du>YQ{m*ewneLh`u0_h z?QF6S>yjG2q>ixk51&@5>O?c$ByEgL5JT@(+v4$~(uYvXq-2-ab&mzqdYy9KI=x(` zo2!K$+@j{MOKECga>5$;Fq2i`6MS+Lx^;hEy4zgv9d4^&-Tfn!*ofVaYFl+GoqKiN z7M4fmCk(jlfAHS>xgAHKE(=3?4hI2?Zk; z-Xeegw-x?$fJcr(&^LNCO6{u9Bn7LZhFB99Ih^sx2?paZfrZgv9Lf_dV0@vI?aNs> zwWcBw=)%Dc3*y9@;FC%{bX#bBb?=+ z_%3iF+i$ZOQ}o*RcHK1lZ(e>yhs!J^MZ1Z4(T+Ltrh8y?qFaWT)gl>oUx)$7^YJc}# zMG*==ySKWkAp=%}ix`4<-aM+on%bXMq=v_9WZ%h4rG*T49oWE!2gkaDRqZ8}yMs1p{His*THLYP+Xu1rHdl?d{g)|3fUUDpmDRn1$-l>8dEm$ z>wXwZ7Wp3kjS5c~WZ#!EBWoogL(`i$>5z9i zSl%SoVjnPd)LQ#QK%hf_{hUCF%D#Uc5JG~-)7Gw++#DazW9L!Y-)c>UKG~a8nO-bX z?+s+pr-?0@bGK`s>z|OD(r#%5pU{OGqvz9Z#R`}^ibn{OBHP}d+#9zs_v!8$Ex`0b zNa9{}fQPl7ir?*gAMZwUXoXBy#y}pQ|6>@;YDNnZ!RTSZQFy2_wBj1qmjxX+I={2Y zFyV2qU=pWPV;A{_+@Y)Q$-Bp^Y#KYYi z3kJn!uaar7skk%x$cMbudW>)O@I1%(ZWd?ZEwjt7d6$YJ8ff%?pdanND4!h&lVVpo z-wR}*tv<0edrtxqPv>9GJ#<9T(=TUsM~|^=ZE;D;ZFw+wEss$}b`Iz0nPR=v-I|BM z!)+m3KX-SJ^%wpLE)QO|oHLX6Lspl8;~Kx^qpKftQUk5TqFY(T{-!&(ot-}xQTkFC zj5!uexO3n`8Ws-h@PT|>L>J-w9`dkC7NaUjI>P#tDe+)fX(D{kOsY9_pp@Q=eS)tX zJXH=lX*-wmxuhYRcf!2y$8#4{I9m_B#;G05$poBT|NR1DX=8m3CoUG?d@`t>Vt|VH z#<(=tp2Pedk}@7Ue+r%OLXlW8|F^nl*G92;v+feuv$k0%(fJ0=SfW`M#B^ux1|?&U zU8vu_!z9l8G>)|tdrn#+Z=to+dXNre$f)m7>d|lJ>ifZ3&3@7j8B$_g)_(yNJ$a8u zQ0g(}&8`?ry6TUX)f{Q&EnT0!z2aIy30fVgS|=E zw?r>Mmr!BkNjG6!=nONsjm%|dxb_qwHmzcSMkYrnw%BN8qgd*YAtAV$5*&r9>lkR~ zelJrrI^*PjCR25%j8K;y zaf52>#dzAceV0$C0ondk>6)+6n^)4VB8d<(=kP&k6WSDcD#|IcCmu~Kq`jEkr&~Xq zScvHG^v`U{^S1{*{_0^;#r+yC7iOhFE}hKV&l?(l=fCft z?|iW(t3O*%M1R!Q{3oE6hX$UHMValPjdv4V{I}wS`69!37P};BWat>TbJHZJsCOvk z>AB-lcPs$M63DH=#gG8=&k$D__t*3URPl09Wl*F6}XA8E&56&CW}Zf7X%pEZc8y|3Uq{+*miKN#{2pBD z;Qa3jN#Jk+Cm+H-oKcPGm5o=|w5%)zUH6TrO0)_fYq$F61SINx-(>rnLAdz`E))F= z!Bt~VoYi4)x`<^X2&DbbqX%_+xj|^wWjy5Y5nRQi^SQF)@w>Kne`3AKIDZ`+XzKgF zyX1qF-Gfi-^GF$mI%8*t*#pw!EoxI56Ubj37RcCPJl-0}awjU$r??h$jTEXkmoV1FNtQ_@et`CP^MkTl_8x#{P-p7L}(r_y!dCwFYtD3m} z(qS8oR7$daSDfhI<@p;2?#}Dj%vW!1kw~~YbUh4373BZ!+KEb4($6_-y^{1sWacw@ zxSYa=9z|bi=Y2deB0vP^PaUvAZX4fhv(1gfi3aF0?X$tR$B38T04i!u*d}+_3bk$i zryZaS)x}_k;j@`(28^DDV4L0?2^|Ua1Yzv5Xj%^g=aDEuAV!q`ew%;Qi~p6~mP;y} z>WT3%cSJ@_q@j+TKs`QxYw$UDsLS=liq7t8sactQx@!MdgYoh2Y7{cJxVlL35R%xT znMhfv7;*5l*aT&ASOI|1@=t~u=3;jIP)mkuswG%+k zhyB5<+wSelSkGQ}r!^3M^fZ}?pO1L!X8*!`yk4SDOqf3$ntqQ$LV_UjHnqN1vtx4J zv{$!M0oG}&8po>RrC2BshI~B{8eSONW8L;F&bCEcU7Nm|>%Bz-fo?(2;7m69N>^Az zmrexzu3g(!j}?4WVTY_NVXrKS1Ox@^mCAnOJJ4Fu-R(&>(;6PDL_M8Nrwcl|k{i!- z>Ft+emsxplTWbNF#Ghr8BC zaZOLq?7c`P$%`}1%sw9Gp;8=cK4&rIJMW%|E`K$2U@E>sokqxh+NHfcX`#FP2nz!3 z6Csqll~1Mo6z@90Y;_yH8)x}mW@8w=bjDg!@c~!g2%aqHXa5PtW3ih}@*jFeunLg- zmg6jT39ZVOxwp6R>{CY}vBqN6h~gT_w!C}{0xzU$3|huj8Of5l$nq^%j(SY;W-9xA z4%hDNtUI@5N=qWX#l-0B_{?%^>re%lyAcAd>1o>V$e+9WHdcK|#^Fpk)rpU%t%ed&HDkWg3?o*Ik zuKqd}8U8Ps2~lmi1zZR%fYSu;oR0C^beBff3M;Dd%lf|W^*2kFNWru`67(K!ZAA_U z*}&1-W9xLwV;2BNGcb56nY=7Kk~NpuM&=~R2iAz0alC=jj388i4N}DB7-^2u0@e3& zzO|y}s592Hy)08%Q#a~)A3{8SCzMl()4VfzMtAYFj9v81-27f!GOLysY5Zh+Ln^1; zDl9kbzS)*W#R(B_IT6YXqBfC_^t^?_<(oPYvWuu-5a>G`^opVS=~PmQW@_1! zWxoPRC?$07OROTwk+3E%8d{9<{BpZb>m(O4nDlIERyNZ!e&AOeX4gcA4`;!ALtrQ4lux=bPy5Y91c*d zEA<{JZMyA6e=krjOZ;zMsCBCXE|>To`_#e@ea=~lmZ2KccyOtHLC_Gbe<|7e^{3QS zob3N(h5pLp&b(OF>7M0@T*3Acky;ntCWNGlFv7~mLFR?*-N(cg1Ga%vk_fNN%-p}n zip)(sAW%IyaakgIZS$h%~2x=dc}Q{ z{N4XxulhfIi4C_I`qTm5hJ>U?paY~q?z%BuVHohM8aF2tjdHv~5k!|yOUxKj_40~N zZ!p{Ww-6LTSu^V5I%PwFj^DxJs*3qeZ(E+xwDuZA^<~nx2(9yey^@y_Fg@f9ZsnYEU|sEo-3mCki0 zbr;hEo{d3G&B2rTy;l1L7%Xt_<{1%wu3(Y5^AdWftQJ{vr^@L^goQ70V6C$Rgs7ocEke(`-+dk36CFXEwJPn44+S+Sb1fQ> zzrSd19HLJ&G#b)~PMDWjuP9MV-B3z>wwKjVUEQOuk9mz%1!EX@nGX~qru#+;XD#Qs z#Hm`J#`qFxX)&{P?V`)Xe0at2@8p!aXX7(Onq*bm-C8w-2RG*W8ll$?GW@=+gkqLh z_zbyyyzQn9V7S9R&7ONw-9sUZBY9Zlkl^eaRNI(LB1dH;EwUA!(5gHjk6b=yXvlgH zP|7a!ZH?Y2>wE36IgX+Ef-ZTmEPTAgQ8_xg9vxdz;|UK^g@&DHhzd>M)XeQtg7|;u z9JF{AYk*?WItVis?`qARoZ8)DBP6cRRzkhZ!HVtO+3PoV)YU5aG7CWR1)k2527K9v z$MNYF2-1Ai^%G28JtI7mn7FKPsjz+Yu1c>irBpYxKPzSHI1H2jo5hw*&mGK^4fZMcQt9eYdT_NTNGtgyW^OZlrjptwI&$ z#jJ)F!Dfk6{pyAn50T<|U%t0_ND?0CXOJ)F5xUo9N%y(5>S^VihRhD;`nCf6b!Z&5 zHQ;C?(*QwQ@rqlPXp855hx*;;V)y{Bu`e?Ip6;4P`p}^7z`cZnD{@+j-Bga%&00F& z@cv52dDtYAW_MP9ZnMfe?ktkCd>Q(iMy0o~<_NrG*lp66&{P%pN6SFHQOA>@+4C>D zgb}kLSj^z8hVa=Y&5qmWiFi7l$&=6Pe~U)G>Gm@pf84D1Km-+&Ae6K8#o0fq_46M( zG2#zW3G(GJk_`H2BAQO-BR*=$=6!D~_EIq@*ia5Mxq@wWhDhl491NUAH%G|2yGoF$ zBzhzS6*_iRu7({E)XneO`_LkDhxT#J7YLrb_bhP>-x!d(_>tmaRz3G;CN!nIFwdKSJI@zHvb?(+(D*f8#Q zsD@)IA?j-roDnT@h*8kJG?&ezAJKjTsD&Ac{oc#X$#zk&hl!|l50E=SW!ztMA|x^wovkSIcEI(@AqALAI6$C@?6f(bAavzQCi0TcW1cj ziCmD#i%lZS=O?r8ymT#Gbs;3m=xM20<7{zFUT`wWhqY&!CP5`?8okxcSrWy*Mb*yA zevUf?vt$*me-2riS^abO`w{2aJ*wj>9WRBc0-aHFYS&)1u6D`^<)|c!gOjhFlh|qn zfqF1(o<~L(hkbo93;-a{0(B5hk+n99A8{~E$3a+^BwAn6_n}-OBkPDgBF^w{(YTrc zt{2CL>MV!4V_4{JV;t<&-mr6~fTx)r2pk|26L5v%e~F+5Neb)yt^yu0B@~Fyma^7{ znKp@iUPLqKp@Ibuf2)) ztmT7MyxU%Q7`g_a(TK(WP!IXOAEyTr=cCkg%39~}VK_!GWaDy1&IIv6fRpBCbx zVIi3u`L|-Rqh;qn2m0a#F%#NoX}OQ(`QuMTJ)EH+|0HN&Aja?_xO|mU)kkvAf8Z4*hiKC^)lL)~h)N zt2w`-{cD6Ewj$0w?DQwSINn)_*T*!)Z$8!LEt6byG1e%E_fQt1z|7A& zvmaUEEh_a^D|3HCY7`g?H`#$o>doTScV)kvWspjj*}x9&RvQV@a@VihQ?uj<0x`kB z&JzvWj+)5U#zE!-C>Kme%6Wl02$q|Zp2f_gj5A`&KJt>2~7oC60p3 z*KVnBDTpI-O`3;AC*%Ea-^Vc>@N5Q#TI}ENf%3B@r3nL8^O-yqt z4f5BUlk7-&H@xkRNvc0dGsc6Kaz2+{XGJ~3^(M$?Z0;vEDU@QB9*<|$wklw*;3r`` zQaiA(BH{w-C(@Yk=Kn-z$0l#rXvlrMEu%> zfRI_x9+L$jjmD_E#8@?e!L7prV1kETxUc21iXwn=3jMp7G5DYEl+L%r_5mZtHVJH7}#g0A6({~D9Dkml!%#pq;x(I9(6=~U?@B`cFe9L zSV((#5jb~}O54n_O=zcPRw;y7UAb_gud1;dVACc9oWsjmY}|b^6pCYV&k^QS%0K#v zR9O{;?l3+4@zWZeb)gq`WMV1`bUi>xeE9s5b0!2{2s8<&!3t=_kWNLaW~*M<^hSP1 z+fN-bBwwa)XMA|lmae{-YnHu%Ly!Kgs@I3@PQk;$x+FFQeRq>~+B z?32@~HDgG76wDUal>@~_su$_Y>!#ox|M*ky`NkiX=5sz-4PN8~U3fu#8#vf`>`Whc zOOql`OtJ<1uLTavG<+7(C+G3+oE_UIq(`%fNTawj7_wL&zGrL3`}xiZsi?BH*h=zh zeP)}#$6qAT)Gav>Nc4~kGC@R|`viAgS?U91wVrJ`BaJn*#$WgNljQpc(s7S&*t3Y# zAXFn?2J5&rYWOGXCNj!RCpmETPknl*Qw_t9qzP5erHJ9nqUhpL7q68>)`%^;dajO) z&ENuXN``jy#=Sy#_TQI@seU0EBPxq3Vm9Pu(^2WUE!6+Z-?{_jzf!&cOMQzU^>}3! zhT2F1H!S|k2>}Hm`++KC&>!V#+l4vXsk1NTD|+fnWBGH-DsFQcAbNPn4p3OzPViLOTAsHh(Ng=ApASwTui?6zERr*65i$R^Qf%Nk5rarqReGt3Jfy~*G2u1=I zUArm&gD=??D@I08-t}!#yGR{kbXgVKP&JNhcIxd@Y{&e|v4auUBx7OaH``RCqaP2R(fy;R_2V?SGCIuW=jQq* z-5d4TE=nQ9{I2)YT*0d^r~0DTUGlrgqY(J1%75cOKsk;GIf(p}eSwxvaCex{1y>5T zMSG5uu3P|6DrVDBq1b2Zp$#Vr%UKZj zM7{WlbZfn2_!5!i&?c-_;U2FA=H(S7Prrfk(bkyTr|Ap#{Vk@EV}--OA#fyA1;FQC zibZVdeI>O@Y!)TG{a{fqK6WD;9Df|bEVY&bYriyvI@2tYA3b`*1LrfkdQnW{i_IA4 zLE4R^Jkw!<6;060&Cq|E>`Kjw;g-9@jN4>FV4@5v5>eG1cr@js@1O?zszSz#hgUKp zaVuIqr38C!G36d}rl1t5CAwF$w;)M7Z6$H_8__pGPS)K+qNnrW|_w{2xI9 zx1qKucJz_ff30nnAHLz@p$XeY`5g7D zX}mv~UgJ|SOI@@qbSE!@cvOGbW6KtskFwSQn@`qiG}Y5PzLY^E=1JFly-BL+*5ee( zJH=FE+;33SiR=Z8@u$kRt~ThxMo&M+i7V&jJ*+LGURBi&i2s;?y>C$}HvOB$zAYrX z$g4N230=aZ+(<5;!4>ubC)_+9pTz3_XuhJIs8<}h8HLV<932Q+hFBhLpe>zClz?;o zzI3mlHHZ|OJ^S7h?&(kOZCQVx{v?2%T&?ft7fNm`!XI8VY#-{tIouO4-Ep(EG$=Qmey0?2|}kbd8s+V*)_p3Iv=n(l;f`&6E|g zva54jF&|IK6;dPqprjr^U5^RMp|J@l)Q!@oGBmw%dTxg~(F}xUAXrTKb_G@h{50yW z<=Po5B23>&cv3peC}&S7{0`=N5;W{E!aH`&%jI{c_(b6j?H1WlBw_}}O;sFyzOoD8$<(Z6!8+CrVfalqlO`q9W?GlJ?1L#)G?Q zB8#(W*$XUo-lLvHnD((R1@=UI)hY2apODj;>p9RoLRkZ%SSlcj9r|#k8^Awk>4*-u z_2^LGFqvqr*R8Hj>PCf3f2-7__7->Y`BwrZ7EU6~ws6JUy`GP^OQ!LbOI=~v;Fn-9 zE?wJ_{xoas?THPqRC;piEjwitMdCTN9oEwcGpquhh0`L?7{Nv;mozIH?74LKL&aZurqR_EL=}4ucL5?T=tCL!&`z?$uEi}B-$|iYqJiSbC`INZX(^Sw7=S+nvFN5;q zDbx@5cd1AEPU9KwIiDbWVcS7suqRk<@f!>btW`wbYHlMfg;^OU?|zpb&AWC`-ccd$ zX1CK_+0V2zvW&GLgDhk|RIjrhod>GAv>I^mr_16~QX=BlGM26m&u5~DuG!W?Xc=F* z_Kqq(Jb1;lhrtNYlU^4+@$&5%Ji3e1U;<_16sb-2~TUe4OrCvei#4542*|Z z*)OM?T-_XD^%LABsA-~kc~rfkr3EQbc~wIYvFA-KaxV!VO^aqgSNcwHL{&=p(GMx( z_Bb^4r+6~g{#o+-$p4hh1vU!Ch2v-8wQ(lw@v@qf!)%x_r5zCuvgl}#uX8?{DsNXY z4td*ZF;y1v@i&ofe|PBn+GbH^uVtQxJ(4U3QHNtkO{23K_U^#S1#&Y0qrN^RDw5mm z@Mgi5qM{6sqy>(n@C=ViZ;-L>if(Z38#$7FZ{sQ{@rXzjUGdkgbOd|MPmAl4f9h=K zYP%Ooc-gUHqg<3)C&5@0ocim#Pw1u<3RrhCL;1Ie5d8Gh7e9kSD5qQZ4=|wne<{vs zWnmWy0ZxRPfb-_BbqyW z8*Y=Y;6-_Ws#+(g9t}ft>RI*bt5Opn&-k_eU(qm^y)q)`86by#2ph(vudH7xr5BBU zsyMjFbNvkz^0kdrw4kTVJ{h(f>HUQ7qe|u+>ywuBo|s~1_@D|RBFp^}y2>(q3g zX!U4vAz8lu$=vrydXU?`-+Je(S0zv>Qxc7pT)|ITB(wWIrtHfoXUkdKdf6n=AD)^@ z89tYC{9&K+wP6ox-s)S^0qkolm+`G1@O;=hs+??};2<;W4s|ueIKJm+uKFhsiwTk> zwFNDAk?X}n`@7s-FOdH{EHpSZ%*I$XX7{N^x36?7wkDEW1wDEag%EcRK|Joxsd8ne z27sHQXyyS>t zt>lmZld4kleP$3L0_;4Gg0eRc9-EGo&=WkgQG#xtXx29cWz2WAL-lTK1K|?7qnr|3 ze9pLq9|YplN7bSo;74X7z)`$8^oTvp3gy63xf0U|sG3ONs{{-7x;CHn zlxs|TMq=AkyAbx)N)h(_V3fv@v){y;&wS9ey^=?X@nA`P_$bXV8+r&)DRX9A35 zj`%9)rVh1r6|3c$>6>^=)Zt@pnCnIC^t zNZ++nqMOG$$1I`4%x$+;+7V~b4_h(iRlTAKD;?X=rCuWVb3*}B9V!KJfEU3RT~`qjvR;K`7& zxJm_nF%T`**M7oA$nh(QB?UkmB!4e0CaT)@EB-M@mKvbn-vW8)Z*!XI;^v8Oh%aq! z)3Glq0Xld4HU*5J9bpciWd49>*E~_zDk`q~);EW5D|du?05bEAvI1c>Vs63&TObN{ z;fK#~SSW2-=Sb_3f>$=CZ6$h`MC7bqxjw~t!%H2tX~x z7;Js`16lBo`7O$=xmZuo@xL<#Fvz%^_(GJ_qmio^i{Tg6nSnzy7J-QG2!*zw726~g zU^zjjut13rE$l)*yRNauc{(^(n6OF`LyJ(JwfoZ-8r`q+V4skuW7cz4&^RJc>bdll zZ>M~RCXJhM^4!92Pql;pPCz%W-eBB}g#92aA}VQ;0J|!MuTbn3yP2O+ zvRG!V)na?ciLZ21xt3P!&C~X+;biYh;h%j%ii4)nV;M)x!DCie)WSix(aNXKryXOP z><{Z>UUkQGds#CZn05;X^#{Lv-CT!DH}Gv++=rXp0$Vq?#6)WW=afw9gx+<vVbeOjzC;`k>PJZECFKxCJW<+jbx*sLeZ8JK8s{dD{a>CM+im*8j%FSubRJRs@wgDzRHpL zXk*h)d8wg!i*lc?I_g7bIWw*)@AzyHM<->U31_z6tqdFDOfDCuWRixX79uhYs(8ug zBv~(@n+^v^KI(tuixQTk!rZ+gama&jI@2THh%VXC71fkMNYs3nno)a3@_^P;t|GnE zk#|$_#3xs15ol-dRnbpF$#*PRDWZ${^K1HI1Zm1mh z7L4QMxu~;_>Oa! zHSZ+!6o+94Jwi9zhqL)FfgO>EEirb5@1yv2+tx;`XN+u97SzAigJj?LQR0oFtY;pq z$OM$J=$`HOA`1Io<4spnniXFJ-v<`EwNu#BG55t3rzCdaI#iCy%JBtAui$+VgucA2 z^??1VqiOxn(zDkZ8XqdSI15YTCPi?-ko7ZTsftob`lQ_hmUV0EbGgx=c!n4!mFW&` z`9aOhM9_N?i{hsouT(8_T|5V@OU^7L1s<*KBdDAxpf*O{V=-uWr98vpCXzWpAbO2}3?O41`@NckP=v-=S<<~e=*zJNk_?E$F?h_gBp(CDAXIXw| z?BS5~VAqX(!iCY{6B~kA@=Ccu`bl&h$jG-Q53jxMt|+T{**>^G;&b_bCDW-qW9vgi zkH;C~$e6Jl&ea-1M?2GuWMrl!>qsYv?|C-&Wfa8970m}oq6F1l(tQLZ!Rb$48hryM zl8#0&fN-Ror+yCi&Z_@$ui0SE7`Bj;bU>UgNi#~0Z(4X!>sUxvs^+4WQQ4NWrE1c_ z>sJ@=VDwE0TPYhccU?W@jlq1+g~Zv$Zt%_T1Yh|OO|p}}iWsOS+7HtBq8fup9Xd6F znZn)fo%vXIN;7r)3G?zy)kDSbRJS?t31aHi_n+Fksp8Xt80nkHQlDP zvW$qcTP3%-o*(zU-xDIO5nilQ#43URGZT_LKfDO1jShTlXEKe&!J?fTBj_4DT1k^^XO%ro}tJ!U#2 zEo%mBS)pqMOMfMP8C`kY>RLqp#X_s!(+muKNE0QDFXP+C<2+#H1ebcPHJDgour7f4 zFNKE|*a!_*gb2K2R`K2DVXmMYr5dKv_2nkWJhOZ5l1fBX-m`GnLzpgRU`F*prK@YB zgbShK4qbXlR&NFvC}IOTyTzCqSzg>lI$Gt`?|JODA?Y{kV;}fU`)%evH+`2ZbedNG zg{u)(FhN!6*|8(IO>fVd;A!an+@(38mFyUE0EOyL2*ejbM8q1bbFH4=7G50rkJ;@F zGIw=*mkSeTHxE0=0h4b;6sVHOpmR>3Mf#z6n11U~uP zI9)wsU`>P0bKr6ii8_sr+7F{mS&-cnXcaHmygHetVd5yKZipDXOc{zS;@*3^9q*iI-s$d%`aycVN-qakYkV*L zk$!8y<^U4*bwkX#7h(P>a&s8UYF>e}u^x3HE)sYfsV?YQ4oYYK5x}|46f}x^ zqY1NS9HP@#z#I{Cm?sU3n@kV+b$Ht!f9a$Vcyzx1Qe&8L$k!ls-1qYMR8ov?a7J+! z%|sjPvt;cY3dbitI!vB+Vl<7}dTe)c9ZQL^-_QqB+&S%4A!nE@8jQM!}KWcNaVP2# zae*EFUw!2(6)`9K-q^=BPzo_k>4?x0DW~T*|5oxZ3lf%2>Ax5(&^}b>e0T4GtoJrd zh_8HhZVJCuCqd1` yZH)u*{S^i^7tZXNYO6;P$V&WCM7snYf=D?yoIy{mSUF$J( zHgB}ed|>z|NkA_BOfDz?E!aCi$HQ=10kNPYW80Y(K^A=fQqf*Q;ZC=v9cYxnzyYT6Ax&ux+Z$Rtj%?GiiOf2V?|b7zE!zhe z^606?)H;#Nc@fzvGzrxnN%3_s!?%l*{JqvE4RG19j`l5MnlcSxA@)Np z%EzJz!shcN;N(f6u3j_c;2nFRcN^hMI^Tr)Lw@%$S?lwhM|!XOC_dwZ{C@%x0uJob z+;cpLL1C)%FHQx1kyA|R>!wiZ1uKnhgrXT_EDqYcn^Ak1_Q}j%j{QLIZ}N)^ec-cSTdgI~ z`nCPnOD!guhkra3D}~;XFyzP4EBz#DMnJmZr{59C;DzyS1{XUTd3T3j>VV$EOwa7S zBRdQnx=@@~%S@;a(^X3xFswE-g0;Zr-B11pPgCJ#Dc6rtkCBDEa$m59HR2@{ChJv% zH5~O<*;OB7;?PTq>N7g1;VZK^Mb)ZGH=vJPvTjy>d+l1weJfdMbzR#wHPUdJpnL!t zpis~&w!DaA{f~gb!gedCY-O>5G^4?|s=hWX7_WFIsap<8)4U){nTC0R&)!_EeS+Sq zwXMt6SZ$nU+v()HMs=le=JkDt?sS*7t`pTTJg6QpPI~vzcD>7{v?!sBPyo}PoPD}Q z)8OT%8IWav%Fu=4iPKzn7*vnB*LNJ?U%*Y}{PaVB-^2AT!HUilLw0R#&~}|ZHe*~N z^_%qBLmMG!N45fyA{6qK6oH(Mz!?N26`+!%Eh<4?5Hs5_1gH@1JW^H)@ofwJb$qcs&rj`MjQ#Z%E( zLq9cY^>t<%`&v=?Cp7DsLnHb>EX2tCVhq3IM-2BN^?L%13Behb=k-Gy34hgyDvB)P zyl(X_ttCPda?fuTuDqXszo@7ZE!ONBnDu zxZNT0IU(H&)-n|{wwNdAwR58 z12of97Ns?_6o+ih*8;O4AqMh4wHoPj_rZFf>}OwO%!^4Z=sr~IPgl0tBLq8yQ#Bh= zsD}_#m4;ZK{0|+$x zBoOMdIbz!^wvr&96X}q;>DgB;{An}|{lgM2GeL8BzI+z0pZp}y5=olPojw}t_|^sq z^2!?W!SY5ggv56QU%VzeEONg!=!STq2Lbw4aIBN@im711#94m)OY~K=Fp#0WSWpL1 zo*%<&%I+_ZmMS;2(hpqss7^dLn2GvNb|k3IgI-I8ev( za2*XF{f*X=QD^t%xsIPu$DtnN2_v^AVO3Mq#wbcdgx=l@Z__KXm<_dRmiIHWpF%2r z9LxEBxMo!vxJ->2s?`4j-D5&TpyIfca>MziibMmpah_}3Tuujmp$~Q$}?;G z%+5yuWn}e1s!hWs?gdtPl1!LOBRON8%8uQa=_2(A2Fz_K%7qP^z(A(M<-j24H(*v6 zXKKtwQRTpQ{LpSeU)J`R-ANa-kuaY28_zW#?7dXD;RW>162)}SR}$xlzK9vTsG)y> zz@pm+*P^%NXaTi@orvAd(;C~7x_{-5=|jq^9h#^z8Vj1!EWxma9m|H*hZeTEv(k~c zzk|c+5e5eb<6QCRDpk_R?4!HNmEJP05{oa#6Gvqty|y|{y0-|LGlM02Uel`jZnr{y zDjzyaK5~fE)P=eg-FPL3#PL(`^(@)290*IjT!sj{emp-?*^?`_J7@xFBLiDXpq446 zQoFC^mUj#Gif!KZNZXg>uixT)UTO*BCdtsxhSM{I!`!uG^`g>JbaWH9E^R8U6qK5z zk*y8t$RV&$mshDd37R>7OG#SKHM3XbhHUjpksjpc)D46>@xF+=B)>hMyq0f7NN2@v zBseLifM;L8`(AF3R#s{56v!OWtbXubuVmJEf!c$&nB57417L?1ynmM;?VZ$+7RO&Ms}CN;*?CADC+27N=`b|dMeHuPc&Ta{vx|B4fUddS+Gkm2vrGjgfojXi$K|;$F9W@Ub%*2iSRSlD>#A>0lK-i;G#-Tt-!y~7jr_+ZrjRks{taEfG`v0mb1+? z)P0u>YNr4`pJb&WKN%3(@k+FRiLA{MtpKXYz06rN!q{lR(qP&6%QA&}<327*QF@Q3 zo4kaKc)dSiOTQcg6Woxq0HH`VIL~IY!&p$ai5)|8(mpBWQhqvMhtWB1qEcB?7Z8#D z(5m>+=QI%n2iS@r;~~yW1y9 z1+XKaTcDtbO=?c?W5KGGkFI+Nq;_)Ka%?M&YybaLcb-vAE@>Qp5djeafeR>AAgu40TmY&kVtPTASEolhoVReBvL}hhLjL$=p@<6-Fx?(d-lUU zd+zRs{WRyidFN!x^US<6^ZfqvF{IL(E_Lqv-*XkjL;wxbKo_;yRb=Rk`z zUH`KGZAN&qaslR-m#Q`3ZVC+tP^;XVq~YWkCn1;A5*0uvoYt__^cw-e?~p&;Ar|{Z z8QOm`fU^nJYZl96c6?VCCbPh!d!QH`sW9=t8OKe%CWj+N!_1-ygWt2NPI`e~fsWFM zxC){_Ywcl)K}N2a8qn&}?UTXzUfTLOwP~u!P<$cdsF1#WK@2?vT~`E?+8tbm&yU=` zug9(#N|hSUC+ps@?`b?(y`I;m`5g|#4MM;$bQNq_E~;k8MfnIg+%ZuWOR2b5+{2_? zpUzk~9t?O_$R3D^Mg>}GkSc9nrW@8i<_f56thc}94QVL1x^vYU=XldXw>Jc}22p9; zApZXJ@1%zttEIml6PZ}-E4NYUV* zBasTA$k^e|QL8tp;LXp#r}HtRy~*QZ_U;y%JbzlH zCQ{yQxfS} z9!=j%<6|g&A4tJRB%#r3jZ|aGq3g`Ng>w})q5Qb7JjB2lu(LYMz!Pb)(`)4U938Rq z*?5;ylKQWLo@k{9xZM9RNqe{*Zd)V?MQ;65`y=QtCOHd-13dbhK#|LBoj=x~ ziw2gsF$lfWDHHtNUHnBDBm*ONt^3npYYG zr9oAylRtbX2Kcasa>K)x-ZHoqo7zWNN{Tw__c3k&F~Qb&AqL#lYRM&ZM6F*tk*}_T zI@cE*@(q?}Aa%otjyNk1oqAEU5Ab9BB$<1cMMK%w`s_fl=W#1PIvm=j;gQF@8#=b- zEW^k;+4RS}aJV6}px}SC8UJE0{I|1fBuUH4in2hoQyAJkNIuJ&p!Arz7Q>ZaH^SLwQ>P*39hWX@Z?-sjymO=x&i`%N&#a5E`SXy zb$e50S^vq6xqfTh^+S8u%>J~~zo@?NmDyxTtKiv}cmHm&BOkxALY`hp{hh_5zr(Vb zFHHUghN$Ut6Ino#r)J1jW+I)e z8=4iqZjcvfX{(^Xd?0#%!xWu}=WQXn4~d}C;%r*9NiBiL$%OXPd$lo+=LBg-4m?$= z_oE|rL+2J|@pJQdAT?tUQJTMSxMb`LJY#oEfQ*jF$nTw**32e%ogNZ_93opE)XbqM zTPhezhI7Zu=MO`?NWwn~R}J&j>)1pDAp0nrBhIdGUk+~$8U2{I3DC48FyO`TXYnxT zl;9CbXIILKdOxlj8KC=|t&rUkX<1-d3q|^Yy_eEE_|?!E4-<{4&+yp>2Iw_VnQ+Gh zJJ`b>1lBEkW%_^sj!!h|^bf|BbFn&*W2$b{oRRJNy4*Z70Hg*soVJ9J!@hR4-7-RH z$-{7;451!LL_OJjXdU31oX5?|Ta_;BF!A78vqbomLbI|`X7%_~hKv-Yrmq$BHS#gU z`;%kf;s>8?h4>)oB`60=8d%H}1rX1Xz(wm;+g-)-IGD5V zB+TxXWog1hv%z%lF}2(+qM11AT!C(-si53PlVW{o9VCX4zy&Tja9}nbS?V@k{DQo( z+-b26RY?G2?U7M3>{uN5oa5gdfQxMt*{sz~a@f-W$Rd}4msc}SSATVweqE>Y!C^!K z2(5XxSNY~ygZ7tUZyhsd!dd6APWVb*yMFU@V@IEtX6;X zMF;DbK6~1PyO=AJO-m2t%kThfaG2`Hx=h#mxNwKP4etI%VT5+TxWyxFHPrlgs{E#0 zdTx29X{iiK-`txU5k1SHh^j+k+Rx9l->_&?&dzMZo`A-W1BoT9-xiT&Ulmg$ozv|` zSbhVD%%D+51sWlzy&3qZq0!;8#JE_ATBxMU(Rb_)tMc$e!rx}e3=)Zr>e1~c4M_UfpZNk`XfbXtWxr8=DvcU% zn{$l9&yCO-{irh;9cx~?0%Mvzf|TZ~vp%tg^oi3nhawMb+x@X4Dy!rT63`~Gq6*m` zpu}g$3j*XN*1Sfwr%ry*r&bjBih@>wa72~7=gq*K^f}DYLjK=;XO)Z29W%VLn3H|* zsU)(Y&A-Q=1=0n(kmG630s^oz&2Xj{rJ{duwP5N8Ekv1x)=bcXO7mXq*pYX!gy)1r zLNb8|dd`;jsO`aLSJ^q2uYDO>T$QdHnY?L&r+C#PlIEGHD&n{fRVzW#O3K$Ufk+=< zjSs%;qxx`)$z5vOM!uvp&STHB4Tk8u`_qDLB=JU5TprmB*kRXWPETlKmOu))eePL0waZw_IK2kU8myI~?88&OY^Z`gRj zivgf0l(V}w!h}$NL3}-99meyw-EJb~iV6}ICuh8r%3b$0;N#?H$B_?{@UN7h8 z(ZLVz4`=X5A{x=2YQ8(MEcD)SSkMf*9qFwj*DRGTF1&ZGqKq5L@@#QRSiX}S^+k1C zs9C>88;>kHg{Nz;i&p8^xER#kfQNb>iP>7|PlY5veNR2%jRK<9X`f%4O65|qu3T3} z3%|;eY>d*b9VPBn8@FZpeF)Xp-XV=$XWR6~=%38{=-7{&IhaaMn5X%Qv4vt1mkuYj z`dkhYYf4YJ#j&I~A$Buq&?wZOS8!Ry)v_}nBIJy?dEma=@Z5`8phKv^=g&)96F_~J uf1<7K%8&osya}HE*(ch6^ooDnJe6Lg`+c+ln;#tHg!HrxwaPBpKKKXpA?|Yk literal 343391 zcmb4}19N3v*RI13I_}uEZQHhOr#rT7+eXK>?R0G0&bRZt=MS8kmD*K1YtNcmbBu9c zcbL4a7%VgFDYOEXmWhled0yeCdhE;V0e! z^Y7wMA&k7ZA$)-V>5iq0K?C8x@^?O*gOnl31+4J@_!X#LKA>T#T|R(esa^C}BJzm8 z0|JOKYB<3JIClf&p;FKrb$_-CxUNDNKN%=jD40E#ZWjtaI?z>GcaxeV_;(M%-p}mH zZU@sy`&(BHB6Xbz$&oz6R^}A8qc7{>ujCHxL3OzNd?{3g29?qS*v=U(@!NjtYB`5W=o@}S@tb<-)F9HI!2fXZX*r*WPA?%do6~x=|?UM#US9uMsQ#2R?4hU15X!ah`g;Lh3)Pz=i47jYk{)w9qt#m=7t=44AhJ!buxL-bNZ?ZrfaLN zD`-)aA0wBfAKbB$vmvvYMnpA{>pqX=T#k(EauhY~ik^PSp9gV`d0j=)3cnbZH5#s~ zUm>W(sI4tf|M?r3t8##-;yBL|1y(FV?H~cv4KSJE`2pqMikGX~t7oqQJZkvVid}fX zX35Uc(wQWb`gJ=QdLz4%&ql+w>pB~r!1CT=3pHga(u+^P2H9_w<9t7S?KqT8y_Rk& zf!UuSfrieeInU6pZ`8qicHp`MRZEw4#(n~OXE|^GGJ%=!W3l&rXaS|$hW8ghPlYrT zAQT5@(oYew=Na#@riZ8gVUi0{334XDvI5fNr?V|q3(p)Nw9Wnun+bulP1ypgCI{~d z{n~3qABqnNl}7|KfL_OMjc7K2b0Yu}M^B6yCZ-tYO#CwrAKDoA7FN-(QlLDpN}T7G z$q|7gI#Yx*E@Q;25Y5f6LyRFY7&F(j6o(V;LExV`a(bw&SpAG=13Ha}TCR#YP8)=c zU`)Z$Y=t@6D>6i#pK~^w-h2I@(f5XMY$w zsM!v!J-FDe1UugJFxvL`jio2rW?;?$-QLrU_xmqDS$Y!oU+<9GeaM2S2CR9J@)9co z?ELKDImNhrR)ldQ#H+tLcji+iCrGDAXvu5|FG+BTej~Mqff6H)rtV8a;}6EW7J?{Z zSP)sjpOBv*m4)0VSBf;@w@}vrwhzAz=xl>P_>? z{FRvuGlVs&IJYYIq7bKGqp-7Z{^a+)zB^{8ZHY!S-sv-zHAPy$g*ro!T7vRu2zAE{8QVdFRswE44o>SoL%dt zZD+l^+`Yu>?^lCYkPpQVn^&^?b`13X^j?E&j4k>mCRqk|n9v`OKePj^1E2%&U{_&Z zVIN?Ju}HB{F_zGASff9D>m0%KLw_QTr8iQ~XqnNPr1GQ!#y{#N>Sm3khFD{~Sf@<2 zS6deMWMkg;Pfya%+BP1VGP>4;n~|DTo6#<1FFl_ypQv!1W9(ysaA|O-hq+?VqPMs| zgUR}x;*XG~S*K%kH+30nL~C^Iu&$i1G_THhfABi+j(bf#BY6sWqP^nYdL11dsBJe- zU(xxXePMrXe9Doo$RWtC@aA##Q2L_PvgA_5qNL(Jd+Dpinjp}X(jE)cy42#kid`Eu z5piX7hI~*Dun40%;_t%_$1vqMQ6vA1f~0}BfuDz$g06sTN3%uAM%hHz#M{AOVB1wE zCSas=blG>Q#z>j2I(n45&t~`89VORP7F2dw2rw@(*Kj@~cAguMup-+CtBZ;oU>^2Y zw>@87X?J*ZcyZ{KC6$fN&}Mp3=+Juac%Fxf3`~P+|D{V7Ok6G9k+mONC~hgWB$1u0 zo%2pg!fYZVxt!>rbSTG>V;N5=Ya>BaL{f4gj+KhZ>hiIWZIL}cR;-g@%hzfC`VWJY z$lc(1>UzpjC=eblvsDgR;m z1epe~hE=`0h(NxiLQX1CH7eh9^u%WT2Pqk=nc0-Qt>on_UlTh?X$+#9Y%y#gHf1kij)vmXJzJbin+P~5IC$tk3iE!Ep zhxB^uslDCnUA)6xrx&NSLyajd?yFDF$M>qAvEUd#S5YF z3^{l|IO=kGt3GYdLrJ4rRdcUe?+9;^{Cm23n!DJ(bX-NDJKyF6d+@|*{L=N8RpVafxl^cf=W3%PzXC1P$&a<;W8JD{M zd|Amc-Jhh_=5LISV`!piqqLqFpH$z$ycOT?ho?{Y^w!WCYizowKZo?Jead)x*yT3^ znI|ny?NC|%G!`~F<7T?nRxNfv^wb#nY`Bfw%1;{f_fy5l$0XunaB6xwZ>Qd?Z{1Yx zZP+la+2*7=?v9O)lC8;!WO3eUJ=>AV*304X{IWls(de#+n10E$;=OX|I`KaW0`2zr z^v>O==sd}P8cc@Uie8$6$`Ydm*WvW$xWBbiYf+ohi)vkUqd8fPI-9Tlvw^Vg>d5qT zc&8QA{KJl}O{hi5TkXmAfM?#r<;(GXd6TxLy8lV&O3yX+diyt9_uGlkOFukMQOt1+ zC7u>{wa@P7zPz2Q-7ucg*RdO=7w3xFW65St<1}{m$F1769@=Zk{S$Wz51xD7d(q8! zR?SeiB~~a_%M<6-!*<4P^N`)2&O5h1uU=zs!vXbtvAoJ&&hIO)6sPl9c^!EK6mmmK zHsJK^GZWH4y60fN+0ajLI6(3x<!t(qTY_&FSeu*M&(F*ZU(o-{0Ky(u z%b`mW3^WV4QhCeVs!H*#IoSG%;q~@MB_uYQ1UtpW#ez%oA?ipW?pKrbCt&^bN4Yp;CV}|>? z)dsoc0zaWX2Cz6~K%@g2raj#Shkl0V8bx;lt?gU5-Y8D&h) zO&e7|Q4O4D|z6 zsd==9+Lo7}DBFJ{)-Huu4Sy5VX`lxdhmA{EP6zMi0>i_hVKO=XvA)RnU;pZ{-y<5) z`S~jweumv;cEi`hVAFc;dlF8#=6UfPa@K9#y=VH)s$1p~KjBJBN`wOIPttADzvuF) z!A<`;JvTeKA{PpiOJY;4*?3yx^KqtOmeshd>v7RAFKb{cs->y)-Cc?xyf*PRoS;h0 z{!I8^p{mBW%~x}jv=)7uB5{kIPm&w{fkmw~Z^^JmxdCai}$J?`XXY(dDLKW5o}6pXYw1 zoYw`t&)XEl`S#_`OBc4*QJHc#ujU0*#6@^7%zD-hyK2UQyqu@O>*>q6$2Gf}jK&kq zPUwED<>7X1Lb^o3tlsCh3*YXJVS9;?+6urN5@;h%!q2&Y}=!vqEtfW znOInYgcQ*ZpA1U}f3I-chduP{`f(ZY4`+XzoB5nb=`qUM!vserTr{m+-llZFH&(k; z1;Fz@r!RMyzUK5UP2ZfkRyebJ)ln*ikx#5d7_)2EFhYv_D9=}a=z8kB(z$X+o09W+ zoev9x>X8IW*FyWH-s*V3ag-^<4ov|+JKayLP~tErPdvMGe^#t|>%Ada@o>cV^%6=H&3-fR3s{4jpTOgB#VCer>tzbw>sTm- zPES^-TT;)ae8R4C?AN8%iW-Whtb`tK=7MD095ZqqGGc<|;0SxK_*64lpIN5E5S8vt z`1ihh%Vgd=pH($`_kiMFhl+9c#Z(*0b4?=l#3^^_x8}CRs^42)PZin+l&N6EBKt~3a|GmN*dTd(*v_Z%Wg@c}SbZr*-n+kY z*x#@FRh&yzQ3?hYeW_w%yzdL_Jom8FD%IoKp3870LgK?yAtzN}<6q?sNNV zisu5D3Y(HeTPV-j(9+(qXAxvst~Rc`pL{-+eQsC}reYT>)W$W;Nxk3Jwm!?`yiP&! zo_+Q7Hp=l3J?3o3Ga#Dq;i=JEvd)Gg3lm1290~Ew4~L>+MYVARGfHmw zTTddg%%#GUxcer`&i=0=Pbc8h^TJAREeM(UPm}y@3E#)bq@DNO9^RKLrwXxA)?guANyo{X$k^AXgiKY=p3`6Sxb5E^WeiM_-^B3h^fnh z^dOKp)m77s^TT0w|8&3ML@N?0=|&t2kkPqwO)^N}ei$JA!gyW=Y?rSTb8Os<^Y2D8 zw6IjKx^ln1Yp8*Ru59L#5!g}w3t$NBZTd3RNOj7k)5V3-tcnIyIVfd=A{A@iue+SD zcfdx8WG&`XelInZY1y&iF_fHZ-?aB56DImgq#5^Kea=T^tUqC&B6S!r9Mc)TeR)K= z5JLDVLC}e5WxN_!0ezKl7ppRhGLHh5U#aAo3KU_w5_o4n9GCV}+p<^hPhI>_ygI<` z=>4Z{K5Ip&zit@!?dt3E>bbE>^4^8D%}qbzy2rm6lyd_Xw15OuluT$O3Sm#IkC=%4 zI+(3vXM`w*IkjpPvg4u3?qgKWThKRtq_B;UF;7KU7|I(mlQ={XCUtKzUm|sunoN{m zmm}_v`kJ6rqoHU^Cb9Q`B^Bi32o+z*=xB-RKF$!3InuUMsbOllFfS+7MJzwEs9dB# zLYTT^COf6!t$A_;()*FmC)q;N3h`ZhN(Y$!Llm=DxZcM04>ozpZu*o(a~7C-*1%Gf zw}!*Ml&6;@3=cSL43B zdxX}y|Jou-i_=#cv1aui!3+%xytM21B!Q-I*i`AxVrWE*bIW!{kmAQ@b^A>zN}3+Z=e1e zGkt$;^Rda!>r9J~&wA5kS;ysw_D}22N#gm4RWxigjITSsFQ?m4vgOMh=Q#;@w-KTB z^3IJ5+ds|i8#I!R)J2ee{xGU=$9jo#g#4>eb| zC8Z6Tnhg(wJUgKr(vIRuv9#xKW?vsqfB14O`=#X`ZcYFJ#Oumq%l`Rm2M`{c)@-G2 zyT9JLH_}#|rg>OW5THrxl;n6+)U}r%H~#1;kk%F7lVG-dq#8xn@r6K;AC+>g@4&%( ztn_OKBF2*65S=!@6)m@KK3sY~wfmeK;k~UN@eVJXX&-S`=bP6WDT-f+vb7<)!5LGk zd=D{(_w^X?W*kweR?1t-WXMgSa68`}t+kKQ8R~B#zh!A`Oh$llYZ(Cos*8LPJk%XXgP29EUD(w7YIxJ{S(}2Ca zbYE^q`n91N!Cx+4ot!BM1no)CWe!Bb5L^fhgWm(@3}oEYI6#SFp}O3MM+x47qpBoO zu8KV-fKD&j?kGKLVCi#TfOjH`jFga?MJdVmz60+Bm-Xz}Qc2UN>Nyi`KpNol*6iaH zN+B{4N!SSp&Os%%KlCXTbq4mR^j@HY5SZQ!<}y2fSrCRSdWY2VS~8r;1VQc%%NnY6 z?4khfunbV8h5;tP8wmS#6XSEth%sLXMc0EPwLZNs7f+4&)e1O3*}7CLCSBJ-bPFt1 z!l?z)v4;U{ohLFZ>I0@51VV1u5y|IM3)SKIY-vFG!Zj>nVL6c!t1NXz;#CNv0n6tS zNpm?I5Z^&YwBmBg7#3n@Mm|u}5_g4>hrXR(H};0(G@mC=xxCYNPs z*~H@0I>yOsRm}v&GF;V3g8d#y`C>wepVCk5%$RPJHQ~@ z@UJV>p~;m4qxBqp>`hy#%?Qql)yozj5R`d{&c%bsDr#b3B$TrIRV@`|vnL2~>!&Qt zZu3xWHI!o#mQ*fQ*{6uCEWBSY(`v{PSo<~hK~#m|Wy;nYa=%`bV^Z5r{MXCZLGTC! z%(_h*it6Rp@&-Z@6Y>nmZ(x8q4=4m7*q9MzTWVM+yJ=zqt<@qicLFXsKPSjYS z?dhL@c$NSx*QF|EG-jjIoLtyji6Jp%{YAG%1LIj`~)j<~$FOg%|B3 z4S%}pj-3FP1@+B^{{-QYL2>d-bY4P9tzO!ddUix12_((bALv6_(Q_M-C$>}|W-jX}_UYeLMp!r+g{r4fDX<=lV!|DlQXGt_FyD^VFjPANY-VIy zwZy_?MUk?gFj8N-v^&63c5jQv!Dn%+C}zVB@@3+8$o7r@)Qg{5y`3Snx-mhRwfVNGE&Gm(fPcLGmYgcjf{?UdEEdJVgl7BP`6GRfRC!uBfBDi@?Pm z6j+!aHF zo!fM3Aw(?_nw$>NBn(f@Veb?!03r%Y)4%dOI^;W0lLP{yHcTIZjsY^TD`_-$!5g}b zkmHMO<7&1|!iG;X$-M?kVJ=T)SXRU=kMuFi%_k?~77oX;kMJge@ooLdmEQqM7U?Hz9+|)r_ks zWvK?FWCgUcw@2JZ9E4Sm76{R}y-{G$kWvDueJ+yx@FXY_mXLb3Uo6k;&)Wh0%Cl8X z%YeA5dvgQGiq+@i3El3UBmcqhDTx405n!17UJzI$mwqN$bP~$aSpHkW-;OuT6SG`LmZtev=pb2`j98Uo8FYyJ04}o|>JV|e7Xo{D z;Wf^Xbzf?_J^4-n!)z-aR#saLhgpzJU^sof*I;~F2VtMv2CNO-fw7^%{kT!#Ixg|r zoRj!YoEWYKTSB2E^9V&*Bjzl3t%;ldsM>mP2pN(0$C+L~b-^HsThqEGw7?xZ63^L`M9gX#d*8!EDMx2*Es{;fic6q#~7au&VVlL}s zukD>5)DY|(nC^{0c^1-Auu6<3v)a#owx>dlcjk*3ZU1Rmo<{7Z10t7s0eI%WsNoWQ z$9YCE>v@T)@f8(>jnC!6N$Xykake6>5A5waN&6M_#X^nx)fJhRh0Cb=4;6YUS%v)w$1AF*(^ZK{swAn1(H+kcs>7bN;=8Eg=XHFF zPFc|!FAI{gTH*VstIXR@f1}A{DYQGsg4)B!&A&MKMIF%@0YtCSq#H1QJ7@5 z!(cCGf=MO9gthoHI|`N9kNsi&hGZ1vXnf6G_HXG_UL_&KIrQd?LB%TWG+ z;ipKMVqaMHai)0Ulv;rB^(jP<$`7|DHLi%ei(x_f+7c7#7fUEig!V})kvxE7$B$+U zUMQ0L+J@;=09iSpGg<>(;zlVfsQJ^0=;C%y@+l93IUVPPh_2+BSZ`z|gm)N(6g7`2 zV<7IA&pgCx#;dX%BsL1K%ERKX^2hDqHJJnxnZ5=!^y z8D1uFSUqkI8Ja?R%oBkTnqhvZtfl3AXilXc`HiSodfWt*adX?XaLQ<%l==q;!#@B3{1ykmWB7LA)N0U5qpEa5U4`b`0p`eg5 zVm$`p{`;lw}HSVS@^^0_k8lhLIDrl0t(6AlEYw95dXlv1w|@~i3~Mxc|9qjbE@ z;|M7f9{Oz|;G?F7*~q5XLKH7~LQFwGkC^JcG*JNOLhm-aNHU*lAEl9o9-KyshQ}a> zYpB}x7O+Php3R)K+^RgvY;zQE22NO9isYpdWsAF_$X&ZFFX6+DjszEF)_D#?>bD(A=uhTmrpf@3;^)Sq&$c8432Yri@ds3m6` zp|P$p(H9L#WfHF@zyZc@P7&oFIl?X+SjHS&WFb9S7w%{ZW`MXbMiT-arxf@8UPx=2 zSRweAjymz&un#1elIovkA&Tj7E`p>zO*)i(DYI~ro?iR&epnc$hNKfOnHBjjb!#at zAkA%Ewu11oXLBs*m-*rpbiAO-gK5o9!Wu<8jE0Z?q8L@|auj%?Kz7~tT{N@9roqn4WT@6$N<4$oSoAbC&+KI-8PbWCG$&~ zP`HlPW}1D(zuxm-GV>@eYTdSx&in>}p5CX>K~phwFl29STTsa1ar`(}6iFstgi6B^ zM{2Le1EVJpwhTnxy%Bj7+0XXGidl8St{0&d1c*bhEAdeON{k;opy`^6*6ziN0j^!>yfdjJtksY zoo8;6T}=x}_RZxX;AKt-kqtSeeINimw}&6F`q|W$WEqkpyEWvqN?@KPpSogt@QeQ za%J>&MS7@REIQB)I6umg7cK`!_V1)B)QP_Nl&gL({5ErqjRtOMsgC#g<`PbY-7`9v zoGyGyL%?dh`tD;{Ftg@eLxv(VVJ4u6Wrl}w%dKh&!FD|1uC!Y5PshP0-LGm!8;>?+ zP316@0AauJI3Qx2yqBWO>XtcEx$Eisp79RvnxPdKYQ>8eGkl~iZX&jz+2i&|h$i0O zrt}R!n)i(gzo?J+2YV%t-9#qn-$bNNQ@n-Hmf!0NajeHOAd(jV#4u%{Q^xIo9J2U{ zzn!D?#A}tA^*@kDUG@#>$~-avslj)XxdQZ zfm(izXDEl>-7dS3LrD3)eU^7b@QZujiCP-nd^7PQ{OEmB zpltS^3>XzWPGXMk2i}q~F*CTgs(Xu~yTMoKB1CcBBsRbmT*gzYzx_LU=?@A+b#{x; zk*%my>i#l^t8$Iq1bY6_pLWY!rfJc=g^)sqH=1JiVE0NHAtBL6t*}+{L@^`*o2mnO zMX6{C5jT@Z6kTV|Dq^{U(FZCTRnrTPXQAM@;j+hA0=Itqkcb|Zh$I%|*o$Xs*7k<*ev~%rNced%HL@7fM=+=zF>CD`~Q*Rx|X|h>rZF1KZ;;K)Ca~+j(n^iHThUFb729to`;k{Njj&ktJFBC%^CrTdl$`A=#f-Qa%QOQjc3X z;c}{nkWg+J6MpPWDQPJbYKfJ$5P#+>9UkAp=4@;p;na6t5k(ItK_TK8k60~CLwHzH z7DAwyVjo&6V#PUOZp3FADWPyd0qo2vE}GBb)=;FH9>H3aYF>6KQH87m8#)q8rm<+K zD#3V%RX$NlNS&tz&@ruX(eM6UfwLT9PnMm(tJZN@h)wZ*y>I;<&AZY!LC6mu<_{dBBO}~36q-y zXXKb*wd^`wByUKV$?nj*`rnx;gG2k=k4t8?CzO!W$jh9|11aP-Ai3_7usfrHyo?SeHSMst|KNLOz6xZ$TqPYfoUb_lC4ItvH;oa%5fu!yL}djEt$H{+mkKSN zhjKNM0c$-0PXHHSVPH@=3ckjnbRVSB?PYbzxeg=~UL+l5={?-v-g^M>;0oNmX&;Ve?5J9f@arV@pl2H zy$LuE5|m#501yd4C}j$|PNhDh|8x|bTKM;T3wi*$@M+z3B4bxh6Gl8a?K;Ht4P4`h zg;F(@S{D0ZZS6tFsQ8M?Jpo+*^;E699IY5)BhN2#;OGiaB~NhPz=vRaoYytF3}BdL z-t?h$#cC-NIDNy+r?}=dTP~MxmDT%9Yb#&->B{qn3xGD|O3023?!U2&eYhoa$-fIb zACJakW^hgmiE=5j!*PK8`^}IRDKi188fyRqD$F{D0`+*Q$Lf6cBY-4M1U+)p=C?m$*20$ppe-n#2y^N7b;kAI%DJ96zj6jU(#xxVTH`A_qYJ> z<2Yk|pOo*p7rp@!nD4PJh5}q}>v@PRcRCz}DjT3wC%gb;)``AvYC%d+nyx3AUF~(7 zkmCuMw>#kPl>h*`j(DG1p8)=b;?i%N1vY1*v@JoFh0N@89e_6x1)#@PG;QC+D1+b= z01rdq={$FSR?BKLr3vIbbsW7ANMqF{_b?1&@2y5OrU#68hx)vv_y7cST_fn$ z)y3+=BPl0RHSCM@v^h?1Z^%~BhX~E0MFNL{`f-p<9?itcEF}h)ByRu~18s-dL)~D$ zN*Bp~nJ_~t_9H3Y0U+Xy>SS8Lp*Sl(4R@SNU(t5C=zQ9Qvue-Fd&If4hsYWf({dX) zFO9MRD8K5C@a$ix>5e}t9smq@{i*fa6T$cPZ@OCgU6ROGWD)NZ;054nEc^zDLL!ZZ z2X@530g*)jJrP$0w1RrK5}Hj>i1Kq6ty^Bm#CU z_)BUdNXMggD!4!=8~@C5CL^&njz#fVM7D!(^Z=C)j^}cS*5)vzR?+^H zkQaczCCCT67$@dRV%)1I87B5{r`8-G_VV;|)SoG7R{i zXN2HYW*@lh;}+bHxV9`hMJINMaBo5bO~A-|djWjMdc#coO|(S-Zq*iGCE|T3lWW8_ zg|bmCGw#7yi{u}d2E7_$>tZN2p(wrwI2UpZW@}tf{+X%|O^wJWDZ|ezg&5W(>uWkI z$eDUQ2C$w0-cQ%_lCS*)xvsU1OVlotb`@MwKzHl;%P0P%>Vz8T#V+sKw-LDXn|YUo ztE(!2D_aajW6&6?sG) zCG6e0KOU77;*5jV$|%xQ@8e*)xIOOy@3tLz&B$n4th=SgXFER8Qq3*ka@)*%!+7@1 ze#&e<{`T$c(~LUF?#@uBG-Mp^Gh30rf5RMEfV01)!mC=2Aew1KKe%d1o5C>}6rYWN zz+%c#Z7Z4H@u-a;iUQ3kR{&=9x;ET_q#_?BU%!qd#=OzNDk?y3j-;D%sG}%ZwZ}PG z8THTiizG2;U065F3^X>HO8k4xhAsC8wf>V+&YQMWZRg|aI0@^Tuv*(&S-00pKkks` z@*cf^lln+}6n4T=2T0eFEoTex+eVvlJUPT$Seh%f$Y8YJ&p@44$0G9U5Z-%&7JO2o zlb>{Pky1B8wc+j3Y6v-H`E2X?N|I0kSokAM0FT1-ttL)_ksQYXa--s)$uBe2K#(QR zPhSXbUt%6Dn?{tZ;|4(#Jr2()r|dm5$1o#G&d=vPaAtHN18Y<-Q{3Er)J!Vv9caho zaXJ75HJsaSw$D3k=GyiJU68SOEk#EB`Ibqr`Y-^nU$HL(6B3-@e`uSHIiLqA6Fh4S ze4#eCh%nb@PUe$L0TX!qf|)EYE#)MEJAf%A6G|;NdMy$j%}Q>q5>)?HR|NnE8N;9v zV3&~)q~R`_s$310^()A(;Km%|S-Tiyx9Rlt9RTx`wU8|Kl3*mziXwwh^aL624bR<-O%!5&}6W%>qKlGsoCL@(SrW1kbU6Ii~XHSc)Ka zvA=cTn!B_NaTd=2v1gcf11gv9OVPq$WE>8eo!`9JjP1TSS z2*5vlLZpc%@1eY}y9Hr%NJI~=IqGphzga_N~Gj0+ZEgbaW zrK3kxj`6gmmKfe6Wan5SG8DPEa%ty$EO#8TEE4JiAJ6%v1_L!zfLZbD_y$Bq_K9;6 zCp1B_Bm1&Kv`|2BqG%8sI^tjFgd0Qa7*v2!dKw??I9k1}AtSNPPn29a!6K z%@&3|g{+(WN=O;A>B)xN-XzE;LFjE9a)D!ZN2sN^aRfEYP%=r^n8UXjw!22@j_1m5 z{aBm&kA^&XeJm!UDBaDvPOg>?U}H!ZQ&cLBUueZoh76jl}QdQ9&C){YDlju6Zh@n3r;abK26 zXF_aIps3@5k|sxSCT*t?^jxCcR)AkYK-a;T;L11d*za^vR>jaVQSUfLAk{csKIQ%O zoGifsH^DjfSEap8L+myrj|I;U`9aE7p^Z8$VT7g4c5zB1^FFkB`~q4&A&C@5O$i;w zb&=vPrxkv;J}9!#aCPUO+K_q($=mpH(Ekcl_E96A)|2&U#{%yYv#!i)TRdocg01o?IaQ+u>yVX$!@0Bn!kH^SZGw9CvmZandd zb%#46VWU|n95>o`7RX}I0%2UGv<^KL2%h@|Vl=(jcR5`?+n{)m^>qADq;TCL1}ym9 zxjg=fp(i&s7dlB6!goFK?)trJz)bl60_h7$$ZSznMg8Nh8h*dPv_?vkbt06X*}g3* zLv<`UkdWewOTr`f1WtiP0%KEb6mtuW5ZJ^~J({unIrr3=Dx$JeR;_IL2+M!%5z_Fs4~W-mM*IDvenUI|x4;0-GZj!}X2;UTGN+$eA>x2$SmTdC1v#2Rqdk zY(syABKvOLNICMrxleVG~1u?#_J8r4@q z7gNMD(8UdB1{LhzD+XYJUhkLGXxjOd*?9wt!LXr$Hl)1qM8Q2#m3~kF_B?W)|2s&pRUyBN^z_V<2S>BK&1Z?NrYFqfr!y z(m=qWy|B#APDuP%nY43J>Tw#7Mb$R;wOS$>29>|SErB~#%QYuw@uZj;66YG`Ala1> zJ9*!TVlxYW+RWJR$BFeVsR0t&#Eebr=Yaai1?I@@{(EJLL^gr)c$zh4eGprkRar88 zeShJ0o(!8HFm6pl%m_DnHO3HN4Jn<^FcNRNDzayf`&qFJ^ukpo&vDNo#{?fPUgnW%rLS-(G8yj977QkmN2$^XJME`|fy7_(Qox1;X7wSL$eL+)9@ z4X^8G2bZ4&DTdnp4vW-{mJ8s=zbHlp_l$ ziW`>_BUh9ziD1c6V&;f4vWtSnpqfRV{=*W}8lwLYC6ccNf)S$z|J!4gi67L2oTj{f z&D^fcd$@c4&vma=@Go!x31O9ERJpBxoKS3(nuua)`VXn))BcBlIlmv4Ep@Pa@_v6H zuqJs-$;)b`j-D7GO};Bd2tJ>*F3aUK`BpxZWeJ{$S4|P0X|M?rbcJV7zOd*RIpAQ; z5l&({BJC>k!LXoJw2UKV^Eh~<7Bes7Q6Act8p7$|a89l($>=Q|?Yk`5OaMgGgZ~&v z0?P1693|r0D=1Ku1gVIEBDml}f6(t5)yk$IbWZHQmZhpZl>>s4>aiWFgwte=gOb$Z zDuGrTt7k5h674Splr|ctqD54`!f6HJxQ2=<>S2%zrxwS6+5|=FWXtfPz^uG-zEFg| zA_&FIXzhn)*-%Mqm+>+6#4OmodzLCi6#qqf{{vKwl8_NAyteaq;@#NRmB4c3Jhk`9 z9_A^jLN=39xe=<)v+ioxzZ$$Cs;)cii3yFTGrS`|7}abx8*ky@a4}WGlEY-I4hCl6 zSjMb}TXGefcgi7M_3HGLROr4aZ?aTomN&tyoKYdXRo*p)uH>`+fC77R5|L+h@}mJ( z<=X?7u?|Y2+?H#69#=CGF@<5>81K{u7b&^mgj2#ug%eblmrT1*XkJln7_Z$Isp(u9 zWv|1qsJZhLBt$ooDgvq2kDw{t1<6y^9vUuS)2)9&hqar0>`X4glu0_DXlzDf?_@P_^YiiVBjj=_qFtB!)5h)`Kz-F85-LYYMk(vpz4q#N;Sj> zmm3IVx~HlV{%YIOMp3GmuDM0C9LfiOom9vd$s{-&B5w{{N3WwS2)pOBIiUZL(bd_Z z&)g^8TPy8zR(G11As|c$zOKmW?hZ68lW?Z0pz(hA08~0?{?Q zv3*g@Mh=ETGIX+|@>Nj{e7MI)_m%S=Dg^!$oL*QwUZY*=2k>d`v+zdtjg+XOHd{cM zmO>DcXDgFZH+Cj2b9za*~&AUWct3~K?W139|KPKZ9SNWimjRxb}A zJ?KZ&vW!l27`%fk-UIh|0#$-AI`SgySR*-BgaQqu3qWBh#<-IN!b-?x{RYO&i#ad> zI5_zEdJKS`RL6Gb%jYXUwn%DmG|%U|AC3YdU#jeXzc}}0!&tcEr~XdJ90yP3!qdq{ z2`X;QaHB( z3XQfXcFh95bV9hRlK#N{@&6#nnyZ8p|cSNI>)i z**0Q<&F%x-Qb#L4q7?%GYWA{;+M$5Q@u$~2yUg$mUQYa-Z7}S9YhgzDJ zfU6g!hHVp3N5^)oxz)IUs^y1E&76Rh?+kv7iZRqN`)wGFjpTv3Ar)1U905|qKvncF z_uG;`QSyIEDKipn^cORY$PLP3qL@9RdjyghNZnJ)a_WZps{p(Yeb_(7&as_eovNUb z&b_`Bi`9Q4R!gqgrDC#qRy40p)e5L9j{o1GNmr0ZvI`W$C zJ+#F+gum}d_|1IJR0vck4v?9HMN#_DNu>X(mi0c0_wnfbyt+5DsKVO@q2V#h?j0M` zRg45T>+h(imlX~FriPJ^cs@j)3&SKuVv`4;5K7WJwb{fnI_(gKE3M)f9=(U$Z+F|o z(rR8bBCVP5V8t}Gr)5!CKChwQ;1@SbO|BRKi*fz61g&LH;Z+h2 zFSPW`$SSH=Uy5=gYji}KdQB5y0vr6AEKN55Dv1iSzr>>5UPbbdHlz#6*lp&%0Bw7_ zr8+%SU@T|VT>T3JaVHG>I_nAiL}4f%SoF@BXtkAjz*FSwC2NSGVJu{ULEFQ}?pTl4!a!>zCC8{X#u+4(l$ZJsWQ8?KusYkUgaV7FF(DK+H+J_LII+V5(iFKLbq5*S`3caYS5!O563wW#&tEU<;zYR}ci9Ud6iW_qGE zp+c^mA=JzNF$Tbj_Vf!p36Q!{AmpO12)sllGNYK3Q>YN9`yC(vrKu>Si+CvD6hOnf zlEYdq_+30kgq*p)FKDHth(OHs?vZ8(SpuzOF-LR&L<;$j0^M=y^wIx#10XI`*7S<0 zP*pnCQgD&0ViPMFj|2wwakWT=tP(C+jy;UOpjLm6?CC{neowu8SQdN1G{4f7{R7O? z+??irM&_lxFNEhYJC9YPmuy`~Dgk&YYW-eAY__~dTs3&$@6(2QZ z_8xHwxlS;l-onhdFyr?l96j)J;cz~p6n);v3*;i9dhKH)MF}WPJfYn@t9YCraK9&V z8e-~s2akqy%upkH1(K92M2}Lo=cEnFKn#Lo(gmkgqpQ&facJz|xccb+a#DNjBL(}| zs!2q|Bz^;6$%1UYBiR_Y5ZqO@Iuyyz4Liw2j{mZs98d5^iTLS4XO@VQpQQDqd9TWI z1k?yqpy_M=+nkLc5B%gk@%&W(kZ%FBvl!a)9B6UO^wP&VBxx(FFSC2SF6C&_ya1U| zXgu#hZVruA=aI&X6NU(gGF($M*lJUahgVj@buH&x&U8cy6LI&I5h#UyAeD|UaU9toEb1*Bz1%!I&+~|0qj0#21>rGo`E_I8bvD&J z^2T8EZKNn>=`KZDI>taGaf=+|fecl~K`|RwFy42((eQ&uWO@SX+R+Y0{`HfnWU^cL-^S;e;fZulZUc{lFd^ z(iW-JKS)PhA$Qbu{n!kX#^4o_EvI-NI#Yg?<|X>FMy;*+QDtqQd($=P19&~WuNs3o zh*7}{f54uY79p}sZ3qrIgR5+pF0L@G&v8YJ-F>`&Li9fjsj!PLb$K3cK{(%5Rb7}M zZ&KdD^e;9A#q}n>+2nlVab{$HB=QGuVDR0o5G3|UQ)OamcQd#<>)vYX-l{BxI!t3~ zx}k?mWLP{VBB)NvAS5EV-umS5_f51*=U+|f-arHA+Gt-jR1Jw|)PV3*RgI0Jo#hQK z6@fB#s_m~>NhS<;CO>I!77~Dvxh0MJf!lH%(6Ha{@K@W9g2g~4i*}rA7}zslM$!QK zJ%Dw*yD$g)?HIht?a&VZ;?Tg|N&XbZs9E{ASe|3lt;@F<#SLip`ch3xaHuL(j2Hv% z)=M1ey2Ng<3mpd`GortYIF$Lz@gX+-y!<{U1H{)hRyq5_tjkFT$@E4?!ZEx~|;=cB?Y8vr0`sQ=Vof*YU14cXy4O z9b=H=E7!&`FVczxd{%nH4VtUxGn+Kvpot-)*=+poXbF+){K;pP$f5J}9MVam#+GS| zn*0~5+sDwa%?y_zZ^HJ2w)jeqJsvBk_;m@BJa;2UGmkZmn;JI4+x*Q+Ib#o_@df!( zC&Fa6Q<@AxzO{gw87!c@_k!K8$wX%kWV4z?zwRT-S77Qs!+4?|TS2hSZMy{R^}RqB znIiR-G8sdRZ&*1#y7UFHSky`_b?~Wtv4{mZdA7jTKW7zt2%Z>Y77~F@=WK`?Db4y_ z@mb7mybtC6C3Y<->+Pg@iR0Khvx{r_;l+@y5{! z^xED5+sDDAwCm%d_uVGZ#|`nK`@i@bT(s()Xw%t5DViUo3r`Fli;JJkEw4*8_LCVn zvqyWPs{N<5xHg%*E$1iK$E_O|N~;wwgHG~?nh(F!L$(>$?t4~FeJlJi?^Xl3J&EkNFLID*A^cK9ch#sFy%f}(%_sX1{^b@I1A zgf+ZiiWE3c`-~7(P|J#9YOqI-r2(6h#v?uGGx0F?pl30hCh&94(YmN?E!_wePO+&M zHN9Ie4DT-~hU&0#SSAc?^zhOvLxrrOo+tW*uKn$e%N_7RY1}QdtsPJ0m!D4*iL1Lt zW3=qnFUdIETNTi7lIj)OA8^XAYng?gFQ?k?AK~24N9TeB{dL`stRO_Y!XyKD(`YrL{aTyj=mQDne-H5ndKv&bC{aEo z3(Y$i8+4Fy$EwwOfz~nSwUYqq|FD?@rc75LI;550w7+&XWGnb80QC$cQQ~5}?M`^+LncUAB)O{aXqO9}Q zmg4$G*3a+!=(dk|wylDj?a=rN+hE!*iXLe7i~t?v<{sDh%U5DJxDW6$9aVCqf4)Hi(-iNui??}|!OL%ddD8nBBKrL_ zgF@HMBghDtKoRJ(v4Egk?0Frgeb!#KwN$F%X?X)whd__AW)CTmxXu{qLZx065Xd06 zS_Gef*5}S+en&Ry{R{jr=c`E7;9<}>W!5IVQ9FW01Y)Lf;<*9X!V~QNsptXHCTni6 zB`tG(*YzRKrE3RFHXKew3k%#2DLQI9KkL(~5s^8q+myvL+Y{3ccH8;)hfViPnlgZQ zewhFDbckic2Ic}kz$NvaP-zy>E58xM!h?ZMyUGf)wC*c_EGmtZFez!ZE<`%_KEc+> zQEfDTU+Fqu<~NUC0+L0ZcKNo4bon&*$#aS@_zt8hTKlD}L_s>y;bR?;&w}<&H=N%} zZh;eW*Qyw@(>bivp9IGzJOBoz!2=hb653&W2{uFP?jC>$ggJB>=B1e|5Pr(HBRbH>xL0{`JyY||7-Hh1 z`Wb=x=cJ}fupc$+0L_HKEt}yZP0A4)MeHAY9K1yf(%X4Fy z!EO69gNaAw96Gco8YMeQxqZKvY4zfz%bkD{+e7=MJ!O#^o&}YFtab;s8V@LE)h0%qZ`ZhhM2sb6L zH~KA9P{_*09}lKH04d9P&32GEuYzFKZ;pCF(=CUHwmX|iJH1d4KM0il1{{%{FA`Sk zamBoHDK~Uxt3d8$Sy;?u8&N0;jNKteaZ;iS_V0I;NHOBB7D3t`=2l9SZCk4E8}GYB zA4f}3o^MT21nzgOksT`^%KHmDF^FkgWexx6C)q;Nt5A;vL;GK=W;4OE3FxCF?HlVK zD79~12wTsuS|~PDO@zj8Jl>81)X=5A2{+(S$wsC-W8>ST(ww!;R$P8g1xG&j zg+z6pMvM}k_+uEj5t`X%*-({@eYRNWUsJ=+3$b zkU>;zumbig1wQbA_bq5%o7Tx0)VF0ej&CH#8Qx1?;kH%hF|$Z(w&N4Ikc zhnhDaKiC$rY9Lrdw>`^%Ecw01SD)q)AIv-!Go>qpBEw~MbtAz?hbOX&Cq>#R#Yiyt ziC9Qnew6vPp^@mhHuLSSDwun2&Ss7)(&L#%D;v&yv?F3?_m_HEHC5{x2!1*)-XrDi zT#Y4NsVhDN4alHT2#de$D_`XM@p9#RmvO_UeLoYSch&HS+cFXv5!x>{6gNn*M$Nty zPTn>x40bvsv6R@BW*q~YF0VxQ9`H-2izXzR$uzcKFZbH%AHW&;iV3ZCd9m2(Q7vo? zX(>F)6w9n0L{fJ@s{LuSd^DuhdijSL{xGgJnZV3h1ad%P-yjf3ECctH>96rFB=Z4_ z^9G)c+!wY{0x$=^4V+iE*n6Py)++)GL540OzsNNmc!VLCuzyY6FcW=j+{t=ElB>N$ zA2y9_+a2upz+fdl?PEs))QS-Qu!JHSA9_CzV8}|VCFPI}5TpY&Y2;fk%9}O#zF={G z9%lHEqcUK&t|=w62GNwAtrQ>?$-V;LoGO3z%+HR0CW(`HDk+!)PnTqevnSZwoC&%DoRJdf2oxt}ME zwXO8_y4pv>f5)<6J4K8X(PGIESWRB9Z2osfZ}|#=HDb~g(fNY@NJHI)l1Ty}e5<0( zgtYhMn@M?T)I`NWCi6M&i|YWIEbm!#LCsCzo%%VqYNEtK?Ymmt4~WLCa%F0Z(!fF+&_TcjkIrt-mKl*@#r;|=|kjx$3B^UhDW5UomyQwwsqlBMPoI`&tKQbRx^bUkAJ`@`zw;8yNe7LtRpTsUD_tOBIu z=6!{8egYr+0d{-WcHI}iTsEIrLt|@jl`P{a&b=K3O*lj`={8|M?n(|-epxd+(Sx6` z@$is-ce{!2c((XiZ4o?N&fxyyY90x=7cBuMLQYduy`x?nOKw=91zv?up|I@czAUDvL}57*E=}73a+c4Ed5YGCOF(L=FCE0fe)Fjn|EW)K>q!hi(|4KK(1(f{$ zCSH=^w4cH%mV{lGK`ux7f%vn0?e@($zHlM0Cg!%L4m^xHM$*Z>UE>j z@)18z)*rv}hT~*sSG2Qh{uBGEfXQi4O*EnN2{t~_3L)tCD4-Ey1q&&Y#X^6xTd1Kb zUxgG4P^ls^>gE|rzDw{G=O9NX?uv%yjg$$7+AS5nf;}Y@2emP41Mr*YG8KZTmG)CZ zVQfxY=qqw#;Vp!dT4_78y+iU61&!k#nOU|jIC7(bUBPAWFi@+Ao|Jq3)EV@9P|BMd z?6pcfqx~`&dul}3N4nw>eZG07K42%4E)n|=f|3we!Ohja9O?e{sC@Me4T`VZ-=Z|} zJBe|zlvzOwl|o7FkDRI6PJS$Cl-Z&9Kbt*}RFx}W3;T$Le@5J-qaaPKdcN9ar(YOd zYm`-YP52BPPklb88%=bM+uyEZ@SPWk0|_Ov5`|c~_5(#D0DnQnW~s+t>a2Qj1GY~lp zybeVzeITwZR1fGFn(JQo(rtH|KNvE#ag(Kz7L}0a=v(?F!8KhB7S2js(bQHIzDk=r zo+|7uLO;@z6BU9VqQ3*+;OgtFy90lf@{!4{XDdXbqz6&|oD;{;D&*64`P;x+*vzUR zaUA{P5iD-*O%k$v_>Dlx(o;|0;N>PYR71V5t<}UNBLK1nm=#E>_zumLE11#{uJ|= z9>EbJYR|(cz?|V=R6&|d&XDO)ZE@%2tn-ilfpVuyt9>>U8}MpExBk`1bc`JSPb}Z{ zR4^x=EBWi1on)7NBLJr8YH^yVWd1XFC4I8E$Mc8Y>71BFM-1`Al&i0F{cn+ zeY%u4Fxs>Es{!!2_hGh9^w6~ePtA00qswIlyppQGA=MQUC-js`8ULS(fa<36z&}W4 zkzM>@e8s13{>k(;=oSY28VW*+C{(;2iBd#!{?$h<-_&`*BW*1^+{i13dn^S^;INH@31 zHQ_@VlrYfe7w#IV*kWN}T|V=E9smsL8_pxO4gRaVfH&_yNKa)m_*%UYs-(wB1Pc!L z#?uGC9WA55BtG``#=0I&IU1gvoVp1_=0j{TtKd5l(JzCo{^y~5;hrjQJ%lr!<_ex7 z^ZC_?ni<&r~l~)bp*?%P+ooKGqgr-T%SJ&5OVzD_#E*My{eD8SZAfo|W0~$uY0$!6YeG!X# zgew9QIrd-z!qLwzu$<=Ug_dI~n+l<3bz0b0_9Y93=zq;~)Pp>oSu-9Q$dgJ#{7>?m zdhF6TYYFbQw~YZtIe5ge=qguNpt>tIc-Rg5y4x`xfABSTfNp^vyGi&CGJwon-{;Rx ztA7;FxJ_}}T)XS!S%97K3ksb-7dUpuE<0|&&CsRQ?!7h{x3jHtRB;`>W$4cuvr$pg zngDo`u8V68)te+zpT9OBi349dF#sIZn<_M=e{?gA`w*BseR->R0)7)+NnzoLa+fDp zVlD+mp~xVVK1mO`JOFCezC+bvrBXKOziSLLQrRdt35kBkY@GSyaB=c=HDdedO{GTV zWC4fZW%<})C_SBbBDaDbie|!6t@-)Hw$gGgWcquuBMC|e{Y3+6roNV%#@fpl=m9*; z7%-}lPZkzP2*;rCzLTi%FtuhHFbAiWOiN^CW-&{*JPpHoF zyCze$djB0uLIAeTjm4OXf!6I^P+7lfL|Q(H1lKBqx;|+rcHcO#eaKZVo8u3yks30j z_&ABAU+>qaMId5--0z2U=5@*w`?J|yr4qH(^yLz3ZVMa>UAOeELAv}VMvbBLF*hz0 zq-7!^;GCRK+jX3vNopI$ZRs)P6YHkOLDp(xGlt^0qQ1ND>s$qUfBSGbXf7>0$4JXa zPQheP#=%L&$025v{Y~@#MywZnRWkgdVb)1`Nw>;_u_{8hhoCmbN4VPxTGfH8!+`-5 zIAU@Y%0?rG;UXCm1m7m`ziqg`4Sy!7u{O*%{Mgj@FSgBEUnk1IZ93PH-SBBm3;FN6 z#~!(*K%@$X@sx+N5?Tn0S;CtYr(4U}KV)$IJ*p4xGZ!s*l)5xH$ z(7sQGu(>YInf`f|;7#jT>&RWT4(4T*Clsl#7om0+<8+=BE@-f6WWD_YMG|_zW7iX) zp57j{Kb`=K9WRPbwC>;{0|gYVo(P&}?sMl7#h{Nucb|K6I&VPXg2=yHz4N2c(y8@q zVpOCdXU2+;xMZULHyJmfvv%Kr+Wmnqq-&POw}jH3nrSK|tSwP{_`esS)q}&-d15EPvlIq@AQ3W9*~o99D~44hjZoHOtfn{^s|q;^2|3nVu^Bti>IPF)C0O z3MHFZNjQL^b(k}eplfg@sk|dTl`qSgy$;uGhWsXmvCW}eMOECebUX}IEr;rrRGTAl zHAu&VhhL}X+#ACCJl=a*cj@Kd?ZlegWOkXBJKn5qcOiRD#<;7e%<+0lmrfup(b;4q z&E?L>!csrZO7aG#S&_Z%I=JQQSIU|a$OO|MC`A6?unnG=2gBO`TxJdq zuMM748n^$kzIl@-isL!mV3pk5Vox{mo-WVq!9BS7)H&N;;Z)yZOm%QlXj2qFfme%) zF}`T|nU*2-gyjYjS8TPpdC_|L5i8HR&NH_WV5u3SNXKDg?C0E)={4Y*o144)d&Mwy zIHu8E%*n0x9Jmsydk}#65G~#$_7La|f&|I9>BkBEhKg!cizs?@FcUd#sHK;3YZ#CR z2U%$oHGUJkTiDe(P5v*OSX~P>W6ADa2AGnqB7~TriSxpog(93u^iqk8`cjVyRVDGu_N+sm!hgP`H8CRA~Cs734(*J{Z~4+7R?7<5h^lponB?7?vO z2SUyJUJ5x4Xe}KnKHYSbm#2!(@>=&f>Je|O7fOUaR7+$1a;kVQHU1d8Iq+|&%~iZf z3luigB=|5+V%{A+Fb-Kle~Uw6VR;llsAk{^mgpd;T9uW1GvvxzC*go~*_ko(b}c=w_ohL8AePX)LW@T;?tl;%nRvQC z_2n zxJKiW+))p{&r8#9>;%@Vh0r6W%k`VxvLXqybpkETQ}*?hvB^AeQR^Z1@YwR3gJVu7 zFi=>-2`9=e%>}V)_`B+l!jaAtYB^=Z9JCAA`_ToN!8Qen(Lkocfz5i84XR-kpCwpk@)G!$iJZk&!)YubTYUsdSg>q`&D$61If!t12a!D!gItV9gYyyRe@ZqS zg*IOg_skj@Yw22633V)R8*{s*U3>b9QQz^I#|6<_A9-6$ zi-V7em%azM%@vv=PF658k;Di;E1A=X1j}#x??OoxGliZQ5X|nm0;)c4B$G=`a7?-S zv$^CTp9M;jlIU~b(S$sBbf^2(KQwf#jmp$75{wb+wVY%pCEfGfJ@oYMNnuX3*7eJg z@I%b&-7RBqAO#a3UD~Tv>nts(eIW^K;~BMRe@8`_<_IFY zW;)&O(Ypv{196bl;|Sx~zvZ?levZxU)!5mWIN+?sW)1}f%?wU_!>pj%j5VAt%CLT5 zZai5%L(ncmup-}D2+vftV~v}gI9_>fO)GC_c37e@H2^xKT)y{iz%KhLnt~0wQkJcO zR^HZw8&Xn3a9hc$cEHMQK(rgx(3V3J(xhgjG<4C|XVsi+6bet&;PS8}*E_*dIh-Z$ z&&KBFQ23rL1LC*0=!rbPSJ74oF$u$W$e0Rst-`gce@sGAmNjo5sr+>7E@sLgbfg&l z?Z#L=CdqsyOe{w8GfPsItSPrmFakSjW^hJ|k*9vZJiTlNV>>)N(XnL$ODOfHV$)V8 ze1Vm){-_blp&vQz^|a9vqKbRf#y$UqQewU+kV8sc1UI~L4h!*9b8VJuq_AdeUn!-@ zu8TiDItsYI`DLx0;y0iUzI;u^Ku)O1R9~ax4Eq{cNn5%Vax*~ASMcLs2K_eVbs#E1 zL!P6vz1_GS&BAbe&*5Q20@jo>yS0V`Jf0k62Q-=kjJ-@E#tZANeyOowLs(azU}FNq zsKH+ehrF_DtRs4PX)KydwGI!9Htv7c+dRe7)po=`I|f=@=iuBJb(K_hwG4xgXXSHu z$>6{`L<$#^q(|13jpABC53*z%Y2nL|0xn0n8#!jVhbkflOYg5{ge|1E2p`fx{~V5O z*Ej)t4yxSWEz+Ig!SQMKNvD`w%=VLw(>OaU^bXe_s}8@JqzI2y(pd5Q1k}3i zKO3ht8|^QzTG!qK<@l8NSSBRz5qYO2eTLIH#LsTEZpG3EmWF^QR@R7Fzi;}TuftIQ zyFC=H^P1Prk3swL{8lLfJ<2}?+`TY?jp-bXwA#uQFUwJ0E(giR7xww37Y&mD{oT>R zK?gBU^Yo8X#uW^I$hW<|yOZSj!;cRvip-Zn-t)HA4CR$i z+}zPd)n+3L4-eJCwg8b4@l4D0c9EyHHRDd@%=bWGd`4cZbjyKbg?(%5nuJN<{)=Wo z>^RJ7?P(kRN2RA#((Ao=XD}lC=}bH;gUzd9tHQ^_y2tU2>FGG;9;igHRr1k?I_%DV zokQ%^Akg3~fy6bpT?&*uM>S11dEYJ?Ww`dV%W{x_iOGO1tw$~Ujpg9=lLY_dcp1Rk zm37sRr`7mMm-u#| zW%_1kzDA^ab!b{~?WwFRg)3Gm-T5MQ^C`s||GuI7uB67NCmn}thi34yX>jAP6;Jgd z?^jVO7N<$W0VYTFf)*?1OFLSp>x^@!IsCZgr2|`RXj5C{)LMTc>qbJqM1^zrVC2O? z_l}}9F-sz0Q*(_38R3t(#lVLXy72F=%|ttmlS3g6qm9@|al=h#sx;oC@gEvPrCjtR zE%e?RGBgOnI9i7K4plYkQK+oW0Ik&FCX^zad@=;S51eczaH*n;6@K^SUpS|czI#xD+>b$IV&y{R?-byGhpLX*C zKAyTqm_!ODUze2u4YTXko;{zQhK!z(?05GS?&aNgo9ol7y7$eiZNIc~$?YJm7pb{A{@Fh2X|vq98Qff=&h@@|R9JX| za8+r_U?#FHdfG5H)}MuDkb?VA$?G2r6G`b{0v1SyM(r4wr`dkr(h%=JgDQZj{{F;1 z^L}se{pyJU!D(fGQ`@Fb5wU3lmcq+b?|ET%_3~R_V|vDe)A4b-1>l;@9|dpHcp_yTHHA76A)LrP@!w5?z088`y(cKg2!O8J-$misSD~PD9foB`Vq=nuxq?4Eg zln_6%EFTIa1_}n~HDgAo>RlhmL7LE=^q60$4=MjJ2XqKE$6*~U-V*ooYkNuZa{t?U zv5&*gJ11pj4bR0c_nmZrjD->INLbn3WadxDH|5-4NCceurnPuh z$BMa5?cf|DXV(p$r8BgOtzCL&6Ewg=D)(U&Sp=tf;w?t6k(`d4@G(Afm$v{b26fgT4 zif3i&r8HVKenCTz``k|nziU>+U!P9;@oZ zc~89rCVO~SL3t{igg8TUn|iPWgEJ{9B*&6)Bf)rHC&Vpoz7Oo0tay_5Ck@;XSiEF$ zfAK0MI&~(@R*#nHzdCFu-RA8CouJX*16zO<-tOQ1rV>AT)WkU96!jlDLq&CUpZ^xv z5*|Z%rvGC!lOxn!&AFbV$p{Fe>9sqgq!@?0~KYMTuMk3&L+fu(8qN+bB!{srUpM%{!n6PwqZH$Lc zv`S(-jPLNg@OT=AJD!L404)^46t6eHt$sR7o&~tO`=Hz{$D$f}JzSE_c6{jV)ZL%5 z0^E1!WcRd$kZG*^%J(fj(3{QvT*@8no@O(Lx1jTfnfL( zrO*7X9|)V+cM6nr4u~r8SHG+gri5g{@a^_HYhX8%vtM+9THm!e&k2SZq7@S52gY=z zIGFHh1J>V~RFtH*drFe&U4!Qeq0y&0ad_kLwRTs3Hi|;UR91^n^zW?U8P00}qcdd) zVS_fS{+UV) z#+|jGN|W9b#Y1gJ$4qY|X6iiJ6z1oG2P{eKpUl(y9fuXI`99fOJdc~+>j;a^&37@= zXdM}ho+I!w04wx;jPs?qT4%N~ckyvE<3>v+)0+Tqo@>%GM*Mb~v)8RnaBd8=$}_L| zC6wNYlck^e{Ov;a#+Mt-1YmO|%0$5Ao8JaEx`ig)ETf6%e3a-gZCnOJ%DV&gCoAje zCZ9X+jSb~m4c6D(L7`Ed*e~}1*OAnU-xA?fw$0kYrl!Z{+`=e%o06CDWHQTxY)|`VeLk{JAEc}u zg7Of7$Gbi7N@zpQnJ|$age=^c;-N4DVCNZ~d8Bx)FB3^Wi(@?f)RyJz%^VO!{+DSp z2p0B3XMiaD_ob92neRHL%^ZOovOFly5?xc7cc}eAE-6mkHu8E-GI7Y1^RU>9l|izg z2lGnFH^-k%P)P&+7t}1rw=WdB!Pu|9kk+>^=bVf$W%c$; zYRP3v+~&<$owjuHwiCmvg4Pgr?rfI(Ji;O8u^ip6bbBeE_xcCPy?Ag91#y1UDlr45gqG_K*TEY%?S#UZh+69tVWI`P z*F4=1$CEN*GdFME!o99mXt{Y+?EbbpPa-bs#HPtz-eKl1JQ*wX*xbIX<1ONpNe?KOIG(72 ziu5d7t`;hd!#wr2HeUV%3+UzHa_*>K<^2{GORiRDaGWu}iAA{N{K8QVk3T_fxq{=w z7P#5!9wpfxK9wrN<0cz+u&w7kWc}>5|A?ObTX_87oJ(zI#w>GXEiqlW${d=ms1tcX zusNzu`&)1SZk|&?(O#a_2XD;{oY=8GA20_wU9on6k2LA8jjzsI4v4rh7Oe1@tSy?0 z5ZF;uOo7^3BuMApZ{+mk(CntCBYI@9Kb_$a;c;T<Mf_2 zwn?;BZ_a>c`_J5Zxj9|AxVZQT3(THYvp}(uJ`gF8(DOL|C1Ud1vPQ$xQ$QY^7he1~ zPM!*_#>bK6m-iU!M5(l2&n3WzsvD3}`|y+exw)~IAy{e7^-LVKZr)G8f+UMdW z2i@rtMI^?7H|9!(hiIy6=d&qRmY$(qm z=t1aWv$zmou)=_zIT8PxB5pGG0l0Dh)1wl46xzZ)oAq(^2I?VB{0FzN+M3kHmaGla z!=;^=<@bzK*QYgjhMOD^WBHjwePFiO#DXR3d={d8b`cQnnMD>QeEQJy0$1SIV=Bsp411hzU-ay*bRyDWr`w=Cjzb zd79tLs;s5Cva^`Naza}!BY9#%{UQfr=^aO_`SCL|cxjFaG-R%06rWlgmS<1d{t-D~ zf@Bj!T~ilD6Y*DG$}O`68S|oVLYt4j;=qtmuJjNbaoyFr3QE0cK7UWLef^0jrG!y zOS)ZLY`nXz?<1(FYpgvtv|irhJXEl49PsSm-FH(cJDuB3`bP!7FE9XJEN+f$gCgcM z`jrmzl^rFdO4S@F50{}|&ep4BO$@3b%N`Jw9f~)yO!=j^-`lOc_lGG4BvZ6NFx_H4 zZikKOIJGc`7~D>)DKa|`Gn%y{RjY4;1H_L#H;D{Ll~Y)}?10rg4Ipm0PIJ)wvSl+v z;W@=irgnwnJPwDwcD8%mo9!(+uE#$stC!wG zrwC>0b$iPIr0-mf?iv;BQHwSj-izRYn3FI2yldX0=b4|c?Lsu2kGQOz>+%+k?w0hK z7OeeG*|E69G_w&X@4xqo-v%*FOJeVlxW$$e-ziYKjj;oRs&z)A>p4E=%;M9osLVdl zMx`62>PGw9##7OP!0NL2DA;M`)n$QJ9BR@9)*dg5WH@plOItb~)$z0Hh$+npqS3?S z4BA+$oP<&kGg5|ng6ropX;cp<(oB(YhpImo*^fH z{%~H7`3mHb9#$+BP8=G_5*T{}A`(M^njawBNP1l20KO!bvrk(?Yp!IjDPF|ytj2I# zw1&4W^D%ogd$5uscc2oH_z4DrKDr_@$#$rrH#fe)>8!1ddCU~$ zZCo8iOH9QP<#Nq-OR)j zp<#RXG_jCjQOzd$->ZKM5N{7-vDgG6_>=KX~^anmG3ra(uhs_P9cbbmwYMRDKGUnbK)Bgi4R! zfKpp_6ZHQN$v`&0?AiTbyNne17Ro#I0a3xfuQO%bf8VzK%WD>qQb##gfiRXUMchf> z4}!3k5O^!gfmVlNf5Rb%5_w|-gtaJeH^9CRgq<>TwhFk}Y`&ntG;mFaCkP2TF=>Pl zz+%V59jMQ-1tf@}BpC$g-@=uNLV^v{7G9ssqyvlXBs~&Hk>rQMoSPJoC|K-v*&ZTj z4nSD~e)0t}*bY<5JDVE~2eNV4!hm?O%>%q(z|X;Ogyyrw>I3GfvJ?-iu5REPwK}Z1 zmQbbm_+(EV%2a|3NHCG|i(ObC55WUOp&AKk)s;wqv56T=$Y*sq;@oL9lc|oE^jrvE zAdb~%wOJFA=~PuFRK{uL05Xr=bsyiRoZsC z%pR9qg!q{F6;e|wrTPOtzYHq029gU$$mYO^6*}N(null+HO~;LhAc^`>C_J^v!#rl zG@&JAL+vEwO0JTQgsB8liCI{Pgb_nVF-OI!H7qta^AAp8u!QYHhK}B~b@M6p8)fd? zvS|JiygFQVMZoToO-43be5LA%R*O$9xKa>W7&6lC>=d%0FmBkfF+i))BD9jQ0*t15 z?L<-fFaNHO#|%y2J|0`;1S%7q3P%T)i@F!#fxxivO5hdzD}5O=9l`wEgS&TEs8lhf za^+I1i+<|_AOk*Y;Su(X*w~{{Bg5`+lT>`bzjN#6xQYoCQ&WT1B%9reG#S<#oCBbJ zavXseJrdXhfZ@{o*kMw+O93Qvv4AQ7WY`=QpI^Z=!-sI?A#X7(vo*eu&M?O0a|`~3-)aQ4AHg++Pw>ekM2CNmzuo-5p8 ze4!X1H5&_s4h$1iS!^xY#k`*yU>I`bC;#)O)&MJ%nP|LK88*jh!;>){IY{Hf$Uwux)7<}-qY zrW3}T2(XbT*sZW+@PywlU;4&31QJjC?wb%51><#1@0R$`bc_34|I$T<0S9huOYd|t60ARPW zpB$7yRa)T`5({}N96x_VZi7sr@TTi;xc;BFwK}WCUEMnyrG-XOQH^uYf8m{vZ0@+c zg9je|S8w0iwGz5c&F5ZztwEDB{iWXLAMUel&UBdSDyKJoe(>n{#3YzZ+1f+O3bkRq zZ2+PeWfiFpMHNjqm~0zW7f2E|lxc-*MTak!jp7lyA(#ka0ycR6?YHGEnNM=I+Vx)@ z@ShqDo4`ju-S6&g(uUxwHt=sQ3;$0B)o|j)dr`ye7ql>(Q zUI&uzkY6}z!;Z)p`&JM_CLvZvJGPOF^(x-dPUKF%XV1BugCiT5L~w(TGDzYdKIntV z@4RhzWZKc|nYTN3x+G-t4;%97gx6j-Qd5nrLud88w$HOKQMKG%+wZ@lmywri6qlZJ z^|k%_Jyz&RU>uFJ`I&SuSW#mtg4Rp1;^r%sz zKKXnE7PSy(Q9#g}f@wv2%*fJ=nsBD5M80g(nH2b6C_wLuuKU4pM)?55x9C@Z7aZqW zQt-xu|2psLuEUou={Mlj$?px^xpA{I;NP@#$)`_0`p|nHj##+-!^J<;%%~q{81KCH zO2bndk6*O-jW0(o{BqO}D_0w8`HZQ1zg;+g&SY!2 zIKLpL+ciCgf464%idD@{KV{h9_fTFNKX&xC9b1RapF3>9!b5q5%oczNu#!WWBN00|8ENs*Z}z>HfR*WPs7 zXG>QPS+=2SqJ7L4ABV$Uj@L;_uQ_PZ%8?6~-O#7s(AQtH73ar$3*UO+z8;T1HR0PI zZhGX=_xe5R%P(?-2<;Eqojw~sx6c;GA}SUKMf`*8ILvqvL?{G(fIfl3FfwIWaaeAV zGufB~P*V6`R2Pxj*ldl@YW<(-b4M*+(XQvsgWh<~;VZ;|Y;j5W%}H~{%v{iaxd=VXF{g;Md49a3&Be{`Zm0u%_h0oJ-S{M?{T$1|NMb<>MUEhuu1&} z!#;Yi*@fq|X??a2@dzrt;b39T{>`&zJTpYpB@O@pKmbWZK~!|O&uXuKO5>_6TP>V7 z`-~Q6TJ0`o)@a`C*|K%s^r;t|cizN_6XW6&dfs%?wr$&H&YWDmTFpzZ=-{NbVoGw0b6UaYRY^^@)vS@9pO18+ zZo@`(>YswHh$CpUZgutt1K+`>(&UNXbn0?NVoGIJ%r5KNW6re6ci(5Ztqcf&(bOgLXKmQDF|AJR8*c6e zKP)Dp5a`HqsxbFJXxsDJ*h$a`HK>!ZD{~*dsKq?g?{R_lVDOx?TMaEP%+5TxZOaC> z>Ri#aTXD#^vTL^?ue`kc+r<}l=zt8w&){J(#*7)$pibRQn>K9UvE_mbFKl+^Ib%nU z*|&H1Wu32Ra9T6F(-}CDyI|qMrOUsqkXotRbvJVLaasba3jw6ovOd5-(zJOq7juNt zqQ-UW&dWRyK&s+&^}6G3lnVoX->HpGF_J6p+P1Y;)wC`1=e_#rXXwn-ZFp++W=+4H zJF~}m9etSSb~vyjxcu9N1=-ngad9+9qf;Aoy}J9*f^%AX!oia5TbTirpyv>> zSFL?Y$^Kn%rLc%3A~KnPf5v}a>w8;IqvXiC6^owkeQV#_t_%B)e7Ah=!#8$MDm`K^ z$bO*LjjO(0*r;Beb!%25)kmzYWV=erO*z!EXK_x}#$~fdeen96X|r#*sTV?C5tobp zN|?J<%a$2wRjX95m2>z=LY;cfI1e1DBi`ewSFhkmHUfSb4AAlj_@<8?_0qk!AK19o zpOrb_j@usYb>rUkD=m2kpS$szgL}4wgT9ZRe{|FbZ#S-=aqz&N-0Z_pTiQrwBrwCs z>)A!gk(4FtmM)kv;`5lp36|V1seCIqcgut~YXXS~PD)GeXsj+Vx0; zYSpb@?_gFYauJRjMiPQ8s!dDh4eERUxZxvGJ)SS0?)&idSAR2PP?b36`+a)P`FfrP~cRaL~J)ivvW8@SP>WAlfORKw$fpjs2hM=i9UW^b0!IsF5M7aY$K2 z4s6}D=7&$8fBv4g`Xge=&dRBnR6%fBl;O^*9-*r)Zx1Ksb}tf^R1mFfy+C0v87c{t&-*igKh3x9)9Zbd!Om`=xYtm zZL?tVIFDQO0}G3b(P6NJ3;ZRS6;f-2ENp{DDgkQbh7-2399ZJai4;-jgs{vckMI+B<>v7uy zLl&%lf9Aq3Cr%qS=>5ci*Bv(EL%vm0M}GO@<9$CJ<7(6j^CRGuzo^9S_qr{mfx=9q zay37;qXJ=ALTbRdU4}KGX7hG=wu-3@&wwLsd`{audzGYG>`SyBKD^hTR{OkOcYW~W zV~^Z=!}hhy9CE-7hNp!#%vIc?`+^R*y+Nz<2haZQ?L|90ZsXwx@1vehe>mO<9oW3_ zvEFxdz5o7p7hUPMC%P&o6~M(CK^H5p0h=%17nfR%6F=o>6+;+Lngc4_5 zdebu-o!2qXk(%SEkkRz)OxW14zsvo7r;nd_eYY#;jvMU?m2#9ZtETKy6eZ0t!e~n= z+;ei;XOrg*UbVf&jeTCZ?#e*sE`qs&frPw$FW!Gkr8*66d;EEhUyDmfV0$yFLO-j$W@vstgY zx;xW_9-Z2M_WsL;Hxt1W!u0fvnwbZ8i@7eO@*;dAIZ0905AxWex;OtE8vrZQK$lDYAupB?WmU zdv?^US(6IEoKjKI-$2JkbKs)gVs&B+i$AmqxQsSS*ywoC1z*mYb=jqtzVqbc3+Bxi zO+?lwLjbpslu9ri;1mUC*jvHvyrz2>(q6S{l`ZJ^7a#f8y?1*&uKusT?i3|RpTX!+V@Bui*}QGlqUAHE zw`$kkRWZFr+Y5%i|E{Ajch&sa+25^d->DO48u$g~1Pc&!GbS(>R|*?k5vDtx%<=I@ z%h~GqZqb4RyLWc#-1+qL+wIx4lkb~7KS7M@Su6-)jlFx={O7g?mM@yMV$Kvt(c#^j zH%^;0;j->mu@`UUyh(50->diIkJYVJD{t@iBbj?k!=X-BcN;t8ll%kQb9Qf^JnG9! zueu8LRwZ75ACxVWkPnPvQ;k4X*zRC+Q6S`y;-Y%zD9t;#X!g{~X;p6RbvrnpwQJXw zmCHtc_7OEbPNFSc|g9j2P^yRPu>gVXMVS6=Hb<=FX`Nc zLDxh`lW^Q@7spuV?x_eDh_O(MwzRy!N82t_isk9(eSrzW3bLrS-Xn+w;(?1FC1# z!i0q=da(V3?k+&dNnL3F0tN7z4lkk%xM38!Eq3Pb%UAp`^^=d-x1QX*dEb7|2@`5Z zUxPAOQs59PCVl_?_h+=bps1+mqi3HUkeh3yrFQ6iMb8^c+w|Rv>pMyz+h2SArB_}>zH9=bNF*zB0jtvoyvW8!#3qhXQGTKBR9=*+ z3yO-~eEH=!v$Bn(qziAp?er#1`@j6EE1{y(ZeweuEGmM>UjNV`Lxw*0lo97?d2xq3 zdfmxbwP4XA<8bbr@e}7vngu6HiQb+vJuR)~L;c@+^N|O}aF9Zk%5VSYKWJ*RF~=nv zlGwNp|LiCYh!PFDm=;$Y%H(JT`oaMy2=`A3KN&jg^}df9afuDjY2CG3_eD!rEdO>% zpMO2*vs%%U5q>)um_L91*Z+Q#iewmLY1awzIQlkpSteP6vm` zD}_bd(q$|D{o(r*Hl^9Ak3RV{W6QeLtBmw&Y>7f*15)F-#su9aYzsw{6A@SR{?Gbr zZov7oD0@7mD|oGYnC$Qnu>qJV9FlCW4zp01pO?$IFphY(lfd2qa&%=`-(z{rX>+>b z6F_FMSb@kQ96ie~jcWcG_MmU>gs;9BGVH@IM`2Rbg_({+SxJ?X#6TTqv4^cLc5w$~ z`!`9#!?XP|Y%Z>!u;2Igfc{k~r(S>4tzKca?Qp`P#}X)E+Yu@=C|ojefX*aH+sXt-*_=V? z1%mm!e(bSOp6skF#3sV-he9O|G;u_09^;G-@dz=>G9IAnE^)cDSkQT=Y%-P*x#)=3 z=ZjBHaM`TBg8UnL-gxU>z1w%X!Ygwsk!{CNTf{-|>Pkkyy=kiidYaCp!2d)6LG4%{ zgS{0#RJb?U-3=mG3~wmlPD%o30wSLlhL)YyoY5lF^Mq9JyI2+sB@v8#`mNUZv>Gho z(b1)ED|{DaVgvw~5@B6XfJCFbm^6_t&R_tvg9c2l7M5 z4;?*r($pE9uE4^zB`!7HZwDPkw5qm%6UTx0LdF~B`>bNZ$s??{*gPqfu_*0DM@0dL z64;QU*sEgE)Ie>S@>{K?AdASZ$xhDKWk`_wkR_o~Wgl90&?Fdc+N#ZI&6^`82kHHO zp#_-g4ck4*Rj7xTL=krgg%I*sv0NfPkW;Lt*rBJC&EZK(k1tW0P7%ZeMltK+sZ@pS@1<75?PAfea@+PDP~j%H?ERF{*Mj9jM}@eM zKj>o&p!5z0#DNZEEe%bT5q|{3_BgRyKvk2{)8z)sD%fSEgo4Wynt_b1o2pU8+Ba&A zGR~2R7%-6GPX;Ype1a?9;$?$vC^UKEgbJz27k2Cfs|P9g$%mwz{=np8FrA`+$#I(> z!XH2Z-LE3tk?LOXZn|fZI@pD@634v`sn+)d)P#Xn;sTHgRXO&^5Jq;Q&~F1wx~2C< z0+0?z)UxT$#lh%;WkiRk1ptIILu!E?D+H@Pl`s4=2fBzTRd9_E5+1G`bveHmUXMKC z7FAxH{31-fI0edb3l|KCQDBj>ip>-10&huzveK%sNsx(bmAsZvlo;$L1@UC=K%H#@ zlyJksnNvcw06>;DFS`V7*y4b(hcgyz71Je>KXGMX5X_U(lA3_2l7pkwsH}8R22Spk zm*mD3f(3kG2LcJA2%f3|MAA-YWSY#{!GdBRZAI%bL z(4j_A;frNH!2Clf$bYOs`K z-5Mc@@&p-i*!>V_0Cot5+(hA&N)yZynKQZaUg9hHEue&UI7thO4Hif&c8Ulhu#_mq zp;Sf4l*ENkrxp@MiXq8T0jUmEqqyn@E>b{B!bwI^M|7!?1X9`>QGoL?)Ho12BbR5( zydb-51_FbXBV|S@qRI{ag7yMc;2oF?P*DScNBjg6oe!8i*{) zQCLzuiJiGPc@j7NA5nGH#RMgQ3hB4yo$A1Nh>TNE42OJ!2aI{B2ZHmXZvvZw2DYHG zMbrR|@;F|i7{@EpVI#OI38B=SowCDC%SfT}kmwSWszo_O#a|N~Hd2WNL?#Pd z3IG9pBoBkko#e{_oKjR-(iD85LaC=BU9Fd>PeBvJ4o z3!OA6QX_eZmrz61Rx$Ar;e(1yj#3w{F`}qsMUg|*7;ha{v%RqV!kWXItAa_1NLl@) z_)=*exmAghHbaQWvDEOQ@**|;QbMWn2mx_YDfaAy?d$}QX4dtHkVO_r6p3^@st70h z#U#ba{ssI|^%tZ#+Qq;KZ{E_Y1!QEmPNc&}I=Fg@M&*MPFwt>!SY8Q2X(xQ6dMlm# zCt0eD2&i7Fw?_w2EL)iucTzCr0aX6HQ*U(`0`Me1m9=^knY{B#Qc(T`j)lilr>wxI zEC(r}1lMUq>Kl7a9b-LHQi#1!bWoWPMXX%mcq;+4uXw03M!;I0f9fP`S$HWePf%O? zNx^iSXc(2Ectt(ro`OkR=mMxXTvTBqu?Z8SgGBf_7o(57RPC)Rt8-MPie_4Fc>B4O zPA)%_<373C`wy=z9ZX!bt7?$aez8ugH34*(h{#f+(JSG=)YeOhAc(AhD1u^aN+bl# zp!8vtRtz;HrUE{RMnbx;t1?8(qcV_WB}8OyA3+Avm$y2EaM2<;)fEtlf%Z`n8PP@I zwSLDsCHYW(ojWhIlcK2*QY0B+Bw?J2DhJ!EdyP7=mjHGw=}349ryS3!<}7b&4=GiQ zu;dRHud+dMqXQgu)t-uA;?HCGdrd>_qlZHXj3<~$NtKT>o2y2PNX?_k>$uUUqyMU+ zs_>Fb`Ak(f3MwIVg(Ae(+2JMGtESiOSe7ucm@_bEksvs(KPpmL*puwe1n!fp!T-$a z{s>3P_g;lkl6MqDfG~QvuvpS<0b<_iZXGS54#5XnmaGaL>BBmhba`281kI{mA z6-HNGm$`h(Dov^tO(>d|PC#X*qN=1dp`t;e?XH6AfvP-dl}IKQR@1Wl``D|K?#2Y} zldj4C=n9v0&lq;Zve3O$IyBu)1yn%9o#2&{x59MGUpi&EelI~mH4z2K8le&;3+SVB zq2NV{7*rI2ceq1h>kds!ZiKG~CL?eYDUJe5m4YN6aRS36q%lFo)!esUGmH9@iwqG| z2Y5$Vt&$=IRVzv*399wI2y?HNQzd0gH&FwbOo-Fcb0UclT2LD&kx3LWcDYi?iwNve z6OpnAy~IhJD4~QhF;}hzlCO5<1K@+WSxM9-1<6*WCbZ{HMbV-g6<=~Bj1(5BJMkon z&{a_@l7XHXsH53=fta0kg*IwnBZw+qtSs^D{iU(e5CwNS&H z(AjjV8<=D;De$LJK;w}>gX*3dRMaA)>fv~n;fF2+&k>kXaivegi)nT&ieD0vh_R5| z6IkQ5dZ&^lB?TMgq(5{~+lwNcPC^He`-me#`Aj0dYRyfUSp^~z;Kx0uEFqcNGZtwN zKur|^%p#J)C$8#V1`N`RyjFgZ+#-J>*_A7>4oe-hOd~2Hm!Tn=I6+jxGQgDkMF2OF zvTl@Ub)-~6qUZpliWDtYv|Osm%F0(pBh{j@(2)>7em*K+l{+Csq)r*Q6Y095n5IUg z#%dCyW9qlM*ky${iSJB~`y_7sKd$Pro)$||LzI3j-`_-u0$+Nmyc6`3w`gBPBJiRN z6ZCGECFeS} zvWTR^TS-E6J;aAlk!%zns*n^WHX@G}0D`L)5lTdFLggtb0^~a3C|N{|tJ8`kDM?cX zDII^MY{Ha_9~UyhCKRfUfHy=_yALDOP>ilFBr%DrQ&v=#e036%u1=O>a4VR|KUF48 zA5|}iCc7gwYwB_M4bUp=4P7oo7}ObYyE6p#)XLtU$kD&u*5<$i6>#X3Lh=Js?=A-CL}4x zh=#%qZ^S7g=vxwD3qJP*j-?z8r^Cus$`kc1f3I`W3CE%nGFk?S8>2v3?Nm}+VSTTk)h7|*>&yI4;yV+# zPq@a{s1WHM^cfA=x+@-CnOLVd+S`2kODQ12A>nvv(#;^ zf*4Mw-(6V^ZK}C=i|0{md)V>o}%>ny%9Saca)3i>mwlQ9WNxft9#d!3J!f z>GM}o0Mw?LqAD4pa1wLKvf#nX zoYsl~BKM4-c8qOj@lVtk5Mr9wV?qJlI%++Z7NmKkEp(g7y$mAqU#-b$K|D{UGb!-v z6fimNU#HJ;_*R3q6pZd4MF4q>tr7j`UW2)LJysNumQo9e(pb@Z`Kz9J73IS9dlksM zm=yR`3YZ-Cuaf7_e_5W7K03E(cP__A6F`|@li^m>!J?g?JDy@(Hjm0f{+LXhP~n(6 z#bxWZ2B}40A}Gz)P?|D=*Sft;@IK~kbwU)t1nv`}naTY&GziE- zm`uY>cYIKQkyWolint`=q#gB$8}o|3GXI(s_(cksqLN=E&9QwUd&jWVj1n1oyGjH~ z_vXiOUmU{v6Yr$%w6vDh93^vEH`_+y} zIY)x#0mErv;G3VRsBC^PzmozcaQ{w1{nf$bj4qDL7P~zW2V7&^h`kqB#RSkf+yri# z_9G;4AW~Enu)J)fjIoO*V*dS$DWHx3lZg+Nv^k+E6M0;5ei zwZNPp{l)EbqGo4`N{(^M{Sr^iv0QX%Ibaa%RyrXRl#{ZpHqP;bISc!IoZ4se1_Dl( zom2kA#<9ZgSc2NJMC8PiV_eIVH+zmllS47pu`V`sVm${6(^yWFf_!49LyyD^v1*W( z*_G)`3j8JoOyK@a5*?cWFxOg6LiXD5JU*P|Dhpb0+;n-ToSMkqZjQtjGountEp!Pp zCqfnV*PML|HIAVZNf(k3hafViJIb_(X%nqT%PG|62uDF}i`5?r=_7iNVQI~*PZSE6 zzPvr^P8OMi{E~uVKyfESjsF?rJv*eS*%g)v)0rMNzRtuo6v*lnG#R z+=`mV?83%LPT~#ur9u6SxM9QPFkD#mqp3JgQ;ATJb14BALo!XP2jOE@U^C+rf&wOR zAE(U(yN!$S+WHX)C@D3vvU0O>b5hdMvXA5m1CD@OSsjY#nMEt4H+S$PD`4jJ7*T*D z&=76g><*tVV0Aj2PG@0pL6X~3Ewy6BcsbXMmL&+y#EJ%1RMifa#HpZ&VLd;_&UxHEn@LG7SngRSN)6Yj&tj?CQp%}-181rq}yw-I&` zkb_VZvrQDicVYn&qAQI2gqB5S8s^MH)-Z8)>>r~7ZUyE80SoFWz=)h9PH?nW!r`DT z9I^$20rFC*iHQ?ZlBj|=L6xL67Aaj1A6(%z!4J4)Q8q&ly!%U%HHt#jjA5XEiR%7 zY*17R6`Lyk86`2({5w&=1n%RvH9;%U?S)4R$rRh?^EsWO4or!GDy~*>!cp4+O0Z19 zJy*i3d*F{fIEF)kg#cgx(TT)GdjXJx#RKmE3)Bj6#J5;QZyZljBPv*Oq;Q zOG$|oQY-8S0;=ZXVLFonzeWKQxPOg0$MBu%)sjCj1vc>mxssBMP=Ff+jPS8A%EHLv zDX0f#Y27j_0x&CB9{|1_KmqzE@YymEx7%383|fMISaCr#;*x~l?+>swM9%udhl2(| zc8#P!vqFERRBpFg*%KtN3(t)@1WTf@=l&p=&J91ZwfKWhxa?rS5e_?T99Crmb}7DF zq`1OzqD>(X3N7kc{Jz>N^-?O=nWz zcTm8TasLh?9it%hWC1KtGRVr6QCwWi4bE=0@Q70f^sygD_)(xcB*2Phf>G*MTsBO! z$S{z%ScxcrB^)joVFe(-O~3*tstmv${zyzn9L80K0h`-y^99in7ssdo3l8Y>$7Z#9 z5tBF=JS09GHzR~i@ysBx9R=lZ7*CWa><))WIVdMAc))PEtll6%9N;Y~j0^xU?je7G z!Gv*uk%H75R;2gHR#cXNG~)(!4Lr1a-zArJJoAF{-@A8rl`54Qp5D|EpX`-AqQX#%$b<61Uvs3SA2Tf$;9kTvc1kXA%veQg zQcH#b?x?atZEc!ds`$%jD1*gM0R1ofASfGS9*rpN#dxKFTMBGClLEg*0TZ}?i$cff zJsnt3C@QelrGU*uL;y&^rh$Q=*!Zzuz481Lp7?~BD}LZvxXb`s;T7JCOcj&=Z3yZM23<(PgWhQq#veI!=#+-_ znxEeWQH@;=@`lTn4-W)qMfBC@o*6UfV+Fk;*J#`2s#o9n2zZ7UMKnU;C>te!k9Ht? z5gs*&cpT+9oIswMDX4MB;ztA{UZgE$(;%n;QF$Jtw*0wlDLl=GlQ9@FD)k|mLqJZ` znH2af3YftCTNFA*?=^7i%(>Sqdpyb-U)W+t))BHGnFgm2T04vYvSYtm2n3d~StO{Z zRt>>qWFHo%3;4r09UQm|==b^5eh&rNfq+A_O2R%Xauyp}AV8m;G_96lqsHXq5#NsV>Ib9B$15TZgqO1}| zgziO zp#!5KyB2>*%ZjuoGT0o$Qkhww5EL+h`}l1p;0DJPBo*Rl4A>&pvYnwxHpfJv$ zvdAo8w+%OJadG}vqepx*e*EE_+(r%R-}KL0&p5jk%&RrzpEz#ZHzP;n=4RJF?ev>& zxizg?)rsGXpEGT$&2G8q(oS8vclRW?{jxzjE|@d;HD6S5EPmb^noT3S^{h02T@ z*;)H1eKT&>j9Ca@F749ssw=y?6BA*FQ4L(TdgVWFn3-SXZF@nxd+xl8#Wz4TBx;Cg z=>;+m9{%#fw-zj1=t+pb`s$un_vm@(z-}~A94^OdkRE7_4DD1LXRxFzL z#pj>x-LfmCO66;A{^vPuS}&e6XWW>tii=8{w>b0K8*i#wE$!Qd3zjWf0(NiMys@CP zu=5p{UD4y8>{LS6L}W6;r_17}cHp?o4x*2&d>%tFHv%9^#d0fG-O#i$*ljwK0>6g> zCUE~ADjkC`x(~yV(XaJ98Xaby5LE-*fJMWuuz-!IZnjAW~Z{vowUkv$l(Zcy2tF^Q!fBkpM z4s2fi=!-AQ`XcLkaPp!;f%9*}fe?_g8ij{5ujxupFEFF)b;7hc!(vKL=_vq6()EG}a9bjQ>?SF=$$yxhjupUn!dqI9kGERU|Xt6Nm!kCypmju}|KjqWsMMX(R&< zI5u|8s0O6}BQp^Vxj<;uw@a4Io?-ZX!^eKTXyMXs-MhynC4TUqH`lHFcHX#=aMFXn z8Z&PoToNvGJnCMw?F;>naQm2V|^vXdT3$$ z&ZsXwv-nCXS52QWW$N=!KAm2v^5n^rYuBweV#>5SO`2_6v*y~%J3ae&UvJTou(#;7 zM;->X?|Z1v)LAn|Gw{6lVwcXHznwpS^|Ei@c= zykWGZZMLo3c350-u;q=Lol_KYv_7}p*imDP3)XF2_XCTI4tHGR=B^Ww$S2v zvz~wa(Ic5ziscLB6&1481}VC%nK6sU9;qyzDdB=>;00t0q}5osVE%JYJYID0kZhz0 zmy{Nj!uN+Qj>M#j9WLpFf$-B?p3Ou7%-^_iqilxp2hkbazII(WH#aLkyGf%)Mu-p3 zwr|hA!a%TIv$ID|oi%sb?7sKjGjZ7GhQH|MJMJ*7r9f`Si!YT0O2zN*-LdQ0M;_d` za=A1ZY1_g%*;y95B{`!;{d$dDNr|VNdU|P2VQKDG1_=VV98UDtaiV)J+^p0>%SI@* z=}Zco&=gP%+X-ERldbS9REh?IAS>b$6!l~mcsS&;hTK4Iz?+nm#EzPf*LOHK2S7h? zV4p26&Yha#j*AO>OMHd-#RY{%BM7{ zCshiE-+cKQ)*FX>_Oab%xuVB4&ChKs40_lY@Z6K#dj2z~sL=28BL5(0TBWr2-hGEn zjUhCpmF*G07aa#+8r4I_36M}FDGQ@HIXSYx7Ye0RsPNH;9|P(r9SVPLx8xq&_vMhm zXP$k=XPQvmDX<9Qm0X)gR?T% ztzFf!`I!xC)t~(JSN6EX)N1MJ8TC^epO(FK>stf*-;jUI6M6_$qQEZqiUn6U=EbDI ziADhvxKFSalWmi-i_zv|FQU!H#u$I0&mRunb^VQk%NFOqUVQPLpVlRRFZqn?E+i#!n$-ut%KS*wIRjc=V`f0B}IR5it z<3ArNrmd{jCT&_j{_L|Bhv%L5KIzlzmUZhl-rMs!K)1`yw>;4I(N~{%Y`}j8GE%5a zSvz(6C?paiM_j;4E=)mCIJEjvZbZWdrAfO3-9d&Q7F8nxpA)RL&ElFA_<0J5TEfr2 z`Qv=qH(h8L3$`3to8wQdR>{e>3sI{ro4eBeab?T>e_YllR!WM8mLiMNh@uIiV@)V@ zv{$U(*7kyWb&65omfb3-mWVik-H8$gwi2MS!7#I-9PMdv_VD3D-r|xQP87>*lxv&u64<^PZIGhNzLq40^fxb0X|LwLCu{3CPB9IY#(z1cv zi&a9FD1AP5Y!($2#m6T);%H37&<=r9Hr@vVhjX$hLvl(LIn^8%UBuC#zK;~JITi2* zat~+6yWFX%sZvUm_QIje0|&8jm7bn%cRGA*CX^*mh6yL!z0X^en3U{_j|09#B_+j$ zMOLRPA*lk>0OZ%~%*0q0IwYu>JFM1F0A+M1@)Pvzm@>!!#GW14@#uE|>D0%jY+yy` z9D5lJ^q2gliHmKv*LnIy+d#RwBQvGJX zqSam@y}H`ZBc~t+EjDL`6j{8($_McnEjVPy$I2cX+&Pe3Boc^FiJR{ z*k-rKCnb0i$hVZGK#`}QjZHFalX0Y^X8@@Itoi}Uh;w*Fnb~<`WltxaDZXk(LdY+6 zOwm7Q>VPagEhB@R#R4K#vpYb4P`e~x#HFTI2vwvPdYK8JhZt{*bH}6SM=(s_viFGb z3?4uhObNp<<8aEZAC_t{8_UilR9!>^M_C9^{E21}POv67i)&KgCloM&`zIthzCGYA zfmgBir}zn3L1ddFWfomv+5RnNUSY)bghoUhO2R9%9L0fPZHo97yVj5c$VfoUtHy9m zxK$LR3C+5ou>CS52-|NHq?f>K%#<8Cf1ax~Ao-!FO$uUA%4YEa$Q|Mg%y-~pL%>2ArBcX(c;ir6i)j;;k$5Sl3i!sUWUxN2 zyw%DoK7!XI%AJZL;mAS7V(OqD^|VIc|F2g?z(^vxK#>QPW#m4R@6YBHp+@<1em0!x zX;MJ4z{svK&gGSYzwz>Qqtk7e(qb~BHAx=k(8qu$&2!tmGdv}T^9d;xht1=G#M>8(zWZD`PZaCi~=Ud9V3N#^J^6N6+i(!fAY5MyZYbxT#eB`TR2|*^lf<@)JXc%xcL7x z;!lJ9c|K8!{ld8P=VffB{I{Th$#I|X%@~D%U#LhqjxQQgP&I<5Kj8tRVI;K}=4y=N z)z5I0k!(@f=IzlutY``T*u^wDe*EI7XDOjBNeo_p34&F;vT;J4h$((LqEtZ%*|iVh z%KW4EDuqAhQl5H8rC5&66iLbeagGMM#;TMf_14@D>*-1F>~s zRAr^!QV~g$T{e`JtMIDqwvk+@0~J(3$xi((S4UOTC_#i2LXb%2u^ZCYUOFS~ukR&y z{zt1I#ZZA`aV3DtUWmq9E}m6K<&pe?5^;JC8I9Zv-=0>p$Myh1UK>t**05F#P@0!Tp) z`-(b(1?1pLc1(S9aHqZZc5U0Xx3=weyR~iGwr$(?*52CP+P3}s?DL&>=Kb$Z=APgr zIXTIdbDeRZKxCwq02eColf)u!`e@v!1!~I3T@@)Z>KsN~4`V|3_K7Eay4MEiT=;au z@?Qh-u*Xy9;WV|veAFzsH1vkt(xFHVQv*jXl|bUCh=g~C=>6rTSggsNPUe9Ju>wV@ zAP#J%P_2=K06HRGY<nehbk9g1V3!xHh?@nY zXoMo%z`a**-^z$GTWXgL3I${{Vb^4oqwR0Ve^`Yd(z!bnjIuveegudrd_sOC+Ehe< z4fr9>T0bfMx zl@U1ZOe@k@idyJei0t+cjTT8ZMa9JEiq>q;j&$+x;bx3{>J0JpfN+6o8gO7aKMcoA z6?iC}CU*J3Qu#iGk=EF7NMX6&yp;(m!f}NAOn&h~Q}ag_Zydf%l08ozJ5MErHAz+spcqZ>$Xs#5fa zXd;9Y;A$j)t4U}ro4ERhVj`mAe2kzi*~5OTX=6+;vNJLlkSZYcFCYjL1=v&p`<={A z7TYqmOU}xfn zJBh#{MT0d}#-G5*_5SO72fAetVe&ARNP$hM%oPcwRAmEE_;eaP!%9N3DB>fa!w#s4 z-73Q&bQ?%ogABV|;u86)w||UFK&!k%f*@LM`;{bx&j!x%A1gnsyN)>tZ1R9;HH7w< z&Q7V$D$jpS4|cOr-|$c=)|!3C&YH#>$pdrqsY-(h?tBXVxS>T0OhL5<@dFVILXia} zPqzo@t~TTZgC!0Q@Hw2|Ln#~NtUZ1E0~2uz^R2=n%()Oe9Glg)@$TD$K3&aIY|2xg(d zmoE7uGAmNs-Y`GVf491b;=kOY zI)E|IcQY|2ijVI7Q%;zs5vnhdPN3anOcJXC*0d6Bz!9_~9z$yREEhM2vn6z~8(msz zVkm_Wy+Bten0Ikbh-I7$ZA~N0)plPyHy_?e&vn&lx7C7AjGZ{=ACTuq-REZBi5dYC zxq`Yp2Mt2?l`sz(?`>r|!=%mdqDsdGeGm1Na%=#N*ouNe`7BA9K0q6aql&)eI>4c4 zw^|<%-dH8Bh^(a`n3~NN^jqL+Jh+{#5CxMz8J9e4m!_LZ**bu%Pe(q;m=}txp&AFq zQT&j_=Pi5XEc@lfojzOM?{VH2D~Uy5mgCb6}Tn9>Cb+d z*)#z&Wl!3q1K?$@xh0VzCoP7)Qy0R_EEu^!jRe$r3jS(3q+Fq!u z6?dhMTo1a^_~C;s2SB(L`|n1|&6&rvL68aznJE+`S+elccNm{^4_xTr_#R@YkR7T- z9?_oze4|gN9|alqPq0o{aK_=NRWy7k2uW~AoQwaqknCNAbmUKjWN0q@CnEMna9dIo zdoDh{OsL@a;2$*;f_s?ZE9nZY2)N6=pdj1*ec2f-jGYhjT6W#Hi=psTSlTp7B*RF5 z_N33MYL$tQSs{aei*XXb`M(SGx*b2&_|$bR>nTSgB2A_w+ZaT_0>=w0xNB6_C#@EP)3zHV3redwn(w1n9vCh`1ZlV5V``Dodc$M%(rKkux7|RerTL*h2$BU0=&BhU zEcP>YZPeJ*dqBDG>7r@oYb~TK&hXsBE%sSaii+WfysVFOs&iZU&$h$dqA$vIJs}1M zZL!IM>tw>lgino*7RxfSCgXC%z7Q6+=;0_#zvoT2jEsyt?>p#GgvA=Im0f%9=S@Tw zvuW$)YHa9(!z1s9ISIK;)?MG7J~z$;$CsPk?wVBHmg3@r{WyUy*V{eK7G`A{Rj@9J zID*Z5p`y($SNN}s1$r&^yuUB3%A`EX6bTJRqen+axvqaLl5NKm$!XPUH@e)e^heLeK#|-<@yDz_vP^jsbTHckMa*qcQm3ORJ{kT{nmm>OKXrw0OXZT_&Ug}q$(2C5QWeI}&=#v@+0DCR^Z&;Mkj36%vA?iI z$axbc7Ip8_69kEnJy#eaONkXUFao%hl8D1Pjdc>~#9_0vuOHm#a0)}f=k|GXu3Rfu zB8^Rdxfw(xriL_fOg(zP?u{!OMU|)lFcRFW)@a(Ox7i+yC7@Es?s)%!fX5YF^?83< zsZxt$p-Kk@-n%)9v0Sn*{At)!quz0*sA{@WyV>tfZr{iRki&ip*QOMMNkD35*449; zKVx1(Q~EyIU0idmPXGPBI&%m&jDauSPOzUUlIOt69$2lvX;*Bme=1wJp6G?*S4&7o z_ZV^b2@GU!9!Jx+$g|Jw*?r5NwjJN&3r`&%Atv40^I+qx2!s~A9#X}4fltwv`*2>K z4gD1E=Ml^EmD@j~$U&Mq25$i{dS{WREb^@M& z^UnGVzq6ZiJ*T_-_R{6dcDuL1VC6Vr_rt+TUe7rKdQB$BFZ@%rQ@R60p4eIFBI$Mrofjy4{% z{rTa?1sVjxm%Ax77eTb?cK(Z+ueGcdE)epQ2)Zn8*R^V?)c6%sqBd#=@JQs5>$L`> zu_%S7j@GIgRsp;3E67fljW*wob*c=u!z>d($TE6%c7`+L!IgEg*jq#Z6wJ$}L5^DEWuMF2*c0e*$lGVKWf-imBu1(aO zF9$7$+p16gitb8Ki|;VLtrogQNay>i>ay#SLDOIg1ylh)IWR;O0S_7xo;JbJ>|v+8 zlSZANZ0+~j@_OW6hx zj~6d*(n+|$%X9mAM)&tho!)xW!Ydy^_iik|!qsZa`Psc@KgEjeY`TW8r}#*CgON;N zzbU;IzvCadlFz8#4@33tbvEN0BK5Je6k0zv+Y5m&4_GgoNJO)lR|C0vNBBr%k>uzL z60D2uUcdV%{7`T9boAe}UC$@WwSe`~)a82FXg+ROydK9p&$w={M1urFC}p_(PjR+COBYU^Vh|;`!tUw)OY#)>wN=#;dm-j zEZI!gbN2$4n}p?R>ork>#gv`1CS!^MHgIgsz$Jm5#~RPZ^m^gAK*$chyV9x+KpQyO z!5f$z4?@i%*1VeMd;1#2*u!}&c@>ufoT&&@W>_FKJPUnoq5+jhGtEt%7$$2I6@cWAVvUC$@R_<%gAl7a3} zA+S|j)ry_RUPM_`>!tPDZsK2TwvZ(WN3rEuoC{S{T4epxYgzBuHpXeXnOjK!eE|PR zjqvr-$I9IHve|>b!7qcOtEPUCZUJ>;CzXoR-a+jdSeSM*DtB5L4M>YNTI2kDz&z*;-3ji zId+_uivr{fHFwiBS?XYD4lM>I=>{E&LxU;$MaO(6W1^OwOY`jaBSF4?J ztlVI(8oVvU2KNE=7}NYBy}{@bSyUl|jzPAnT1`&VPsC7d8tctmc5993zRRDS5%3tx z=Mt%xOc%b4cQ6zvKrKnxT;aOnJfSR9hAuFkq1JNit<-SuEQlAo!yT8548m|%v36UH z=mAgHYiqj$Pxoara2VhiGuhuCuywMzTnlQ+y|%l7_YB?b9_RN8RtrxPu#6Yav!EIo z@5tHj33*N^xRg$g59BmkLJ+b=Cg)F0NAy`=n>cRX1DC&TTSKsUNIW^4U%ngM2<%nS zkGFe2x9I8UV7t(O!G$`(iz8T-6dMdgVua-A8ir6%<)TTbma&S&`*r(4*wMs*fvE$* zPk%o*CSQ8Jzkg13JU5!7eDp?MG@9|d%>Y2Y36>eIV~!c#`$-nE)n||~7)kLn*Lp5{ zL4?&lA3+TKNHB3E7W6D+7jSAg8FmdmE9`;;i`H$Iw@*O`FIvy;L=Nm=NRWV>^;S^b zi}CO`E!>;}6R1%2sJ+Ls1`Z9ZW}sHyul6YVp?Kh_XlxK$UQLxqhCuB0le>~Xs{qH0 zA2rZZ>#aFlc3YF+u%%O30xnx1OlcrZPUrO&#l!e{aZ`cBOQ+-24K{yN#{~V9wY8Ny zeIzbBJLE6H>U0KAl$&9h&}+JoGI!#Ybv&yu!`E zXO`u(3c^aFh`7L!m@t2o&fYdEDn{O!Tkj5p;B!0s)8O=2y{iscyUP8uqP&bEdC8tR z(_=Hjjv!}`#~y-+fSzBNSqx+}P2q#LAG6q*A#veD05-t@O=f1xx$DW|k`fBu<6RZK zR@>=DNfr;i(_&TK0js6A$^u3PTcgBpuA`8)5jqXd>&z;MqLEgZ65(DuHSR+b#H7Xm4}H$vNK5nOl9COdb{uOZ)F9h z?&Uux`~IRwzQDr-<}_;A2Vyoh93`cx!^m=F=X5$F93LyZIt!0af28I5A@DwO>#QmP zJ(;H=L+51kV9mY@X%SG`-nmvMxiKcfNY8;wP>;^Ygt%yW8-{ z?LH^q!Q`C7yVv~(KBN|?1gQ|5RDEgR7YR*Z$QpCECqM!dpxcE~@AD87%v&)ds@JS4Yp+tKW<M0(foi%WA*h9zv7lb$H}H z9tP#C)NIu|oHx?jwEv!9eR?)O%}LZ~H&AKop4>BqNX>-&=9p*T1LjweM6SzOVQRB^ zPs_vZFr5p`!By>f`xq0abgtMe&!DC6TF)>Oc-`?a%U;HXc;q>+rXeUlCI4F@( z^?sjnqjWv7Kh2k6r`JZJq0e}JT#>onRQiIeEn?Dn~@waMD@IFEVa z^ID!9PY7s(WNTAOk_2EcJg3XW7h5X^gcHv*wOw~keajANon~a<_QQST{@KwF9=rz7 z>+^XeOO0~eSfLRR;W0QMVBwuQYaZ|n-B*Zy9Y>m?8%_+p{vTuA25>*K2~4M`;JG$; zpHi?x=y2;n^^nULi@~8!VE3%J)~j4e+2+)cHevs(`ZN-xVd(&Ros z6_BxZ%}IQ@zvsDLjeYO$&=QShrQO#l*)3IMNkl;);5c7zU-P#Zjb4<>t>-QVh0Zr` zbid`yaGyusGoCp0mmFuiVAQAO^_H1+?VtKX==!+)F^UsQ*ltqWrvB}jGgnnc%5t9k zQ{mHg*PDWM?f5H^w$9yiY}QQA$7q>jdmswwOP4n`|Kvup_8DR|`>VoCxBYR%3Nj}) zl+Se$8lT{ErFVaeb*DvQd%-k@aHT~pFMVda;e9->`)y*}N4}qBVk8Fly2p}fo!N|S z4F7Ftp{KcT5K+K;QE7_6=Rt{&c7$0)MKzXw;-+`wbWHZ#>xqj)&x^hYubD5ur8I|@ z-y}yXi{+bXxu=!u#t4B-mGI+bgb^#Kv~hJey{A!s?#f<1h?}2Zba%Grc{!0Kaqvp7 z{)k3VWw_O;wx2DJ^~BV9yVGiV@W*JIjH=u9LbXSOwgFI}I*6f4GJAetRcr)?kpHqMET5-Vp=^0g?$3i7N3q!2I`{Q|ZmOYLF&+;4EP7DvlJ z?TDVmpbp7l!r)>dE$0XIWZ7Wh$BDF?$#gByEfyFmEWUK&ZmFpS6x0=3We7_NDD-R= z>eBL!NM|Tb^nA{SgwrF}C5I*BxKhQ4%Scan2e78m zEwC{Q`4pB!pUEuN^eeRQM~VV?9s7y+Xsa;xp>RYl&&))B(~7e2+7jYWPT7+vk7n;G zN+}7|WNS9_9NnR(C4+nWx(_g`qA4;MaD0SgprMtPmCday!8Hxhgh}JaDz_Q^jResX zpMuap5I|f)5Xb=SUL{T=6*!~%+rHbH8)Nm&R5Uy_sF9;DvmxMplyaQemV`Mg{G`HB z7UBlclF-g2W<@_tFSz3X&ITU-hARd{LO%g#KRTss>n}}fZ=SAUZqrdA9Fd=R@Qfgt z*aPMOPXgI*^r`kdzvL1Si;Ntv&n)fjcuJrzRu| z&GJ>sLC}n>NEP9HyhVH=U@ni8uPc=Tv}I=)bZr(RWHu8J)^Z;?y-J3(gF|rv7ussL zH&cT72+Y_*q)NkQNL}duw_;jKk_ZUvUb#xXhRljZMf1U4Dd`@Rmrwbaka6bOE#&-W zF47>vUu-v1W0M^)*d?aQ|l-6(tph`EYi|0gC3YkbXItNSBF41(X^~pn=7v@Mfk>&;j2@S=IIZI8l6{1kr^* z_3DlCHVO2k$j)S%2-Fv%P6a{SDCmEn(`7T9ZQQK_{HisR-;s7HhGrGYQ&drtbq~r- z(S<+j#m-=nR5kK>ub^l$#TD~AMPdZP*=OR-$nYpghi2O{Bsu9gbmyrnoKtE-UW$Vp zEs1r3lk(Ih-KK6R)*Rn^rvknQ0v0i!^{&?E!jeI~$Cqcq0z#sR!Tf+?$&5*fQNjp- z?!#e>1F<{Dg_cdT&*rty{b){N!8`Pm=7BWPYz)zhm>)r7f$(-FHWKdoU?{?fY`^34 zMLm*H#AJkBtfbO9;n6Y+YosIck@-Y?G+^PO`^;cgCt3p zVCaKYOKr)U=4E!8cY-ug=JP$l?`VW8kXGb|E2EUbl~hZl4GdT*DSn?R6QtPS|Kfk0 z6l)X=@(tA3iSq(8HfYAuRz`Or`mIdOpm6XL>6tD)IA3d2uJsQt^bfByaSM6jQs6P0 zHzpB}Mi{2$sUs5IAq_e9<|<>h5T$6S1C(4UxI!2>J%y_|qYUnObd0~g|U~Rdb`L_e`(xb@%A~HT4BEdZY z9B_k7VDx?@&PpfJUdlTIGBXp*-1IIC?xaosoSQ{KjKVs>^_}{i6#XUgBIq-s-=e-0 zIONimPhhPXk-^KAe}}TF%{i!m4ORGmP}mprNKY67DO(-VkE&CNsz(RaVyo54E?JKJ z6ea#U&<6HI{?~$yL?}zCgOX_dj8X*!8>O&WGikO77f$@WFf>0S5Q63hw3KnnB1r-r zSV5!ZvcQ6&4vMyI@1$>Wh=dghNsDJV7E4N(ncG)_1V>>2Qqd!~^Bnb<8V%1X-pkBC z%T(b0P%R4mMMJOg_3P)O^lfFuj{@e^eq&A7DNV5u1PM8`HSbMpI*NEfSCMVn%Sti< zpKmZ&kC339TzjOF_5is??t}Gke?7j>w)t~HwNCxtg(nHTwh;r;vi=aZxI`tUG)Wkm z&~x!YwxzBUL;>FBK?O*NSlGboRnXw!mSqy}FoNjts3eu4WC`--4V4(-( zDIwzK!i%!Wa1fi4r^deMhd;XTc1{%zki|emphKv{#ZIA$P|)U2g>c!jAW-YBgry#4 zvK8AJijebV6pFJ#f1#X|tYOZjLgZ$3S_P))Wu0R&Xe7+~!vvEe&Z&TXX6A@?;X86q z7-4~Q+*TeQ1E|6SMJde>TjJOuzaSy4cfJ5-?=L%GsuLb?Z-`DX?&yiF*}u8}){qi3zV9AU9wk#T=E0=(3o?(GWn~H1Hm;bw!3?y3IMv z@E}H`yNSf`a;Al-6-$idK@+JBxwZz>106HxP>^eM0|qWK6^2|5=38o*2N_frY0wo< znY0Ldi4kh_rRed6Sg75F3T8w0mHK?l=e+1rIH;sSG8LJN6(>?2n~PD8#L0dvZaTKQ ze7U7)8*u6A{mqpuC4T|ye75^Z%pB++Ip?g z!{n1_(1baU@;g(28!|_QD zr&vSqP)EN8F{S{uSI8ec)I(gVWOOJTT^R_4E%Ff>iRZ)sZBb4A$}?FpjM${$JB$34 zG%7qk5m|~>|%V;uTd_y5a_PL`mCWIq3c~QxzOHIRCGF-Sb4v{P=g(H!!hKAj4;e zP^_O&&=1>W^^q-&Ds;R7ze8O%*m4(^VE&p!OW_ThU=8!6GHh-%&n2*dVkHfM=o)*E z@AQWUrIh^thv-Z&4BrA=G2kgq+;sgvV0klhdoM>?ZJ`)-Q!}zNR5Ub+o|^27=^avm zU=hLo;^X>H9$>~~$dm_GL58Lk9E!E>Liq=_1u-E%`GcN0-Aj;?(e()rsUl{j_n3gI zxW8!Ar1GVK1@kA!{1GCXxFKXZIo9NK`+1qE$f+Vi+>+}K1&&ZI9hh^$@-wuPCBuL` zo%U{pQ%gBC;)CQNX>tsl_s4eJ!JLEtfZEz+NP~up)rPMqh-!tBV|-Qy5rQ|x25J03 z`wS&z@PtSRhc0+vr*h1uptu@usXMgTjej4tic zw_-gcK7Bw>5CKQ;hM!j?4_do-n^`gR>uAW*xDnFdnpzHS%u#~p`C{D9@+5D~e&;0V zJ%z-dBIBnCPP;(C@`B8;M&*-e5s(k-Cf<}SFKcoH@IO68{ZCJo*;hE0y#7f2dz?}eQ~A>rL7)5gYjo~_x#|(i=vx$&6$-y0C#@;# zxfE@<^g-QPd>&Ftc2mVR5zBupFA0P58o7QHDQ6&XkGO= z$%~N_D*3mWrQ9{B*cB>!CRUoW|NKpuv-zeM@o())N4HkJM&PY(U%gVAHYTm%mFoiJ z*yYK0T{*9b+fy}>)OgBE_D{6&^_#WM0tjP|5%|`C3SY+mWl}ax$h+Hf>K8rr?^fxY z?M?#Ywj&;wS+z|a`A-E5CIYG;OpR-)WS>yAQCp%kTlhS81V~sfZQIpGAOQ5Ka6GS% zj7O^F2++i(w_CN=5v^&di;9VfCv9s_8eG5(sIl`HlZ3#Ke=RwdcQ1159I}f2J`I#9G@wsRPDj z4N#(+i5CewA3Nl~UpRsOO?#jq06*-1>2m9T@@(Wo&LYSH{g;L3|4%6XLQgjTORKO0 zO4M1)4VR1i*GG0pgg)RI;Hz~&1s{cY`f<2syjQWKRBU_B=gNnpMe+Z& zdMYbQX+dxfW_zWF^;xiKQr4H67Wg&kqR3AIS)gIm{~97oXkB8P%F-g15wFg4!Rl|N zze+7ji8QzgqXnJiJm4Av7M$gDE+|{M3N!aq0T{CQf2|8eY*FSI@2d)#g1(U>YN%f& zm*O3P767w%lT~J)T4Vfe{3ME+qS7C}1HH@V@aN0SX@}|G5Qg0j)>O-(y0UbT7-$Bv z3zU&s)~_ViB?UH7=qn~7DNB9^Ic@&4GS5Z<15>rlB2Y}v{$gFvebtyu9B&jc^&_Bz zmPS@BoLp(MvzZwuCbGk-mOi{q|2Ijymc~U9pnWNz5pf6hHo2R%6XU=|2g@p?g4sic z1#Q54LpmDUI!o_O`-S9)bAiNP$P zy0{mv!!Z=(83`yf#%pm!0#qKoUQZ!_I->r`KzF7|x<#D1bpgbX2xl_3Bj8jWObK_^ zl}EU?eH{w{W=hBfiNb!}Ff1Cmw^J-+sq{;rVvUUE}hFfQ--H6!(bnw+7-Mh~nv1 z8cju~rRe@jEHZB+#c<`!i87OysFCB7n||B0^S#GqrP2;2x4L&iR=%+-dYW;^ab2#Q z0QSKJ?148{jz9@q$S5?ZI(cmRg6FL^LN;wIXW?Psv$yuTIo2dnah_l{(1Gl*`ze_9 zX~nfa0h+oz^xulW;eQZWI|!GtvYE979p(OkAf;%{VeRQWXe!~w!cuq}E>g8+bRw+T zV&`|{am@8ww=Bv+#_CJi(?+-wUh-<05#1;cu-YyFD)HIiJDgSxl-w59Wq)iYlv!uT zpmO7s>vAKYMu9W83xi>0tZL;h7`5qqnI2Yxl?B_)b1ohUskf-L-6q|Pu#)zlnyFRz zLTbVNgb+}mJBk_J_)*iJFhY8T&;=?%xL$CDEa$^!InuD}%B@MTUGdz38H!xXr)wqD z&ANGX{Nv3*tIH=mw?OBz;GdYozOYqjVUX&9wecjS?jS;HLM}7iokQ`roESUHf33SR z6|36nM!AlfE{{hg_U-qzRZVttjb;uFyjc#jbv3X63Le~zcG+LTc9Zr8=xMFT_iE3&yG}L7@fqkhAFZw zpYbP_z%*nhj%+A*YqFi#Zd7j_{Z%~q0pVcoRUso^E>|D9wZhkoZ;T>?rGs!atWgX6KBa>kct7MTyM8-;R z(NENtk_SJT7;+OYC0l6c*s-O8t{{)gRV!!%$BfIs>rPZ3*NOx>4yt``7vB2rKG`3kM)%tZ~nStrCcnVL+-HOAzWwaBGcQ~WaLH|C#R&OXi@Imag!j2&74Mx7a_Qj zlz2=(?dm)1)B|6UL1BvngK@e(wFkEoZ&Ac`wNrJQy{D4mu%4_=(h44@Vmrf1#9+x7 z!$>RNiAA1!>Y<)kZs`a$8KT@$)JcogVuDZ#3*)h?Ny(y3qLP_v-m292{;VC|RYPxIhl2E81q~gY9j* zpM%IGd!eysH3e*Re;L4pTYP4~w`tQou#f|VL#zCtmKrp{vdBFcF{}8M8Q|_X7FB^R zE4wEH;0^b_!ft^J7cUt?jn{3rs!oOX*4nYMrEcJIz^RPBP#B&tGZ*J8UN2;d*vwWf z&-@9!M7BF)S<3vuO+5a?vZ;~+L<^ZCu8%&`@D|I89G?JZGzA|48jMxoLJ44g=K4py3OVqjeXB@xxzjVv65xtqPLs6_6bloVa*Q(g^A0Ao_t=s^(Hfg zGSt|c;Kvc-(0hfMM0g ze}j_@3SJCd;Y$cuwR!_LZymvwdK~_yxi{Nv(zHZ3nC9wBa%Vld?JV5I(y6KMzl;=Z z8Ts_%-OdvW&3P?z5M;F6OOOJWn57JoM0PUn#NRF7PlxEy=VTG+ZFi`VWpsR2pk0=K zWZ@zTxTuHjHV`HkLJ_f{K6Z}@+%2Ek2afY5=!3Rjq7QT?Jqk;PEWcHrFbl=bIG1Bv z!6U!JPFS7*8zI}?S3%j$PthLMEUc57dbdHrU2Imw^`+zoO&Z`XHmpWPFT;g6{}o!% zH*}KpgQT)zxH8$8q6DSBY4STYM=bXho4Y`MGM~LFcqjzj#Xc6!uKIHvPC)h?ZlT*& zjSl76-5pHxFTv0{+Ho|SGA|eB!|LI;9Rk^wYi^M4?}Mz7zHkmMyg4+J)r6D`4GcIX z`PE`2ZXP^i1Qm~+S|}}yI`hPxIe+)XODZb5+O(f)6xd4C;EK6epCWKGPPgGhMraiS zcLy_>6D;}_cj;k2gsEuMKb=?9MaZUQbui$erPH&ANGp`wn6hR2sp?pdkA_RFbo|y6{F8D7TS$!M>%;^fgz?l^jMJ(@357%QeS}X2}$d3vIN{XW{(R5S!$` zlf^mVAhKZl+`)e?0%oSeuxJ+6!)1dd^Kgel(Ql^Wc28a@(VQEHfo(g0D(l&dlaI2MMASMNCW1qPhY|ZnczgF=qCaS zQynAcC`cTU8pI=Km!(pY5})<5@YN3i5A>{nFDE0K({`BFQBI5~q9iyh;}?!V0avG2 zL%%KOVOSqX#+vkC9!n0)`=wU6KFbHHKVe8Nn`}|Ms8U?9C2b9d1(%`fxd2M8@T%bO zmcdT*cQL(PS)h11ezTHIqr%1ICKdyiOuOujc+#F%o`MVr!q|pDPdLda)A5F?L53d) z(p`L6E7M(R>VDRX%!q!Y#7Jtd53imMZBd`?F_S!L?HdsVva&nYOQ7dwi|j_8F;&&( z<}b9u6mG~4kaBzWocSbX946)kOls<4ZPxkSi<0B$z+so$(f%M+nQgA~KxRXd0p|od zDOhHNa3NC}=r5A;o*X^dkB2J8Uh~b$%hxnSaaT%AQm<$&WKs8&{e+#ZOI#lUJ5%}E zKUom)qBvt=1P3y#=U0-Pp|}W=!76d1>eR|5xHpCJu_NpdXhu?;%&cjRNlIE|iNacF zzk{ml+QDAp73ld|RCtDnCbcN%_gTQJD8}-2zB09uixh-Ys#zfZc|vj?Xi>~w7Qt1||8-re)-WaQ3(@2kQGIl3ztcfp0_OwX zeaTBgLCR)oycC!K&+Cna5`SSo%cb9w9i;oGP9=aG4XXMv^^~pY0Q|R?xE;_7upob% zXy=0G6K|Em^$b5w*cWcRPM7y77o{s_m(l3nZ?8U?H3NzI%u+1zVEH31YI=eg{biV9 zkIj55=l%^lGld+^%lo*!T}RsmLa3eWSVgKm2XQ^0-HfI#=Mz8gbqzh)g1Vh0Q{Cj; zLF|ZEsvkHm;)S~9kw(3ApI*&)w|Z&QRBON{E={?YDetmHgHUD(Ouq#szkQ1=~& zE%w+gNTz?sUc8MQRN|4yEO6H|)ssQUi9-ha_XaXh1>Sv+WO2yorO76!f^|%(W@WaK zVrRk1qw)m;14ssU+E!m-!3iMxxi!Ty!%T8u2#A)PTf>&BGR$?C-ba5j#n)i2Q>nUl z$kIlu3}NSC?SVK*LX0D8Em@hS6tz#d#U<`MA!5v*>|-LF24PUcigxwvGnWSKkUDqe29?AaVV#*mBQLa|yNC;8+4w;i z9`-a7LXz{$bM)AOw>#fZ*GtCa>ap$TrE>jKp(MF{>k}#hzd(|rFYXn$ev*d~$fc}+ z8cO0f!9#cg6w}$QQtAVf0~K(a(IJ(OwrwyD6>wbz;yqXO@t#uijgo+l9OoDsp}zw2 zfgJ(;WlEX~B?Sl*S4R4VnT9Fdd5vg|p zAze$~>e=1?f-7dGZy6Qqv|9l##lwIu!}ABg6S;87ojGntl0Mcx{XRS=|Cd5RO%8nd68Me<9xPI zb{#(Y)nRsw;EYM$ulRjGed_Bt!=dZ@m83uc7G(1KJ=^v0apirU1*vv4?tN!d$u1e~ z!e+g-WfaqVwfWVquG?~@RwvK<;dXIGz$oM7)a*&%y8FH-r|o9bP5gST6WQ~siWp~`Fuf?yUj*$Q z0h3wA(56Q=(%xU0o6^~u^+mursIa#22vRJ8_Sg&>H<#9FxG^C{?|)NbIisAq3Dgeu z6$E?^=0*z-3o>2Kdid`2k_6nn?$VmR97it~D145d-_Hr=+y^^dyxh=<9&NSTY@gA z+}al!oz8t8CY&F(J|pY+ik`>aw%n&CsTYuDtd{*aJjoJb+f^|~rC%k!{YW%`@Ceo?L<(K{WMymm5X zRfqL)Ac82H^C1@V&|DbEr!ezvdfzDa`z=SROl*IX-aGxw#Bl6HXaMu}kmWz4*27 z?Fe~(-2N{leXSs3*lzepR3jgs$@3Sini7I4AIBRmdrl`FYBaI{JTl=^;5(*k%IOjO z!$Z5@cM~})dVZtu93S_)VY1w%I)%pwoIY>KQro`Ik3rAyF@S)H+D(QFZ!R*J41%N% zme;xaBR%(fO{7|F*2HX)t}QXLIk!u4ZQHpSS6S>|BGn0N%nk$TZEm((?)@Q{x{jwM zHzPE>q+1;HP7QUrU(y)=HzWf0%bJsCB%qG&Vy0xVA zL^kfpLO>I@H|T15pIh&Qre-CK>C9_Ns&8}e*FAh^@Dc|77YZX`kuU>uo>ZB-nZG3c?7e z82;XGHjl?DzP3MadpRwhoLuq!9Eqc)r$4MB%g+{)39PmWG{AZ(xw(%kO@|3vP`F7ezdc^iwviF6oMT!}jFS^fBzQR7feZOr*#}L z-$N1MqTNCPRqU0DGQ^N=G~;};ICWJ5GhW8|+955=Wr1>|s7jms{V>TAPw77qQ?>GS z8Ox(KnPp`x3!T62deL+E#B zzxBI~QvfSGOyJG9n_WJ7n814{K%eVAU|377MuYkJ9T3i}pyAPE$@Xt4pO=%%`GVW- ziXNAD0QG_y6U@!)f@UiphDMuL-PZQh=ZGY`IhD%$t&Rc$mrj*hU22NdhKprgE`_dJ znS4e!3^)F5EM<-giktK4+Wq}_Qa!s6EWgv+L+{sP>^%N@^ePTaN|>=%7zuzeW)wAA zOtnTMW*WaIdo2j$hlAM>TdV!<9)k||eFn(pgFSIJZvhhXF}lX*x1TOBkTF#gBUaTA zj9Ws_)a4-H26^Re!qaq|Q?aZ+9vf+{Ky#!*RJ{t!-7P&YSuCl+v$9%_tNMhy+JD`X zS)Wx%`Vxl9-@NY}9O>i|Boymg%pZRF7mpGiXM^uOvqFS05Y*$=(`eRxo1MV*yo~R2 z*Eal#3(Ct_xBm7|g{-g74NUQm_a$V~tIksjKhxGq2aoQa=5pM);s>aC3zpZ{=@X@` z2SNF%Oc%@Jp~Bcg9k28Quj`y2W!a7}f5Q}oe!y7yd5#2<%eY-_`KYj_=H^a6=uv5F z6Tm?*Gh;C7B=$PBQ3xbC%I0`Jk~m029nse?9{-LLcr4IYG2 z240j@&Y{inxT}aft+ndSds%f$HJ-_F;94*$B8guaOecVG3tT)YMlFR6Neax*cPZ9Dz2Nrct=4AZRZXe#IE8}W8 zU&eo3?0s|zxUO|~m9N?4X-mCa(`ol{>%zF~&dAuk+j+U&e}Ce+uhP(4tV&zIPh`gs zpf4V|gJ^jX61nu5cfPBrxLagtp=yW=iwBK3HUzVi>A?`hr3LC?G^!hBX7G_*Re7az zx)PDJZK?3wPWvgB8GV_(a2SLb#eLXvC+gRvH&4>y{p>+$So z*KR^dN≷w_G;6%ZbRhv-^TB_U(WjgznA50%=;chMt@*XjcY1WF;#k76PRhz;OUU zbj>i#=xriUtZR@9{tSwa?P!|=N*s^kgO}@%+9h(zoL?_>tV<#Xs(wrk5Y}97sllVR*h3r`7aw;TX4U|z1l{7P7D~+;*!_j z1Q^IEO^3Cvp2p%$NMdfTuC43%Vl9nMqXf&bM+ZmwcXK1)fcx+20 zECXUEDpv;c^pE&fACm;8KRCC%d(sM{$Jk7mqB0<>9oYr;yzwdMW0Qs%2<#!w>?56P z!(4Z)#|D&v9&jsU0BT-_WGWcdNPbNRTax7-|3w2qZ zE`ba0;{j?GhwaIH$)DzkycEn-CgQdC#V_Ky{?P#(s7~BaaLo^bdk?D>fj~Zp6X@pm zj=|fba*i~5-NnjZSuy9!zFZzKCbT$<@08sc__Q)3=T zB^{6#j-V%EqA8aUurQ>+d12VXi__}BFv&ues#treWGL`NKoUQUhhc(JVoGtLEIy4r zB#+zKgaiduO|u(3J~K4{eo(kP5DfxlV(0JIgsCgV)XmoFUBEK^*hX7}2PLjmt|DQ4 zv+Zj?l#H?%x_DXWK%856QK3fPf>O@P4K|Q4=;>*)A5LE)>Us-IV!2$t+}AHbr+eG! zhN7uqcS>4E$A%6i)a*QQJn``={dbggMN%@Ub-1H~3LWZ3UCXuCdKCZRecu6~OY*-! z`~&J_>xtxw&f^tOJlG;Vn#pu(KY*m@@!~9DMXwrPhG18lU_v~&fpq>fp?ZUf6w%#@ zHK8C_JaT#>=Wk(Ir`z@qN0ZV{^-J(Z#iw)e>ybsT(gAB)aS#)$Udc(!B^)hEJE?+q z@LmzQbbeNQ7C141O?=a=4RT^|+y_5U-;KeLEl3TmQkUY9N6eK_O`krdNaPUIL3%=; zWo%Hb77;u$IKqpPfOnFbhhwptO)L+3dQ}NDirEelzUw>m;c%&DXE*;MPubZlcSYg2 zCPt!*lWSLw6#Fe3iu?U34zhF}5r#6PqPR}Q={(Jig(&~gCtp9U|AR^@9+u_}2xhQ} zkOosHF^8Po}=QN^h+Yy|ff=j<;c zyjW7hKjug*K-$_BT8UwTS9`JWn#^2mDs<5hVjyk6xrq zl$5#(jJ|QM3LW@QLHHa{5f}%Vhl6s7hhmk5be(OV@+Xmo#GKpyREj${3YZtB2#f-j zUC5&8WH-(IyXAr@0Kf*qD|hXqk3ytIGhPLuleq-ebP#pLkfnSe%5No82)6D^Bh0U; zu`i~C;EPZ(R3SfLpjZ{ zD^3dZal^0^y|H4MOj#HJ?E&!c46gXTiV4|0M%k9yR**@y~mR3JELK+$?sDGAiVP(?~; zwQl~eBZUi6k&ObD7YzF>g;4Z zkx(U_I(NvW8wuQ}q*Tc%Tu!x}T*{OR$y!no_KD2_n*)`Z11S)z%tZdew1HXH&)}p% z*#A;P1+YdW7uE^N4IJ+SK67ayZ^dFN;V1K~Ke6l`=tLDw>Q|OH(oSiOm6<167&>)lTA0>KGDF4Mby8#{U>VK+qFi zFkp?KOByV!6M(xvc1JCXZWh_FU%O222o?}q@UC+!R(C1}Y(KCZG>L(AS>y?IWiTjf z69tvvW_wHi$z;&#HH6NlK+$7?vQf|*)K)Gfvs$VYCMIz(?J#)_RTXbpmF*v6U)db6 zIZz2Xu-`{r3F$kiBnb_ZL4vAKFnRrh|A^b@A_U(RGg*l&{dg*;by7iRZBhZR7Q*f; z*d$cnAalU|hQVvu*;AV4oEgUqWD~l98VeFmp1|k$gN<|a}%2?3CE$}T8iA;F? zW#`VV8IiJApZ@o^vu1mvu{Zzy_|h3uWYaC(h1Tofw+YpAoO0cmSmQ3%x%T_{bHAAC zLsF(>$y)i1ll1`k}QZH4?Gos5M3+~yz zkxk@{mAQ*0qO8Ja%!NI=X*CVepn;|F(?9=c%Zf#AHk(ff{#tg#YX1mc%ORGvMdWp| zkq{mMw3?qq{i%+`I93s7uyCjOsN}iDkU@h0nh+0@OPq!kZ2M{4u5g5g@$bI%*|@g@ z@p5l8#DY(rh#pfs1Wi%$h+XQQ;G()P&c#fX36ZREf($VjerS#X8d+T=J9;7sUecG9 zg?8@_S!J=qC|MTww*wl)_|;g=)=U=Rum z3-7<XHD;b`Qe4?kSHdJSmKwgbzTFXtp`Rx_raImn1Fd9Z+fVBE(ajdsJ&hO`aH*M-9v!c_i0>V=`s1}b+nKb#b zK^L=r2cKCKrgiePIUXM8zb ze3x%sy_rhybkcol(<1mFy_F#lAZCR=1>qS+tM=Jn%@qno*nACEs8tZ-cKaZjj*)H> ztdGYwZd~8DTbKMDThu7cM)Roti<-=E+CJDEsI(lYcEF9}X{`kp%(18t1PcQVe97Pq zew;uz4V1iT){}de4VXsG@q?DZhQ(?%oLOKx`gX)QgRz6ns*>4)f0#7`};xF7F|lIOL%g&qq1P~1hg z2NT#6juwK~$nlJAs0f|~@C~LeST(UpGJOC~kjIJVuW+agM0dvMqBLlsrl!prpL4;$ zC;s)A`W5-A;aQ6?JJ=kkavb1`s9pi^K_tZK)3Sw2rcRo+Wy_B4J$m%(*YB&TllSi3 zo10tjlTSYO1$-A@@|#9Yj^@Mv_f!A)?WLF1%c+Be=>c&8dSFrVV+?oRs8(VT<|E^) zmoK04*~A^2H+MOucmLD+P5W{R(t7_h&KdjGyPGzy^WzpJ>O}JFJK&7w#~i(Q!S|nx z`>3S2?5wlSI{u_?5g=L8Bwj=!Pe1)666||p-d?<6!{Do~rek=2?3hK1=X*T9ey0!U zcw*N`DERC%&yeP$_ugH;ZvEijU%P7g@)$24!zp~Hz^P5-j6U{6+7*2GDZTD5I=_IVew$0mr1jsrD=R`Cd39idRs^CO<= z-}|&LCQU9Z*?Yl77p7;{c>C?QVb?{2E@{x@Xu2Kb-_JfBKYi-#jEsy6FTSW*vu0p- zX>rLrZ@#r^`Lg5MwP08KnA~Bpubg^B`3W$^^KaeqrC&^()T2kQF=O7&sa@;x%P(KE zX3e;9r^o5%Rj&`gZtY>gfodmlBLd?bb<@ga zzdQAm;GR8=>*da0@SWQcnm=>;BY(f^p4)G$pIvL_q)#pz&=0YW9em$<;<4R3wg7^{ zXw=Fcl)da>fag_oECk7wTURZ+qTi`S#k=wv*Z+RO_o#}$p8n-$<30?)-FO6lf020k zhabK@_RUut)~&Vh+u6TA>-6%%Jy~hq8!x(G`SO(>w?yDbD2BX=kN zWZ~CUiV&3b_)ZB&(%jA^i{^Fd)>S$hN93J1U%T$NmoAz6O{8$opDww0@P+3spFcNT zwEGX|4cNGL4U+iSQE#kR@I5`rH?wEmd)v)VKlVsYX2w5<4Snpv`_W9$Z!y}SAQUv{ zu<0Beq2Qa3J#y9MmxYT87tWk{?WKb_kCc_n|7yks15S6ugZy6n_ha|mbX|7sj4hj2 zUv_q1Z>TUmT5|L629Ny5Be}I}e*VcvpS|;zMnCElE^n%GT(UisT&6m`tvQ<*Un5E{rrfBhup<;LV@##-~Vpe{O=pr%e&+H>!wbc*f1}5 z@poT8I($gL5q|cm$EHu2+USU*vg_v6Z-3m%B@2Dd$h^687cX9XSbW8HBdQ$-s-48G zo+=hw$L5ML=jE4PbVQS8K}mR;UJK>tPntX>GrQ(hzrEz-_9x7n`(5)^?a*YSRft8U zQ30zsB5RwmIa&;0MkD$SYu7p=!OMShN#iC*m6J%@=twn8O&1Tk&=D(Rhnmrk{qx25 zKB!Z>=8e}s-2eRZ9)4gL*t>JAQ|htBAEhyD51XP@|I&$9;IeCsW*JvRLGv(M;y+5mhDPMQ39X24%mShi@v z!Y?LFIHT_=YLIj|C<#P`7e7pY7!7ym*7?zYJj~#u-Q>xBulLD+ zJT=ed>X4R^O4I4Z+{Qid?d-T+cFN;Sb z9b32j?(117krvLFHu}?vt=qP7#Y3Gsb(Gc_R!ifKj+*Kzm&b*=2Khez@WT%u+pbN$ znt=!J`unqEKkU@0Q?rKkhp_9!6RuD=GWGM%&;k_~7cQ7LcgnbOjR%=(ULg(}N<4;DU__2e{fvU%WYA12yf)z~Y*sV|VlX{=o7l1oL;Mr)00*svu?ds?Y{YTjvK`)8WQ}iN-r(#fVN<8`p~s# z6wkaM3RbVXL-%eix^+44cWn@?nKx}Fxn|8;M~xaS+a1UD3MkKe z^y;~K=_1*EprFN$LQ|~qV7UC{*IqpCxVGK8cAPzR@~-^tnoC5_j&?UBy#w=nh^@@p zZCt;$$q|j|bg-#$ySyiK>Ku0a!VYh%<4)|-qqoKdF>l-U9d~ctO8=qBT0thAmHXH} zr}#26qb}but=bl@UgMEBT1oQ32%DWbd{{05uQ12uZg)b*h|kyPn3ni2@6xp=s`Z8q znxOH{-?4MkI(o|Zoj2dw;fVSjn>1!GwXeVVe&veQj1wrF!~#OZC;vaQ4EZQr45#OZHFT_vUEr9o(T?S*H%ozT8}rw-pt zpSE}BE;`xV295r3*FBHiHRP(BZ)<)`D?Fex$y`6T!Pd=N9qhm<<(0qceX=&iHU~6g zRLz3yCt^-Cu%kx3Idb&q2~!)#v?-HE zKJ?&gFTb#C@#2B!pD$|@8OS@n_~MH|ZtfS8r?M%sNQ_4EcWi4={|KpI(019u=0Mft zK(zyI01Ciavt;S~Z{~FC+WDf3F3MlLED9XPoEw%b`S`uD<$HHM^Y|lqA>^DY>;YvB@hnIW3_yMQzP@z}Q} zjCtq2A@@WJc9-wk$tLP&^gnCDgb5qgZ}R(U%=u>i<{i7EPNZGzteswC?2+57`Qk;( zks`Zw?t+&0-;e*JaNDLu-+lewdmo&0!5~cAuzId@RMRC37etFoini_aXV*TeL4#FG z7QFk;n*ci7*|6C|EFO{)8f^n))4RA`N4IRXZp~UOJS4FLtiU%-kolupB4xE4U1`RV zD1e(*gPl_MsE+_RX}U0#eqrG^vq!)9tZ)#76-uioArsfaeR?1o#n<^85>rZPxtxSKhd5#4|!VG!IS>W-7yOy@kyR&OLoV*JF?Ce(dp+ zr%fMl-uXu~YktYVi#oMx^W2NClxsUOiQ+;*rbFYgORl)`>3=-dy-(lly7itM{nm%$ zKRNZp&V$cBx7~?duD|sTRqFL!Hu$QM|9G-jpT7P3pV^?{k)4n4bjjscbnnwA=JAx% z$$0%yuOG`j@Hnh}bkKNvTD5C8W7;%{+nxaX$D7$pG!aa?A}(VZiE_pKKxnrw=x|~6 z$F}!GhGTB#Wa+pfpfG8o@lQVZVAs}dU^Fr{N6agQ-|)xnfjMlyN&g{}Kxe!RSs#59 zc9(A7b$`5N{>nA|PwLgD&uLSq&4iPlw2YgF4}a;2f1P~nv9G@IMyGyflAXc8Q%^kZ zh=lyw>k|fRD<3w|L?we+AgqG1)S-Sw-6{Hf_A`q2KVv}N5lv1!?u1JQU)jA^ZwgN3-hGANt@!BOZKU_3BmN`_^TPo3?1q<9_(z2VZ&RC75Fen*&vj1Hy}g zO*HdA#SX{xl?97-hjSX#@nPvD8A`&Q6|^4CSbin0|9}2jt=Ly4Wq{E^mt)(u!kRv3 z%O*7fT0IC9Xx~LyF|)pA@9x^QYa^nvPVcD)hp$|@^68ge-@0pWt*rF4w2UC)92#P0 znTzFz8d(*sDE~fT*i!Idd=Ey zw-=>Q!LA+sk8d|?#VUtCll~6{2by#iATdNRI~xnaHHWE52iS#Hhb?|Em1j;kSX{KH zW@Zi85(#UFgoU}<1qb0O>LW*V=-=<8TfOVPM;UU2LP3oO($Y;!?ZbcJK;pyFPzQO0 zBkve|Gnc%tv2wR-UC6Ou@%H1I<@wA|Q}+&XU^EwEm=$#tZQO8M&&IWXSe+S|X%u+a z2EKyo6~I0=f#gHxyRVf~8=&Gn1X#5|+wBiKJ+AnO>Mu}*&;(CfCJR~EKawpYGjei6M*WTr zjAmaQ+Ey6)mD%VSXr9Lxxc8nRci%l^eD8j=#}n{ssUhZ#;2_hC9@g1NLMJ2rNZbcR z^A*Z;9n6>dUD^`ho5n^@?sjsUHf@3|q?DdpT`;6dAXWz!f;3XQz2GtyEhzTsStuB4 z=mi{gbL()IOpxjE`oj1KaC-By+zmGJIv!xi-;|xLs=rzK=$gC@`U|A z*ORpP+m+$sIX=K#KeL_$C$+tPydOBRF+3P4a_dZyDKU|eBZ zSG>H`7YPR<5ftc!TekREUXNxx9uLXO2N3B@1DJ}7)-9bED%#2FLBvvoRREWIJMjNWDyrgj3mVzCdgT;INtlURWj?fJ*Bh53*imjiF zQRrHr4FNWK#T8c+?A_x;Wb7E*)VsoEYZonv6cjl^<_D9bX1ELmtB2B5Pc4-A6*#6AYIe1BqR)g zo(_PgK}vOmEIc(m2Y?R2Ff>M=>T;Kc^t_0GKVed!=#R#mwQN0Q!P2bU2BO-e*GA2D zusKi#IZ*9@8%Y*vjg{gvzL{_1fi2d9p%^SOmchHOSDr|9v5`3lep)DfJOc?Iiw6RyW zE`tURJni@s?)lqaS>kEIErK(yPG--X*7KMagU-L8Z^sTFj2(ly2}mofcRIt7(C6dF z^*jF9(>r$T*}B!`mz=w5$$V)xoStwHQJKMjT#jIl!nEx6EgO1v?|$w%XZP*c@$SF< z*&QxLVg)&+wQxkfp7G_$&6-|z-g!ORz|yhkM=$}gy2*Gn^zr*+&p!UxQ#&1ha;xT7 zUVhoiWs9P*@V!HZJn`5+WCk9K2QV#^I^Id|6w&0afqa%YvCfqa1bqoQ6VB<=m}_8& z0QsE3MsPX8U`0%kppA*97j>4Tb$TEk2tlq6ST*JK5e@iLv^Z!7n*&vo1J&-`1&Xi} zj+Ykw<+^Kce&CUBm#+Tv@CS$8d`nrdoH09C>37CgEu8z>BM-hX{?nO@SNH7Eb?7~J zfpH**UyIyv0fGWLcFLeZUoKrda?Cr^KYoAAm^Xkwt;azaL@oytN=qNQd&s?a-^03^ z`|lglrftiw7cYKo!e^ho_VR*nXUD_x@dR2$%gb)Q?&{lz-uvCM)z=Su@cuvE6fP}B z-C;cy*2Q=+D6Cg^`ihmWPMBC)RQklDk6|VVrb}RyNuGpC&?^=RK6LNh9gl4@bMAMq zPn`Jq>#xk4Gn4XV0TXuw%Svy(>dIU1ANtLTmDk@t{DIqVEic+D!5S-)xYOrBbdF@! zuQz?gvKK%9ELyVr@kbs&d`7I5nuErGz2GKojyi`Axu?T%ZD%i8^1`Q|em?ru1#`Xx zEG^TNP^kQt-(U6TVfTN#Z23*Y?t9>t+k(3bQJx?&(+!}N@KQ^?x|5c!cEetc9*qsEA!vfh;gI}Hk(lO3 zaPeqtA0_+a9EJxWS6mvh$e<9ViZOfv&EbZYo-+PGX(ifB*H<+^U7PCa%zSD z@rai2A;cl5q9+Zcr=8fPbN#k$ckS8%e2^FMlR`kN@$bFYFt_e0r<{Ue$?VTQx#ss* zF;~~Jb*t80J5Qc4!KvmLIU`=M_u`8#LdLx4;>$qtZ@!vgL}3Zi#;g#xN^Vae z?U*)gyL9fhW9v4_wus4L$^MoRGc}eJ7k)kA4f}kn*nL9!J$>+i6&Q( zQVR=9=6p8csw)Pw2&ql`cE?c5iw&-%(yTsY|Bix~k7qLx_v+u3sfN|s(q zzaI=~ol!KDmQ|~HtCpQRb=J zrm^x9QPL(^IwHC;L}#O}p*i$2>T=*DIUtN@LH9#uX@mfMvucT=7}QuLSX{D?HV3K} z2ddrVB$Bho>)NnkU2f}^?zCDA$pe{@?4}KN?bymk4vvRh?sc1Yv~1N5X%z5kc=S;q zL*cgVIaxJXKpv8-ISjhPJ2$NTcFL!-r_I{FV&(adKhB&Z00_9dKIo9O&f3BwjRe&P8XvKFb%_tNNLI5#ZxA+X#Tyo zKK$#@VU#Q-2l{Nr2F(942jFmQ+Oi59ZQ7y@8Nt3|TD01-bvq6NNShXSZC<~jUc2KQ z>1oUzGV__!tiiVY_40krtYgfX<{>^_T(W2RlnL|aFMR*)F@L@1KI}vQOnYT`sDILF3zXr3u4S^L$qSttQJQXpH0o)I zAReKfEKF=t3NpN=01_pVg^;wGFm;%67-tZaOhNHJtxI);ElFMd1T-XdGNwANAen<^ z4-}C4O(`_ZMBqe*V3oj-LMb%Bp>~}^twsdY&V#WE5MdUnkRodzc`*>>R_&&J%8E6o z{%KKw%2Z~&&y$XLy_=@|L@i3F?v_YeZM6a#=)=t&F-Q6IB-|!m(ONYOB~}uP9rTP! zA*~awOc|V*LhQpqP9mFOz9l^3g+pQ@D%TuyzdvhNBUmH+|>sJ+*4qL?jG?jgDAaMrL7gk@d70 z7Vu?q3$4pv{&v^y_3Jb0=lZ?R=p!IWYp|vB+|Tj^Ia=fVo&MvqZu+Fh5ZPSAxzLE!9o^`Gn`ui zqySS$q!SYNF5Q3Bdg$d{fG;8QFMmm%$ek!7XDb?fAzn+47A^--%JC>RJp)#RO3P4u^WFr@%Ijp;!LZH;I>e$ivuZ@j%CKg^j2-QJQAt^y zoLt5XaE)3ZggMfhb?bgI``hWO*I#|z4S&7%4-6DgQrD}O3*O}K*cHXOikX=uJ#Y{F z(|7T(Qtzi;R)cyJsZu2&O3JJFQ=3vUVIn6YEI%bC7f%6bw(w5ajnc*Z_SGUGR9B=| zNy&3!h>yU1gKQQJrZfsv|Ig2&L{oL# z{)hjQrTsA8zCM^7*!P?cW-XPdYzcrKU%mPbH!fKcD=Wbm1OVEzW?jR^jS!IaEk%P( z8nbeGA)D6Vr+?MTHOREN4eINSN6mtCif&nQJ2GgzR;^mSd-vYCeEH6uJ82}~?gQ1F z&c;VI*}82T0(--T4T)d5aycdYaL2K5;gL;`qN~7F6LuYS^Kv(0C{kYL!Ief?>E3ng z>fwM1I<&?I*-qBtM%Ao^nrEJV2H3Fp`|sHS2R#a67k8ls46mD4kI!!Ps#Qq*@mOf# zlEp_hYYHWf2xi@}h7B9-Sh6%yT88O(SyAEMwdvF8cGg6-UbaCY+AT5SXxH6 z7Az^-y=Fs$#z$#MsxSo_aUUY*RzQi*7wF%=KM*u;-aKS;{D330*UM!*fh~;m*VqmM z^ag+|R1m6I!C+155`-7%{3w{W+TfQGnA@p5TvX|Xd?pUO#rU6F-pUb$aDdTgsM(;=`T^WN%!9m zl#^<>&Qlas$U@`~<0Yyq{2)F-Muof;ir+8RzBt$%;2o%D0V{=Qylcgw+b2z2P*FDs``#N#a&;v#Uy+qZ7RI!wHe)To(#Y>z%KJUs&E)YGRXj_=$V*CGhcP9NVa%1$|G-Mo6` z;%{c(H*^T!N&D8VzW?UyS6>|Ih?IkIxRJtIu6FJ0cD+u1{@JIa#d~H>owWM9ug^Z` zoRYoypM5X}f2ZBMo#aSMd-LVzU1deDz4&Zq<0IO3JRUs8mOSQ+c+p-*OMKDt?W>lr z_pd+7z-BO>w91>o|Db)m=hIC;dgBjdsHnNz-4{`J@A3_RZz zE&ueRaoe`nlx`jYh1XR6yw1xpX3 zOH(bkOajtaIZ`tJZC1&z=Jgpz#>?z=O+I zu0H9QR<~St!`&mEtJmnLc?%W|zvUJr&9t<@6C<8}a>UceHET9{^yrZoP zkJQRQrZmfT?mfF_?<)ph5vWn?jgLQRSiivs@4vficRqR&(PPx~VB*l>4^T_DmMw3) z@x~!fkH~9u#KJ|3hyU@GjT<*+)W~{b#B=}p*S|ZoX#U3N*PngqMO5L4;#9+`_h2{_ z^LmT(cb?g&_mx*%Q6szd%O8G}SGV>@<38BFeLE#0*tk<|1OJIiHbmP=7XX3B>pU*!3^~Yw!rj|wxdE3GwHi3yx;~J_IqP35F|cz zl1N@LTZIxhaOD=eAw!Hso|cR$=s_kWQtVBNgd(16F%vzq?vPX$ib(*Rg{q*x6=P)q zs}gdtxnVsyE2Z)!#CFa!-)Y%qSn*cX=(ypysgX3AL57pcWyoQ)1R2Qw#2E7vQ+0&{ z8I3TWh_Xnt;20XnWl>_q^FR|#Gwm`7h)F0?$YmuaS2CEp^nt3yDnosfl}u?!Y52oU zZdkn9m)qP-G%Z;pydqd_(W{mURyO0SWMjEw{?hI3o8`$D(6168()LyIt8lu~RKgHg z044ZD;PmA+Xow9pFEkNgOJ=QFpdrLE!|2P%ZlQqX$Xkkfn0V>7 zaWcxME9Hgkn(=cqWcctQLx$j~4mU(?7aniRY%wFfBeJ)A-KC+ zaCdiiD;2CoN?mAiN-5=O3lu1Y;O_1rxI@qY2|+{LbNv5Za}rA1=idMOx%X-N-WPUH z!#?}W9$7QZjGie}vW1ZauL89Mh&v8~o1O`3vRa;7ww-SY)B$ zc5fFeY;;lrgp~DP7(ZE%tx_uV$&#aCDDuqB0TkRzg=muSGg;p7MnUHycP=H~(vJPhj-<|g2-=ATSsOttaY51*t#s9pbjsLuwCx3t~!4K$+xHN5N<%Tq~|3(k$x+Wlw{S8?Q+!527L~krJ!| zSNHGVe>d{Mmy@SLD4L0_F-D`r%|cJAtS5Ox#8GCl`Utd(4=05o`|F%U2$(Z%GRkq0 zN0l`*?V%tVgPT7^3-gf1g5A0=KL0p(-dwP!$U9eOPx%rXcr0cJnB_9KuyZHX-p!kK zZQU$DB5%QI9HIoLB2*VocPx4EWye)XbtqMkD#;(*2 zlP7%^`}C0v0NDi=gmnW^dP*uE$OwKV_Uo4~ z5T6}iQ8M9G!BfQHWG=fdacV1K@jmNwpO=9`TRyOyE z@$tB#B0*pvBtub(kW0E*=8c=vk%i>*`ue9RkkzsORPld*xeQ?@zftRY`Z)65of~+V z;OMb)*S5ur7Rn@Wk5Z+SQ52Xs8bL%c8TT5jO7M8JxwR!gjncH(!Ai+YO*)aelrYF-mGZ$FW$7&X zWkS)SW>Lu~3b>J!-~jfEZo@K_m`N@=2?f?llq8Ml%jE^^L&y#8A@iDyB!-D#Fri4C znwW==?%upEF8@yF_uqfNcmF~1p~iD^n#~}u#Q*(A=ZPrYDJMZ1P;j&plL@I1DnIIk8r!)va0rjB!vqul_Tn`fi7>TJX zH38h@2p`e;KwJCXUzaAw$134fdR<->s<`^~?R8-P9{P>?GfqH%KvUXn$BR`B9wXL1 zuCTDM*K2mM)Wb7KW)mCdEd`bDwA#VUOC34!@P{$QfQ8Ii`B0Wynh z3~t068IH_9q-ml{%W#!=)OeIJ6(lAq3w)*|NfBT4)8`^qeuz^g|V!_SrT}10jJzDNj~H$8+`)62#R5_m9d?Ye@ywA3+VC6d3!SJwhza+Qdjf`F3Ml6v za9E9i7SU%#*N&JcbDnj801!1$xWfq*8cn0-El=ORYp_}6p%~Gd(%CCTLZ@kR)|f=D z)hQ28kzoavU7lL!^e;XwR-)8LpMFXX*iAw$QKrLLomLhJG|6SN%$Pi>M2X^+E0m`u z{6;n8^fH5%V;#UO8TSwioP*!Oh^3FInQq;>wMLV+bC>o8Fc02KqoBN)a!S;4{D_Pa z01{4XXxhBl@oQHxbtet967CHQiUSs-L#PAw(WhmJzjEU;OHf9u8H0ao2I9C;Li@r; zg@p7ArC&WXD}HQ`-kDkDpL*B2@}TkY~8wV?_PO}6?L!s zpb*1UNWlO@t)>iKe+S*5f7V5^-@ar&eEsi#o&v8NaHBwF-frKx;m)-y&mKJZ$z<&N z#+zZ+Zzjekb?n&j9nTAaoH)+vQqcZ~#1licI zZdF87WUV@Ny7%m@W{OyVAcPX(ddUxS2M!$Q>+io{!GaF$S{*uch&6SuzWvISudJ8= z@)c4GmeID$m(HIKJ#Npib?Df+Y_%#Vd~q|pbpG6doqLU1efth=kAxiW_trZcJ77sk zT)%qN&0BY>R0}GQI{;CxX}h*3LQkc}KKJ(WIu;T-dGchtJu`UOs+)Ii<}X}m;E*@` zef@#Lrw^a3Tm5TFQgV|fjqgT8)M?tPWSO#wvCmH)J9_!T#h_}~F?PUfbp6^@k#}yz zJh?Y-j;8nUG5hu&C{wOHM{W{fB;wv(j!dHoWy+N5)4!jmub(5+vT)wqE*(4V+`k`> z&3=7)lqy$|2z|5;aowcH%3L7$VZ`0dTX!bK#I$YSp=RSoVEUe|TXJ}LRd3M(^OU>S zZ=61NcHr>gcO&la-@awy=n*1Qi((v8#bA1<75;{b16_jP(WZ54 z@87xo{K?~avrU7BznLQ-@b>N7TeoeFjeb_TRLNe0hUW0~*|2VH_^n%yq9P0?Q@>%u zjV902p&{qboyn6cpmXQ0f%ytz0t9jk1R3ZxE74dr7E-Z%gCW!OWnAl5^>C6k8{b9!XA!_w+D_gs43G^xn05--*+wnl^3PtXVT$ zNzshmyLn^7hK&}hvt6goHR{%f2jNb-d*_a0$B!1upFf|ECjtQgjTBApU^d7O^8s^N zz-?H$ylm-Gw{Bj)dGkj5HtlNFskdz9%BV+Ax^(GMvsSGPt2I78{>1Uamo8qcQMGd0 zcJ0mnfm^rjaAcCv4<$l)VL9z1?FWXO=-gZj_^>4&Ma=bSuy_QLtgqsEL9Jsu1L zuU#Xj?}>E(Z=i^00`IS{zk>oY%U-QZX)Op7ih+h0xQR$8NAB+4vBgX4yng)9v=PIC z7cUI(^_nyB{kQw|Ui!;ofZ&VaLw9amhgDm0YHX*r%`TljQMPE|xie?1S+)!*5@}SV zOxBk6%+=q{NRNJ^v!t({^UbhDx2f!<0s>W4Yg*Z zn=;dewQZU3^pTfAbN|xC_queBxOdB>x4+q^%c?okY$=HdX&;UmwP5zF0)c^hckcN3 zjR7YP9{_PqpE@yp(&v*u{?M73feL2C;DH;K|5B<@{v*40eEq?@-W=SMncS^egWbEf z=FaImcj_05CVv@m0f!C;2jbuYX)~r-kB+)CwqK8Xw^3_Yj_lim-UB%k7``2LY3g3QTM`wXU#A<9Wjq1mQ9~Be)K4Bz5d9ytz!lcGC6G+$8!t>u_VQViAhH68J+XO z@ne%my}jbsB|av@w;zmqt52^LOBYcqM%SJ#>ovBFOXp7wY}3Y`neO9Z7~7|3_^s>6 z!asiZ?WB?Kc$m#tc78Ky62h_c6oe2_ol!>|L}oAy&%R~#uOkNbxOM$vdScAj0o~r{ z+v)bT%b95jBfE8c5Psdqq?i8z>{&<`c|Y>_@#C_lQ35tPLaT&vXAK}i$m{DLq`)gDaWnaur3l5R zLqZA_Dl~fRI8pr)*Mk+MH3KED&ck!rvSo-r@lp4WY~Ob5-UDwBkEEEVr;Z)z(Xa1{ zUw-lTGzb5(h!uH2j+|R}?ioCID95#AiUL%$C`?hE-9qSBmg_@@4t;OTyVlh7pfVNC zo<7sMb5~R=XtdFi7b=kd^yN#)WR}NKtJegF9y{8jSI+~xcGPR#>Zhf@AiGv9QTU7Z zKS24InUr*H@80!$_LQhp1&w9XF~iZ2Frj@ai1E{AysSQmI(Q2WdjwOrcPoMH--@d&= zLXQomZcWvLGLCBG#jT$y!^*(Iqpdn)> zq}l9^if=l9`b6UnJvf5hg4sDr2~?{t#}{9G(Y$$cP@q$nZU+w>C|9a97{p2&k0G~S zUbbAt6i1Xrq(>74|3I?}WZXP?a^0TorK?oVaM(K4s^NwSuvn}kU7EfF28JFBDPFAP zyJOyC7h&W(GlN>RSh{p6#|(At)w5guI%kd_>)ogO(IbaS6)!RF-S-?u&(0k?4(i>_ zo|$^BX7x9E_Kv)Nk0a~g1F~r>JFkR};bdq1!-&WnMT(j|yify+9@*P_`LY$n){TAm z;K;6B$FJY?^7BiNevBhkmp*;bLQr!YLs=8B*qAl4`SBqqM^Vm2i&sELn}>ORCagF zUJjiJFoh7fL~1!nOQQ?I&(Q3FS7Pb%6%u1(4Bp-VN6C^U5AHu;u)88|+)j#&s$QzJ zC^2Q}ZUU@P{<3!Fq!sj*O1Y_Jt<8@$>orQVyS%)8iw6ZoK6(s3p%7QfL>rEmlc#@u zVCODR51S?R@t~T1fq63Qx?4Bz0ut>SHx^B+?35X1&i0<}M%d*~KKp_*zB25Ze1(d7 z#$ug1+=Iyl8a z!IQG_){04_meVYWw{1YWza4h%{m(zf5QV5+d5RYI%NH0G5w6iR``ip)v>qq2bB8fWF8YAD>xSFAw2o<&Fda?TD%Xu<{4aQ9oVxI0)&U%3O^CDd+lmv>Y;sFKej{L z*1&PY0GmK$zxws4BVb*pe!ZiiCj>b;P=XL(h}r1tIqr_?VDzam)U4OQrZuyKC{w9g z?fMO^Fb-ffSZzRtu{bmgo7Kb$zBah+N>PlO)5dYzUa z!xSu-pPR>zALlDv9MZ6{%z6rD&P0HooJc3U`StaWQs9;2oGhfuY5h}ehp zjd}o$quT5da{_`*p)MefHm94zBFvs1q9Q>RskDIT6Mz2chkN07_a4}H?&A5t5+#`b zSQfCi`SeLN#bFl6A>~XK=A<{L*`}tZSuE5F+{KsyeJrX2q|boB+*s)`V@sDS5BeNE zb^6G~i$^YBK6L3ywQ4oJy}je&;%G&t-IsO#K?8 zFwk7=4n;1M50FGw;#a9P`1pE1h>8%`BsQ`vw#;YI2&#f|#Gmr=71BW=(aef!Qx+S5 znb@JA(9or%gZ6rC%)_qVs8OQ^Dk+%R^Vn#FHxfU3@{lb-D||`4Bh*F2WeKj;unf|q zq@-qM(t6m*m8)T_)af9EN`=B~n6V2f(Y4e^WAO2gdianMp>3+wn*8K(E`NWi57
L#Sy%$FEk%gvO{bw6ak2scACZVfBSfZ(yYnwE$U=tALUqh0qk^1K)RD97 zF3Zj;jM!DCeEETIy&ZbxO2{Sp@#3OI3-fVAr&dF+k(~VR+_^)Eq-Qr2wnrP?7z%7N zdPzfuGUTN9%tn(J-HNi^Xy#A~4M8G5ee12!BS&1m9X@ONGW(x(Hkc51}YmusG>z8v}l1*CLAN#f;V7;wk>a8xpe0IIZq$2n|JSAxEaP#V+?Nv zTBHN+-E)Yi7?7|oMSPQUfn81u)*CX<#Z*@t7yUe6F2B4veD`kJ9vczKoOBpmZCbV7 zv1#r7bD>FhuPmQ8QoQ=|rac#q?sdgISoXslgwh6e>xoQFi0QV}K^|s>8AHUo&j+Q6XE`-VHt4qJ+6>--c?#aUb;}7WcqqY9I0pzF7q)Tau6GqJT{QJ^q&+>|Nc?d`X?EizPNB|eAci(; zD5KtCuwmDZPE4>Je@(WYEE zD>Z7dXz}v6hY{CLg>GN7!Q1Zgwx-ykBj!#1T$h%pb!DPrf|-aDlw?o;7{yQqBO46i z=^HqP!_F>3dO|Xm+$D>b!nHMcX)w$L8}Zb{xUu6#f66g*AAbDNN1r4l#<1BLdG}u4 zGNp)N{NUl8BS#KNY+&+Lyo~`YqJj(o^`Wm{zJa)}ufK%?uO4vYgdpen>f3efaOvFT zx)mx%J$)j=G&asG=CRzBGq#wUs5y-m#yuB8I!Bf3%S%+oQ*g7*nTg#g9Jz zJRvbfKorPfb4zTyfgyv_N|MorLf!8%U4>uW@F)s zRS}dBB)j(No>4O+2w zf>2@W`etMsjvNm67Av@wQX~8dVGW=-sj1hreC9N|4p5 zt=q6ktwzn;wQSwFd(Wx_F~`tOZ@|l+MUI`LnZ;0zg}c$y%V+H5=?mul*r-~K6>HYi zZqqJP$lxKz1Os^zflJrEOSki9FVw79`SG*3k?)SJ->}Ju0fXyTtF_m_h)tXV ztRZojVs~4aXD~Jp-Vw_IRW%-j8IY`f$Iit{m+RN1d+mx97cXCKw+W9PJ$C%LbLZRF zYdG@VajiOb&OnIPXwO}|bUgHw$?PSA6ZWp6!H}o$7zoAQ3BwO_L-NN~RR^85g^Cq# z*{}cLp1qsYty8c>Y4xI(5I1=X7U|J+gP$qR^DHWzz+8R=K-if<|jjqX-U56Sij8K996g^6%$gaFjdpt~GMkBv9} zFVc{`d-twgy*fBJ*s7H5QfWq!Xwm3gw+*?w>CQ5t#AEh9WR+GJ0rCCZ!8SW@4uc z9LDzn+!8s616eGb>Z_1~Fj2C&C4mvR(dazAaxkwzK;lU;kua^5xx=(|S+p**2^STS zL4g_uyV52IasiQe12VIKXZ|8#dFIF|@jYY}N-bPRLqZZ_Vrti_zWLa;#miQ<(L|N}GCS=O*6L!g~8UnUp3TQHbj~aGaaZy4nHKND> zn!Wvu9^Mk0g-I)12}#aF4%-!lDq;UrfuiQ6v7*rszjqM>T^uh7dRj##Ey5}KZpiAL zD-eN{tOPm{pJ~4Mn+P8lvojz}i&9==G<%zko&;}UD@+_e&?G=^S+Pq>!8n7C{84)F zJ1sJJdgt))M$1DoszGHSGCa@I18kBSa&j9FEdY>c6ohZ{9uhv{Dli87qJu!iEr z=Z8G*1{!(-F&80Ec5QeOIVg6GoZx=4;!&)3^^lX3ld^*`uh)acfpuW1khS zTJ`%4n~Rh#Yq6m|muZimoSaSIutz_6V9T%-EmaZ_6W31og2{CF`0>TFXCkE4s9pQ# zmEW;pr5YtdsuanGOQE5x6=buEIf@6ls6Oyak^J?9tHm!-oM19i;vYnYmn>OYsZhZv zb{EBalFYz7FbzBvpLnoJxB&}lPCmmalDq^=MKpK9y|8O0FnB6JY4Y`hA5bF5Ol8>h z60uz&ue@T=Wk5;k4XGr}gh$XYf)oh%Ath-G-Sn^wTjD1xq7j2v|$^AJs$&rYOppkpu}cT!M*A= zvq^0g!dl==!ZXm7Xj(LH#oCHAiiYaMWEMm&13DMhSO= zfj|muD`^UpWU#4Lo6P1+C}0@^AuoA$ejC2um4u65~KzX1TCAbKMoFO&xl~D*A=Z_vb?ZbYGCx0>ZS@%&7p4`qJX?Vc=Zkw#h5a)b>+ zo59F8YC!Q!#DwQh-W)bKChP{HT|ns)v*ygL98?Q~IdUaOA4?jQNAJw+w18iNW|+>Y zy)3{^w6|hA0~r7!N)x?}6=!M*bRj6|uxU+%=+LZKwBTn>a>$ia6C2xZ;Lxw8ePcnL zqCgeJ1gmixV8gYJI>8sv4oQqARyIg`D1v03yATyo!U8=`&DOQ6*Q{ByZp${TBBXl2 zgDe{G5l6C?sK2C#V8a!*b{oK^%_v z#83w)r>VfJM4J)J%A~IiV)sJx(A;2MhoDM@agsu&rV$JVx?iMe8l@0P%B6CuXvTvR zryBSTSjAH~HVQGP6Ym~5Oap^LBy{9-RbjsJE~X%;h^Zbd4Y?H-qP_^nlWHr(lg(lW zN4UY8I3`XW%F;o#S5_NFYRHxA-0<#uU*M zf(r{|nPri1C<5oA#uM&KkBdMeJX;M>wN@Zk8|o#fqZWrWljg{5kP169^*T|N+<020 zRDL!D{vP0_s;_uhS2gn#9=#%eh^!<=Xh5MsKjQN*L+?f$yK`^(x(y6W(HL`vx-6kZ zu3(!Uk3^&?<{C4B^|gse7veL*auE%PI85R=r?cTxqqfzU0IX8MKZ|wlobP+|8(6$Z zaqPG9c2GAJz3hk0$BLZ!L^-4&T<}2tCX_VIq z9|*;`s0k&JE!ugW3+%cJLS+Ly^P9vPRITPGqJm)zDgrcYQsxH~MM$O{lqQwJY)C|G zsTCC=cfN8jCy%ORa3jE(k1PP-yyU}Rfi1Iw5nvVsCtF#vkg!_9U!&CindyMmeOr-3EpKaP(K)?)K5P06tfoU&QG*g$;{MEn5c9xE!G zI6^MzeclrCiuDseQ>Ls3WIZMLEM!sbR}`R2;Xn$trJ&c>KSF_5PE{fT3joS^bvgb0 zeGNE718R(3rlH!jWWXZGoH%wYBO%VqY&v@MX!#12dJi1DbLaL`CqtSyY1*t+dqj5j zCl4GxcslfSVBUPa2M_ZL3}Czi4NSDi2N6d$Y&fV6RP5EO58%G+mqoX3hsz`o4{X5; z%0fTD5d;hWHM?EAcKi12BeGzR-n|0+b8r+6zxM3izd+H#m=IqJyK?-(`elUXp+wVK=XG z*ki4_4OXsL@%TYx_Z~fnNCHMWGp&aXNYISH+_`%Vek)I4E+#y65G!a9ne~TOKYGkK zA(Gxux@>umJb7M)wUV#A)r~Opu(c?!Bu^mU~jEWU1fAGmCpM3OTr!JkTryBxgm$#^H!Aa2DJ|%Pc#7l0TJmi`DR8__!#&aqYt7oKAwH zfOjNuU*rQ*=PTS=1Ys$e3Q&wRgkKeVlC->NG>RIBpM(gqeBmw7T%u43LuE@GRzN;V zOr|DOUg|~=E0G4|{Af~1@43}1vUea6zrcFNj40CiRqZE(>*`)Mejp#ZGF4jvd})Hy zg;G)0LnIbfCu~*mLREW}^!1;AgaX2+U#SbBQ;BC{1igLhhF=%``1pP};wWo)vgRquept_r>z2&V<>&SJ+rzhPUe5@h{OLzae)^HnOE{RZ z<1e~hW+PL@X(P%{k?K`BN0>)4`Y{C7n>84;$-G8Qku{Y?+_73#&H3hy{(bG48JEwU z89AWuoBjIOE!Hz9Pxa~C&exF{b^X#?ty`fH&X+siwTo9x1df*3C7=Q5Al3m)T+BUb zmCJtU&;h(9`B}tCL_ZfHiZmNnE+08$VEEMwwuHEe{re8@+c*5m6-!#mo1Hs6jJW5a zaZLSW!jgG&E0-!A{qS*20=|@@G*vhv79t{(!f6C>Q1Unk@Ra#Hrgr@XvO~vOfw^q6 zKYJLl^qXlPy+1y{$9Lt@MQ;rqIAP4YIemPWE&OTb-G(6CJY~1G;c2C(}>7O zY&oiOy5_T|kPeU$gaRtDBNV`u_NWFbSyuru(k$S_DkPhcDBJ^xL0u7GVZ<7?xmnHwO%PQTcP+nwfhzV#aSOnNapG$>AlI17anG-8a zex)~6nF76Jp7mF;5Ve)cHl`#)=^(>^(q#)l_@HV9O_owr4W%LSRp>6TEbLIxMDP=? zrjkkilt|6pW${hYfWt%&pqXSM_M>WhRw^%L(I^3WSh{Fnq#;U1k(`7wDodIM52HDJ zWYMAWXK6?x+N;PTm8Mp_B41UV3;Z~N|h=hp|DSgeU>LiCdAQ{Vr4QgwV1j*m%^A3F6ZbG?+h9~ zqEmqdf=>`xJoJ=W`%)$4be%zHW@oLswPwwlfsUx^;llti5x1RAYmH+^&z(6_wnXvk zSFhBoU1xCrzC>M#H>3DfhK33|kOUF;1lLdp=`|^d@uNq)+2M_0xeFE&OB>nrV{2Bl zscdr1nmN0C`SQ7Q`po|Fs~^{GtY5!=v4VM~Oqw)f_BY`-Z{{mlpl`2UI{yHw#SsY} zUY@$Vxg+l1t(qrawxEZIuOv``3qc%;Rl0;f&JO`21!o|&x)2*7Drh9%{$)9iBZ}I$3iK%|pCHykG2|S`o z46pyq^}qks{h~WVVIEne`*VK6rMPIq%ccK0-Rmd+AO(IKCI4V0{?4K?!eFx$qrJXE zhK?Hh{?g!86Q@jBwq%h=RX_?>J2ag`+FHi`pXVFCJb=V^#%782)7|Jd` zJDFl>CsGbbuGaNP4;uVJ?)^*;Y_!m}rj zSuMaGu&)q1K(YNsiJ-+YkAIFH*z^5oAO5gzW9~vlMHQqJsSYvLqZC8+I`zdlQD-b4 zRI^s&Mpl}rH&v=w<>`}Wu$R#jCTv<6+@^K&om)1@p)GC`KT)v){z!{L7P3rDNfI=FPLGXW+|+R9n1o!RW^iVP7Z|+#k9CybuAAy-m3h6jo381?dsM5&aGD#dFcV zyML4C66{+N3xYe*Mv;kZcuH2keMq1(RKKeGY=Gu1`H7G4UWL5m{ijQ*VR)YX;a_=^ z7pW>pr+!h@$xf4%tE8d7q|34{cWLhD*$}BR;gP#pq+tCgD(UR!?w{OcDcL(KPY_mp z{+PczrMqR>_gTe5DR(K^-{oIXoV;}XI|>Mi<@5UbM=9{Te)>mi@;4PORwS~_5X%8p zf2=l%wrr1!!!blOk7}ifM%2!M03K#pn;VETia{_D>QNpdTynO6gn4V-y5-R;SGsrW z{Poz;`}XV+jRAXnC}N#XysIBRdcch4d;@^b8)tBPrp}R}wP(0xLQD%}(_c2wNF{MI zu)Vz^(BB(_@Q3Zf@5{Lt-kVj<84x7?8 zkP9c&6fa(U)0S;=EXffzMMwV(uDs-E3$LS`uC47{^94BivGhd zufP7sDe!Lx*nhl~|3(>_^|HCosaLzV??_8Xc>Ew@!B6uWwC|WBFt-Xm#}F5(1M>zk zUsKyWs0cA46O0TDnQPE)PmYre}k!@ESpr}H;IgW zrgp8m{re7ha^#5Zo-@WydHwJt8%sEJ4u$>02&-gcFKg zx#Pr%6AvCd?AonIqlQhw@7!_NoN0-v_PB&%#Y#{$HsIXKSW!nK!V^wQRIY9*GV5

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

Os4s8|p4GB-ga9&|8?W|O3heE-|nSroJE*u8&cQ>GT z-oFyq|Gkb!ZrtIb72U!#?H7(Q4XfPUxMuOQhmLu@W zEcTF!)(%JH^ts0V83#``&h!cUTEN}IgSdlY^CA!g0VL+lh%`1#-BgzJcCA{B`(|>hRxMb^Q@9Zgd^Cy0 zM4DW1a4`ygV{R@AGj{RtPF$;)abx!a5@YmYg`a8Lw(a!UbE?*?rP%a9LExHX!+KiU z2XN;CkwG#g$`_AVVXNgPGzm;a-rPP0qYKd#fq0|J%=BhPx|K!dYz_<0DCK2J%c?bh zd;Cc6>dO7e)ndW*l;R?592v}IFtdTlI|C*Piw>?3;tUx0@iIO(cPi(xgfMrd$6X-4<+(NK?for4`KUE`wT}RhSV3=0E~fW=2|CNN}*bn=A30__(^c z(hYI4PKb|#uy7$Hw_fiC2`+>E%(xe^tZ0)fj}Oc#kcmEqK?B;P3&`~{@yV0NdA+=t z;sb*Ko=iN;!8N_Xl#!N_oE#S$|Ki#6iQ~t1?AfbHtF|x$PPAf{%>Z?(*Tuhx(dmpy ziHVfYWHG0vW|%Te=yWbnysoYmOM395V8Gp|!_8iUA12U@WC6`sOwyWRdi3a#&S3QN z^u~Of+u~YdV-gF6pUBVyMnC!x)`g;wQPqWIKt-?VGn#%wX;hy>t*wEr}+JgUlUL(sEwh$f{6)nFCam5 zdKbzfVYDcy-4+@W(y~P}&Wm)RAuuyL= zPmt8z(=$`6XGB74a3gGrh~CAduTF0uKni(2jdzIuy*X1RFJ7}!L;`h} zppx7{rpjebL4cm7`VZbHt!!B+&XZ*%#8A~@GL^A=sober z#g0`f{V;O|%H5^F6AerGec0vQ78S}K+_MuWN0dJB6sZ9}di~U~#{NYHbO~rvvDBWO z8x3gBiY^E|LBGr$H!`3~nT|EzY3S!WcGzHRa=d7Wd|;u4v|zTAxF_t|4({1Lq+hRA zWlHScu!hj8U_9DBsAqJTe%-OYUDa}XwyY(@9q~z)|1`f#ht>f#E8n|y5o$#uiz@>T zpba8w#e%t%d<)k3dbqHg3PvzOR7%FxNN_UE~+%a{DHbK6GmlzIH{4lK6llnP)e4XA*& zf@1>G0_eOGcr*dl5}*bt&Qrq72z{NBm#kPW@ZMbyue^)}P}pTonrxjj!@`mol{OJy7{9!cUI4uvw7dXyR0SFtl9L5<3S=F-54E>RRwGogWB{7lqkw3e&oZ3d^~2_q-#O<4T{|{yma_niLX?AhOzy#=P^ky^f*Loh?_a4b zbOaGds3WiiddEBqpEO~N*24u+#qFs0ojZ3v{GcE9)vj8Fp9OHyKSgD9+`Dyi_3Rm2 z4(z#m=T_rpO(uUg4tqg(6;Mg}(&6^mwnz8CySGCbHwIoz1)&OMWM&Q-^;LeqQW+XU z=_=J!d429bd;ozkf6>yeeclIFX)*#*Fp5`c^*=6I*11^W4WNvjI))Janrx(yie_~M0>q1m)4IGybxebU$$DSKspGYgDTqc;>7sJ^j7zz04Y8haMjg z==AEvK#)$a7w;4t^*P(IS8vV+bnStfclq2U114>Z@D+511ku37O)~)AX<(=pkF|;#Hgo_iN@vt*6P#A;gTs-D1TpZB54`7Gq5&A!K5z=N>^SH5N zKORbXDqv3u+k;i$f>owTWig^L+4 zlSJXB;-cXbJH{2_Nbluct$zKY6)VKVJV#k?+p1+Qqv6=W1IR6*;CY%NY$-kpHMJ3n zK(cW1AkxlCh!rQqOIM@9S<16uF+xP}pNz9a1}T7WF$Pkek$lTl@;Gv~oEG?JTHvh% zZXpN+WH1L{kWCnFm|FwUNK}alwd>SXV(-xlL2a?ECTNl9ScHrM(Mr; zR067=Mn;w?T>&q8htB9*660=DT=X-cBb3C>XAt!?;?#+gtkfPA64Ikb4?#_`m7WiX z+V0%EQKvxz_!kSNC!h_ArCPIU-I@Mx5CKMOBdzP!N|3u1%b-6LPg;w)w+6OR_; zhx%D$RQ}2pb#87{#LdGikH3F>%nJZrR6B0L?bT``qWpI1g}W4KM~@7hzlk zbf_%og9TqycvOYTl}&23Nuw=<>9k6f@FJF$6bT_HD0=6!r%z9uJ~L^`Dfo!B_=ZP^4&a@=ADtmp}8X%8h7hiUU1w#+P6unP*yPgn9S*`}d^UJM`+ofk74ajC9{m(!46s7c zOhr?K=TuNwf|xQXiQ*@EG$bZr{Irh5#CZRbWo01mu)DdkzOdC|!e#?g2D&0HCAc7F z-n@A&Wa5~_h^R^pYT3-#r-5@a6Ge(lTgKx@54zXvD~4#oz7pzBefH$Zq=@h~9onN) zCMCvsl49O}fA+dC6e-vSCZrr@QDU*~(tH#NY28G-K0&iZs z_9T3Ka(HCL+O;uYXWCAA8i)!FiZ?4c6QxUH63CRDhy@`S99Bc7Iw2}z&bQxg+qYk> z7i$bFQB$xRVV;4Qi2P-gforJ(tO#5Qq^r$LOo}N}%3qJ&WTuL+P^4v+mCCV@1&EA6 z5%Z3BuU(lQ_B1Ix;+=+dGR!6h7HIAz{7diNvrm>r5(!6JEAN0JpLlxYpftFG)bbel ziAPd4q>C7HZZcV|$Tr%ReL4Q>lpI&~MStgw5?F`2abB16oznt;*8;!4YJZpNzv!Hx zrGO(a0@s*nN`5)~lR>!Ejh`?9!vdIN=|jpU>Mx+doErG6#3mI80S$y~1$BjrWjOBY z;hvh2ijv3B+mw+`XLQTufuR65;)C%VZh!&rb~qatc<#=v!UYSB95F&h-yn<3lG1TC zAt50Y0Cz!DO;H9N=B-;c1q>MI>X{epC8S?^x``eRNO4U@hRMS-mxvOk&9K08%VkP8 z%TQgM{WLhlc;)s+nM5*4^d&KT(6h4pi4!Nz1qJyQDKO^CFK~%OgG9X*S8-Wl2Gvyz zrKR4Ax=YRi>sIT;ao?7~YvxJl!9$0!qI-D%9_t&RJqyCpi0J8DBv%RhOyLnc-kx6R zX{idz*)e6IcHZ9Jk|9ve&4f|O4Cv6|#HrJPK@SQQF7V}-U&;tTz!B;Qi$*RL0s{qC z%&2J|E|b=l?b~tVN|h@|5Wd5Q564753%+;%&h2{?L@el?$&`jw=qV^5h7ndTFVD2J z6vehgmHLvvFDlR6&hqf^1><=gItB!sJ$>f0Iuzw=qe@hbRWvqAUC?J(RCE=5g-q*W${Iqdgkf(88q*F^;sH$R=$1>@@sS@0}h zlV;80ABUtTB||qrXpp;B=UTkDKa2;E<%Fka;UU+4J=mjff3wO+z`i0yN<6!J-(<2t zO_(z>VjhK*Dutg5ECT$1ra-}>(YJ$asb+)2k&&Df7aCm5zbJ7q)Gl}@!k`0u=)w9- ztpfHLa5G(@=#Y;%UmDOa_YVs8-lHb?w*l#BZshX4+ zfA8+?hYy0lLxC-33NXV1f}|lBY=Gv_9lAnAiiJLWq_i0V36)wuL-?KzZ^NDu(qU zPncN%s>}1?(VAE!$jhTy>y`;2A+hmsFhb!`;U)cxAw!s}ASY+Rq!yxn3-a=j8Yw?T z2Uk8qjF6Q7iueb|88r}BU8#vE)N)qYn4o;U?~#BbIaVdFnkVP)KWKrsjw-=R5OZCY z7&ND)eLA4OIW2YCj9JmKi8fu5cbNIE{sP zr>eyGglEsA-1GRDY}VsPk97@b*S=kcLCZQ_xpbjo)r!lOENt7OkE^?93aEh2O>e|+ zBU6>0k`^BI?9s#El`B@)Y0)}wZm$fyH=gXVt+Ra-xuV3%?;Q-1; z8DRDyC|Jl?f_z&nDM>FA;$vN0bIqPJ2Pg!Z;j;U5dC;L_rygCquslpm^b0(U0H=hc z#F&^EpWJx}_i^CZiB26lwQmerFqwh)Nl8he zVG$1>-e0>t~?5PjE}x3@4}cP zWMzDl-I;h;pm6r~`-xYFc(g*20Q$^s@*app5p{rLa-j$S`Oa3r`Q-$6<;1I8a{lJD zz+be$TL;`ivdTbQ9T^t>@Z1GB<{nKO;+DhEdfZQo-|N(=Nv&!soA^sl88Hy0a^B|M zO&d1oQiXtU%x^iyeg26`Yy5uW7K}Q69679B^|O_#SN(9zn4!IT0Kyd-Hyk=;W->N! z4F6F>1s9l?)1EtceBgx4?n~m1F8fcKT+7UnUyM++p%Sn`t#3O(>*=I zymj;TZ>G%@#a>p0v>9^cnzL-x*Fy#_n=!|$P{BD{Hkl1>w=Z9t_t~hbRjZaPQG&op zUkx6zXyz(;MdKRiPY?%Jwz zGuSmI8=5&7mPt?%%{QM9yZhS_m05fJ)P>y}H+CC2{PV9z32JNHu+uYX^QNhCnYBbu zwRWlLuP_*fF~h%6>Br35ut~3WoilR8J5?)HsaL)Kn9+lJ_Tqgj*SP7Z@1~^a+?=ct z3^!M?ZqV67d(R%(uPx}?wd)7{J{UmC{kwJ%s)ca^LAe;KpdA}+j&9W}3;5|Y-+nfd zr!#Nk)&|X*o5j!7VRqP)U2-L9jaDjSvvjLj5wJ%w{_3+YRl4!>)~t0gxXvBbYW2MxsKCzC*1cC|rYhS$!nwM20i+oA2H!(Aji1#q$=;MsW z@whX%x$6nd0qI~inXM+zTpsCqHwXtYv=e$<2FFa>fsDeh2yY=>8U>f>EwPB!s9kJU zOx)TuYdUGllzL5Dfdk4SXQYuh8}xdp2`1XeFgfN$KA*hFdZj!zTY7rBi<<|97v6># z-0=x{^A{p4twhkXF!c&|f%wQYW~M7dF89^2^np}RrE1=?*|fQ{Yt^l17J6ANmW$%4 z2%BqjWgRiKCLNZ=NOV0`3Ze$Ak_Itj#lYEwN~~qYHM4k?NLW%$2LAlw&}2-+In$H5 zS=`p`R#S%AVsp>sX=8#B4<8gv(J!}aXyZJDe%!MMbRF}TD$p(PbQThFkC8emY!jHOs9G9Iuer>47m zc)_h_+S2Vdh6HBUTwcV66{l83iI(!<>u66Gv_(qlv}Vi=WlF<%^w6P2-_Jd8=#b6L z)y&w5g3Dw!;~^xm*!1x#Ai0?yV1xv4V~D~m2-61G8HqB^$VsdinX?7TQMhSCL=Zwo zf`XA=XcmEaX%kM$JHeNN*dk8;12<=>l!qwquWZhX~3Kq-&G-O(nb4E80SY5$6JTUp` zS+EcxsmYYdG-gem@|er5T36Czxrmz$wtqUU5#N>ujapQ#R)=K~VQ*!YQRrbt>X-@Y zQ*qw|yz~ZEQlzTc^OnG=tmL4Wlo0h#EaVh1BtQwA+3g;v%$V_II*7QWaq}d0fXFRS z-DWmOrKmn~iZzFf)YqDc9f-&k9tZWU)1ZF!n$?Kwf}g$EN>XlS2%)X159|svrIHUQ zEfd9<1&L?8(?Wwrq*0Nb;GRa4&eH^_#al=3>g7o`W+C6*9BNlq`UGgC`Q_PBGnsr~ z;+Q*OW@MJ9%!9-ZP@A5bF?rgI1@phR>J2my)U&`5l9Qq;LnR&vSZpB_xbsC~K^{s5 zku8YMOoU3$kip_yY!kVSISt}P>|F?g$kRjeQ!Q>xdDsN0IV3xZR96nliJUE`1^&7P z-b(Q(YZriQC`OKJH?Ds*=IeEvw}>hP%*lkI_>|E52mmce#RgPrR3>E*jD<7_z%`V+ zB}mW602v=nS;3)K$U-Us zMf_F>sG`9d52X|VK;bEjZo=fL#Q9)i6hz?-;4q+$PY)=E?uwrct$`(?Gm@fA!oJ#Q zG-5fC1E>!E*=0gfh! zg5|X4lxZ{YO}%iD>Oe(M7vXPcLf8ZB(@kJ10>l86immS|RlilxOQ z<{~Nai{g=C3Mr;FG$6u{>T!_rOQp$(!rSb(Z{7H5{sIvgsOfs6B+GQTxVfG@eYRHJ z`WRp03IszYjmCqZ+>!&x&!gs{IZ0_=4TXH8D%^-w6$~nig)oAO(9OhbJchvl`aa@{ z+EIRF2kkxB>z>UPMJbxqH_D6sc6Z_Uv7;d=;UK@HQ0f9@}amSpZBlY=9{1EnppF z#Xp4`8K z8Ks3dwxSvW?TQ3qSp&ins%!^VEsuKU>XiGEGIgn&=3NhQZ?+znI5I$ zqRk3-f%dMnwMJ)lE>SEup^1Thp_Kt$jPV3{p;XYVH5%fXJPQl`Wy=QQ$`J1$A;D%zhhK99b^M=*ZHeZNR0I zr(BIL9CslgEQO@z;vprok@7|<)sZSWq1?Hb^PSTIf7=3YA5{V`36o&UBwo^~GpE$v zxrntb>&n>7My(!OV)~%~C0JlJ&z(M{xL@H)<;oIYo4${-Mc8{n_qiJlD!Unr5Nr#z za06OJQ3WiS$`ia%R5SQx)&tP0%_b|9I+%e;8&iQi4pdaQSF4GTAZjReG+R(AZ@%0Y zLmq-AP!@*>#lLmS-h+oMz^GnK!bt@)LLd}>KflTqE5Jrim@w9yZV{MvI3%PY8#a7W zY%Jjn7%+mk+>6kxHj7mcMGWkjGj!OhWAzASEIPGtAz0YVoc^7kuiqCHCm<-$_PLh! zaWm@7rVJEONuyTHoHE|OM5z&F${C@p?KWTv_jlw4wlS9UbV5K;0?cpFDFH?FSy6LU zs(Ewf^yu5iub3}C0dA^hwOENVE7AK%W5%Lf4BBB;#*ZBXDw@ToNxY(@5veq!Gq#M< zb_f~{BN4=?UfR&k6BObDZVIx4!~DY}29#$;5tL={gheADDV&5Dc8DJ=8PfU?JvNJp z8v*{}dr4lX{d#UhD@lkK3PdU7uN^-ijhV$pFX7Dqvy_VW1)+c&I!8X97@s`KUbDUiqOPBDMOLEV*;rwn}CFA3qG*x@j#Qg$0Wg1s1^%>F~D9 zP3zYIltl~VbJeJqFIq%A)&f4>I-7Ozg83y%l(d^|SI-A-UAsQ~!Ht3ieB8aTV9Qvt zWWIk%-&@zN?b*I9_})#zWqRh$o!iGpFu-ovyJ_R;Uk}B+hPl2<6@pZjeb_KVg(PQ?n&_d&1;wK+rB9}GNOE$(nc4<_D$1&b8$a&w`ks}?Ql)VF`Wf(6Bx4+2E33JDJS z@$0XatX@OB-18UC-MoD1d1Tm@)ytzJ!mCuQc5?&@4bSJDiebhFbW^7x2oEy)Iz@oD|n9x(k=Axr1gK-f! zGT4_1F>He!*RNdIzk4U-NNiN(ia9gC9z7Z(^tG#2e$>C;s1YN4ymGVB*o?{J0i@KV zq>imx-MW0Sc;Ui7&YHD-^Clt6(GW9PB^~0J;Rf0$-rW+Y|VF)KIkyvibojvQ9omBo6fMtsUcjf{+(KXob+ zOh!Novmr{hvWf1j&fKi2tm2pAY)Zc>WvM7t}!1Xeb*sV|Bb7#&K zEa5xyv(e1Jp@JGP&9fq1q&m~){!p=E#R3KL&HMJ-B|8q(s#T+~Z{F#XCVuBIZ``=b z-Ceh4+creS7u3e)4!RzY?E}9BtL>7R-IWR`V7=uiFTD z(6LMRE{z(WK6$!NulLXWc8rx0hmRVKX~d)%GYOt&Pf6F>)bm#_nfl!%lduyKESr)* zH$b&BvHRrFeEJNG4EYMdAv6z!45WwX-E39r)nln08oBzZ>hNdtep=wr*pHn0?ec|F{>6Gt zn=%#B%wfr}X4;mnUOQ>(cLR_a2lg*t`2C)}`^U_f$_yKR9sdt|=K&s7weJ1NB-7FZq|*}!B{b>1cR>)5UKJ@86jVe- z5kbY?5fG#o1(n`=@1YY)2qYmP36P%Y`F`)3BqHaYd(ZPd&+(pXvK@!Xp0(FrYwcP8 z_g(+>78J7N!~|5%PMy0jd$EgP#}M5URP^ZM|SBN|3jer3``9#zKubY=a34FhXt zJ$f}XsQ4zbO(?OJL`y?pNUn1UtcqPYbTO$FVT3(wDtB#-=sM(ysa!RJZMy~y9|Pzk zA|f;-#Jo~870_wbZTXz=zi$Hf%fg%G^ts0&b+yrv~<=Al7`V0w~K*cba9k) z(eORI(vp*0#4)l)bbd@W)WH*M)U-L};jwGftXZvDGor-cb6K@|R7z43jP=E{ zXV$G)9$mRgohp&L*KIg*q(C7|bxP5=?*wVZv{B zG;7&P=x>8p^;*#_TepSM2417<)=f!DgcLilcVFAKZ6B*%^@jy>9cfsPu$QZwF=NJ( z@4gKS4eHmgA9b^55)>MG>((tBySm~8AU{00Bkd?XN=M-TcLYkD#*LoF?OVTf`#sxtAKS6_tFM3P*6s0KTX&w>dwBMzU!hq~ ze(B}r&6>`+bkUiW+P-N^8_RWeC1G$C&mav$y>xdMO;(r@JbFX{gpI&AMNGw&*&!{L zv*3&vpUHm)g8>==fIy)m8=Ep6*H(|Jm3-^w@}*0O+KL4X*gUUHvqG2 zn@W4w@@3ahA|F0yI_qj&x4bu!Fh<-#MRzZiZn>ii~X2uA7#vb!UvWrvbY z5i$!KFt}v2xsZ_{_`w@vo}K>rr`?|HK`fVP<6eSxC(r2d!-qV}gT%n>tj0@=_f(DxtDsXAsLKC}`ZuL&br1H|EbOiqU5h!uL60ikSkwff!naWrQ zIEc5`yl3x~KmU@KZ^q?sCa#{kAer3xf+`O5YnTeSMiZPOd*^MPbU{J`lt5-=fi>0vW zJFv=^A!>ARz9SPEE(oU2mT|fB5Kelb7e!>(?(_x=3NJ12W83 zG*nYSxsc0OuG-CJr^VjAU;h=We?EF}U(Wrz-+uiK8h`VatpS|ikg$uFu3)bX3r`m# zszwrwY2lK^d`94iMINa#ZDt@96B;lGB(4@2b@BWKRw~d+#2#UYI0+0>CrcT+EOy+e z`C=*!ipv-x1LVym_y-ZT9a!2vK0sXuI}#>g=iG&8k;Rxu5av zoOxMUnZtW@?cS*VrcE2rjO+#O#1etaKzSr~pPNC45j6qc@G#-PYovYW&VW?A9zF8i z-5a%Pjb7OK@h3>=jmA6irB{CV{Ofknb@H)w<03bFBQeO2vDv(H1pp-$_m*eg;Zmfrrsd4Im4x0lQH`NyPv&jxg{rX|zUe zS}VHFlg~UiW&GHd)gz|8`Pzf~>AA-H*RHYSNUZ?K`|O zWKh@obyokf=F6o&2LuKKUfp{4IeqF>o0?GuqptIS!6W+(?AEjyumx7~0+Hdk5ms~@ zQ^3<%E! z3riIQRcmlsHRn#8-oJAvmLZ56rF}yvc~}eRsm>yJv5)d@OL1(~SfRV%R4dFheNf zflk5Pzz*S(%8T1!e1d(Vc;#q3Do0jtKWNaX{sX!P7#Ub6)*(W0E5k>@Zf-31EQ+VwJEZ1;gv1*W(!t%m^hd69_kID|AF!TAIIq8F#&j=p|xeO$zqh ztbB`wh^**eMvsE@v{a+Fx87i4b_3C9c~}8sL*ml%7$m#~xk@+GHK#Ndi8t=%X0hV&kp~LdZANW@1y^pH#n(RoIf0^z?XioQ zK<_oe_+zu3pY7-8mqY3Y>d4N@&B^j97tCHFo;WC(P|E~t$94=$)^YNn`WmDoj>{P^V;?55|f?SD3~YW ziY3Gin++@x6c;3N5yw&BoG(NOaYv^)H^-A76OR{nB%xdoAtG=1J$P!R%iA~q{`ps5 zZP>a4){S20=I49z?9{T1T9ho~q5j9~Px7;+wd%cag-{x+#5EfKJMSfqnhh)|9yq ziGfPYvK_MscwRTWOPG*MctDkU!DL*rQHX&JfJGo!>n3POh$vhrg*%o=7a)oi*K2%s z^w2M&eE9~IgLB2`P^cKRQ?4cNx|azLcOgWC6~-Jw@B*GiPz%T)G=@wszC{n#6XysP zfptA?)_Mha5mbcppdQFA_$PrvfLPQ?%nywwJu#$|o20wWw~Vh(fH+t(8d?Y$je%C# zwcdWjXqBpUUSzjnl_5Zhw4$}}P7H-;j_Q|;tMbhs0>wwtcnQ zb+|q4!K9DTB+Qr?D$4=CF5C-Ssq%-45)Q$_?U1hqFJC^P-BVtt*Fj-P0VJqimrL|8 zhi&=?vp)UeOAS#3cwmH0@$uo8BT`?9TzLj^6pootOoia6PM5xwj=(<|0j7;hnEsRR z|Bs)hZ;xo)qUH!v0D-cmhD?o(yF%WUu)t8JxA;aYPFbc9dK04~8+{yv(G$}K(dm>6 zAX+nM%cN%fMC}%|6pjQRKJI{-F&`|s`RtJy{5`2AE$JRAZMAw(3O(Qu zfOdFbVcykbX6Y|`fmXJ~#N+}zf&{?RnpL}KXlxfU$;EJ7i5{>YzQ%erF0OoqN?tzR z^aZiPjc5t&WK95+!3)D@D60%U({=~yF;`}0hh6m0dB;GhRd<1 zMrNfXC+FwqSFI8$mjLWyY9eYk?;sMrgSSAlror@vz==fowxgKS0s@C9tz62`U#r8M zms7^y&xRkSf<-`C$Q03=K}ivGAgHVia1EC`D1Kako={XCL?)pg5Kr%c<38k&5@5&% z^?Ze@aY|HM1SX@$pAzLzgqA!@d32H)Lm{;2(`@&mG4q`eI#NKym3#)oWh7<9kpW70 z;)cl?wiVdEEf9kl9zEqL#7Yj;TVe#V0)tMG<$#-8UIyzkdI8Q$ya6KM(Ch*?@voP@ zikKCyNItW$`cDOJ)l_M#9ED;Aimo&|pld|_`giq1{_Ai1zjsHOCo<>7S0Xk3YL6E6 zp6%7~sRlJ>zwwIRmWR&F<`YxV29xPtVE0hzRb(IeV69M!#bLF#R{0y7#XpeGSXh?hqk z4koX`4O#;XV_q<~<$i>Rm2_or+tj8?;Mm?> zyH%^S^OqGSv_N4+JN6@xw}OWv9*8U@T|jjPw-^n) z#z90D`6epJzW?gzf&ClT9NwzI$R2I&_tK3p#EPjQ6Pnm|Ks>k?=p~PugiK*#n+dIC zARadntGFH^BM}n;W&TXLdA(=j+WE(jpFB8YDPQwGR9Wc_-w#1X^-9szp?JqW^$TKKp{^i7u=V(1m15 zya+b&`lpBMA*%O(TINq*6k_F{&i#*n{A2w*5oxTfGhBxYKwT{@KB>w)TDwxKPGVYSoQy z-gNJdEz!+dpkfN+tZ|$?wEup3im$eU%EaLC5r*Bw7{v`xDX9Xoe%2v^~D zWZ$m+d-s92kN4?Yqi$VASxQJIq6yCaaMtJFd`-%j4I4J}@891LP!CZObydUOv?bvbqmaRGYd6Xo(?ur%5Sj#^(a8OXCNU~ODBq#6QzUAE6)77d) z3>rJxsMlHuu!82z`arfG?MIFt(B$Ow=|8}#(>*b0&}Y-%#6=UgLr7cHTch56|BlV{ z*J$wEyMG^k*nyLbKa(KmxqE2$d zBBXX97t$5f|1W=_ARTR^0C46)J;1Q=s-g?n1t|K+zw*#se~J=bd=z6S9~6sST%kx2 zE^^V(@A1cB}`t#65s@jK9yLbeOuaYyamK2>TEU(%l zEi64s-UvMMtNQy7fDjPikkgrTCjnhCGBT1KK|B?sqN465Cy8f`Iw*7U;QpB}j7P8Z z@$vqA%Im`hKD}h&!ZHE=Q^$;6^UKNtyKT+tRg=a(7akZ0-o(enVaC83A0-j(6~xy( zNUQT$Lw2dI#m3@{&Ke&>;L24alM@pqzBFrT_OQ5{&A#~4Pe%}{=goOKPsTM1SH?&9=AJE(k3B60XeLEtm2KJBmvs9>3^@*&j@O^%WnJ*Yd@S#*7#?`Q;bb;#@d)-Urj)B}q*}+_mAIJD4*bgogy4 zI(Zb7_9+t(SS$MCrSq`pj06b9fVf`Vkv|I`hU>FJs6XrA45@uvnAo|kj3Hat}DAJ51dju-cGy&f)H`nhxj z9vOiWr*X3hB_xNvfUQDwMQ?9^Y-*WXo?brj@$qoBY@?wJLkCM35HHibm8%RIXG+qo z0~8XiF4)5>#^gzL->24NMoW{U+Xfz+q{w%6?-GJcG z&)#}t-s)c(G;1DGzT!vkP4oEp!<*M)8TcVX28C9tKQ19I`8yT1CNGxAO=L7L85RNm8Jp`U~GtzhNV`^CF_Evn|-LNfQvOa^ z{r!Chj2rvjq*q6d9C`ZmX`CZzE~dxIua6=aePG?@CFF_11R9okKQpjw5D2gQD4E#= z&DK5Eu(7*`kt|OTv28Xwjl)o7S(rF@+_93xC4G!V(j2BU%Ka=?!(LFI)&-gi(bP@nMBp2CxX@ z3cr5nA6LoZ@~Y$?Jj_D{|MbySo?HKS2j9A0st`37BfIM=+Rx!?TX6&Upk4}U)eVZy zxGwrv--|2!S1%|n_ct8@*U#*4+PVL#TNr795g=o5a46t%<7zD1aVRe_=VPi?uO`OG zlJ-y{0g=#3Z}1i%!Nx*@!Rm3N#&#G$6w3ykDZFCUwOh8o`Qb;4zW;vy7oS1&3Un$X zv(UiMgq!gwx;XKai;TP;dsT}opUraP+BI;Wc8S7>^#~ZOu-?OHAea#^kV`NAfnvcR z${BQu)BW`+)7I@j9N_8sY_IOwSr0G-=8(c_qZLmg(jX+CJ&$X?j<6c$-1|3gViZV> zBy3UudCDRjB&G-Mh{_}xr*zK1-eAiiIgvH)U3&N36mzLl-vKiwys&@k4oohD1M$G1 zf-A)c1kJr%WK`UhtN8V>YkDR2LiOlcR`x|PyztQD7lVe(NHI)|p>&EaVihNvvup++ z{Io{Xl2xm}T>10GOR>+59O3DvF%=Y~-$@P)3Z=Vr9Iz8p4{0j^C$%s_e3F*}K0))M z7oUs&EPl@FB&OQ(qNEg1m#e`p`m?YG3b*NGhRZXA%%XC+h1krvUWI3*Yy86GCG{6IgKvuu*L~%Ts-Hhy2S4OTWJd8N zr62zGBT({yThv2*`kcW*!L?hpo-=R0-JE}D-`=>>XL|H`62m_rRg_?2Yd{x>H--Qo zRwhd85n{YGkY|QCJKm7#gqw0dEEDl<=Y@_k^Uc7h_F~k?no9kVva=pe)Sv}BH99|@r z6ZgP1Um3>1NB|aMc@6EZjM=DX(}vC5?78R08OjC4#l_vbd-s!Bv!K`E1}I3gusQ70 z@98B!Eh32`vGG=|T7|(3R(e?aF+Aw(vf5xJKw2F9!s#9eC437RYzWoFs>Or7%2TJ# zv~1pDK>w#x&R=wpn9*vz91~NcMhzG5;esJRP~jj@?z*#pql~HoEY}Cu=R&wtbJX=a zc3tawckN8NO1i)fSS&go`lwF1it+K``ilw|{_FaRA~JFUntxPMe7UQr`nLG$!YNh^ zxW!)-o+*NR&Qp)8Qoj0gd8y=)yA;<``k@4lK#AkHh47WI*Z5r6-=F=-sA0oeRgcv8 zc)$70_rVpaSj7twQ(Pf$*;iD7I004<*^$#j!b%b>K%6NA^c_C1>$mUb0BhwRYw-Su zAKBQdgJ2eoTpJh?t_cbxxJS!Yt%eSN=5)+?f@4s!@0c;Y`t%_sfoRO&Gf|>o)dgu3 zh7vjz=RFnK0|z{$2QHM^+1YQs`MM@OUE|}`|HYRZ)vNpN^cfyTFJKba8XO6seo)ds z|6$IgapUHF^r3rj@S;VFB_4=43W;C1hy*c(mc}6t2-9a4S8vg4F*`vG#Heuj{J9^D zAEEK}Y|yk#|Dl8T?LBbh;GQYdCgTuDc1CQaVep5HdiFxh*-lLwW5D>tn6bV346tj+ zOn_^W$QkiY!f=@sUMyJvad9Q3ew0nHNa8dh58fK{9O=n?Dpj5K&G#C20O!UUz zfBWgj9|)*lpffVD^Km7J5DCp zh>LwVnQ99h>Gx87%LEX`3HG^T$Ic&p`dOWN^>8W|wT}o@0vQ(e46zdrghR)F4?k>{ z2wIj5BtZtSh`$8*!bRgcX$A=$1hgWv4^UvU>Tv#XI&!n`lSp0T<;OigeDA$-L4l)R zc%FK2Y{aS^e4$%9hczqn0j)6k_}R6%0t&DKq_QxVwC<$PyukM6D*Y7>g7o$Yp$iX~qJEO8lQNv8QH9p2@qb z|5QTDrIXx3gpKP*B<=e59xjkj1HjGIy-H)1{CkJ|zpq_>PbKOg*R5m`!1oPJM@PVG zjaOip&BFwvfL97ZJjj3}aKQ978qGxeL$j5X-uS^81-slli04O4n7ulW! zeBDh1$VRh5HAPh<;5Ge|LjUm}Qhoo%n@w z3qs`u1ebTJc!4t4QRiT>fnHH(b>eHSAPf~7iXI?f@{D_ms5c% z!)6B0OgXfxLE)jG^dcJ&=K{9kr7w0hT*4#fz&Qzl!bg!4@=16q@Ky>*^Tl%=$fmil zUNQnWYZdx2fz?u>06G`)L}dKYxaH@?r6MO-5uk%hxN_;QUJW~XhEQs>Tl{vB4733c z)Iw96zqQgOy)Hfe*$9+4jT>zUJ%wq6DrU7x#&N5hYbRpSEK&Nv9S#@sKz)Myq9BUe zXyXAOLl_{71IrjXHFx0;gyn18q?v4p3hV+~(if-Il9`d=&mJ<^C!~u6GC_KPlwc>l zqQM}e(^h~%eYdVX{LA>!5lA5pKwSn+G++6X`ywM13lFqQJNcF_b6~?YG~yYu~xFM~gu#AhVa=~!Yf_0QjPns81Vn;9)Hba|DRg%7kAN#2$d=nD(9Gw8mqX2OBg1hfcZ-F zOd&1f68)P$X4p4-7?!@AJD-i`ja~u8xRbjwvY%yw?e{Fjtj-Y z4@K6^K)(cEP;JRUlb)2gddU(ab(!s~ErbmK+8y@e`*v!r`3i6xW}>5jYU<#&sKDSO zoW_Ym2b{JXpqLi3#{g=}f&<$eJ+_4&n}7X1{^CW6FeQ4Z8NYdMX}5r;EVG#&%5vFhWEah0 zqX=_RW<5xO`bPYq0t@smsxaZa^*|qL5ys#$jg{sNE|PbVwK>BnK{+(Drcc|v?pM4k zQCIm2_yofsl|gXW1pvoDjz(p~P4IpVQDYqz60W#AtsVt7BnRyQGod*3cmol}T7x(# zxOwIB%Yz4zX^h$+2Sf&-hyj&A02(6i)DOS`io^*KCPsis`spB>BKTqVkf10ul4dy2 z+qr~ISmXr&0(?oAVJTRgTWva>yOl%|AeDotDNY(eiZdd&P(7T30!&egziy*^`D0u;=AUH?b@v-fh0c(;ZVt=m&vX8AegEX*mcb zt);06H{;f=S~dBVNfI%JVHZKk?ynFZ0+V(N@4^xn*nzpvCQ6Otx+e z{b}yE>(>7K;9gqYS~Y>rpI5GO%gNlkZPUhIR+5V#EU4_Vg+Fds`*To$Ur0!h0z3PS ztCyDiIA{H@KPTUci>@AFFnX_8vhcw6P3b9jQc~~Nu36jDv4yL9oVc|RoHj*F~R(bvleSd-%6=Vi;5E?N+G z^=i!;kz{kHRoFo;o%7Y!b?ZUiy}S1`Yt=lmMifj3x4`%^-)hmL5jdPD4(z{p=2X(H zn~Q&%mzt6sU8CCWjq6t~USzZ8MO3MXm_&xf96S8msuhP0?j!I(NKoLZ6Gzwoys~k< zI&7-M#>TE#zVflUb*ojW@ZLMqpBXcb1<_|8e^4&4Twq9u(zwMA6G0(M2x|Mwj>3D% zDgmtkq02%fH_vJ{o1?=0$jT~<7FpVr9{=_cC~+FMD4{I8m4f&4@+A|-jNZKAHzGru zb2Gbl>2&hMkt!9!zWC&WrHdDWn9Tp@Mhv+bdm$$+W#Z65&kh-U>FjA+ZuY1rdM4e9 zM|C}Q{K(Kw9nPOQSvfRt{+w?KDgq5HewXCFL5C8PGv}!j$4B(&>c}^fkLJaJ1MtBC zB@rGZbAVkHmDuTgZq#slP8I~q_<{Ye#a^;m%^$z~a$asOTuQ>NggNic@I%KI-T|f9 zW<(Wt*vCHe?9v4bA}dzhx^exqsZ-JE-hKP6@4or6Mzv}wcM|U3OH&GX7s3~3x23kwepq?>!wbbf)bh(e|zk(A@=+n{C7t6?0)x7GV6wK zzWm~yabr#H?#Z{~KYnMLD8s@f$W9unH+d3-P(oKlOD)jk<>Yi~-E#e^RaL`74({IB zr)Be$+wp*%4&p%6e^K-ACMS;_F~Xjg<7UYl-TU!7canr12K5yDD04m78`&^8b^Jur z=H$ABO*Sw3 zzmI9b+#f!iHG>REJd$Tej$HKPyr_tX9a}cN{rVJ?=#2FAzMVR3-LNh^sNAd>@8l-k z7GoP#JMXX`WNL8-KP{wOMhChdf8&OQm4H7T7GVhm&w?mUVU@Q9V~@xcI&>q zd+*%2!`6$8zwFEMt0~pE|LyqrAA18-hmv)${RaL36lmP&VWE{OEZVRMU17-=vjYvf z#S4~VL=ooWyJ7vh=SGZRe$;8%dip1yJIvXS?K|Ss_VoiZ{=bpnyjvTIBDeU{t zWfW_g?~WZUpU%8`;bf~uO@|K{P+)qN z=+$uTcDNS@M6%9+C(PXkn=Zx?f$G$;Jr{Ou*?iy_smFVCXxFoAckAb$gj6hl?A%$C zR+Du%am}h#M-Oc4)xFo^Pd=RT-pnV*jDy?9I`QCvebCGx6?W)AVOZrV1BVaax@_JH zFHH9I_WSzV?|{X%`%XEHenTct>)NRP@v!R@=ZUy1kj^t_XreB$7)eglUFRfsrz_NSad`9D1tX z(=$Jq_2%SBP$k)^Nt0fE1$SP*0AFoD!0j9H^_n*Qb;lk?iP3?Y0kK7`EsShls^OL_ z1atMa!~wUMe25jW>|?Po@6@q_Fv8k`ix)4Z-M-bZT6ML@FPR!iHHp65q**g|19f`O zS`8j++^iK$3(P~U=-Nq1cVLMx#+-kB+BD1@s3bf*+&44`UrLl*=v%Q5^0cho^9L7QxvsV7jxnAgqJ42J$}NTgU2Sm`g-?X zz1h~(y6gE~ZCbiVHfreS?@Pa~q4K+FdbDpRMh|SYxoKl!Vye}MdVki8T^l!=;BoWK z!F%nxS1zjNDG_EQUT@`#OPv*c41PefG+rwl#rOifN<5gr~I z<}!@qrcIkR#a=$DdQGsaMT-`ksZ_BF8-c`f5#W|>R{&FNMcTA$%Xtu#Nb6@#pF`Oe zyBDfsd7#Fe!m=H!)z+>P4dYWy#fPa=uK|se&$(nYvt7J$nPKV55V60>PNl{z=G=Ks zcI?>6$O(;GrScUFp}{O8u-Q|l z9Xq$t)9lR5FFtyI=Z0UsOa@JE?!B}$c>NyTy8p6r#b@u&n6r9~M21mEL`1}$l;rvf zocV#URAljj2;$(k#fj1lO4mS@wD%GR+%gLbjqLc{>wUcg*d}5di^!`<35g3w~e zFUmYRt|9?6^%^v)Q>*6a7bY^Hg8KKYzYq%hphs zeFqL6)~)mLBl|Lv6MH`12S*J&cK{vI9iUfMt5yxku;JKIy}=VL55Vy9^V1njjaqhG zxp?t=6UO^{dN%IZMRv`Y^O!wowB>?A?0EP%-KtfOfJol3f1kHc8NqZCAqIGV@WHIO zYuC4I-BBqlxNXNyKyz?NAoemTcW;M8*R!}9;Y3A|3ca&Q`JknGG1OL7u&9f>V=BJ`6dL~>DZm2G3TFZ4Rb7N1G6?ZMQC1(0}h z!kkmY@#DO?*RI6w*tsh-uv~}Eoe*h^A;R*^@e_u?;63~H*KgUD1|#xr-MSSR5{#Iq zCk(v=1QuHv`X@g*q@wDX^riGDQ6s=_xnv#u)P?for!NU%!C=X~&3ta(xl8={OGl3$ zHJMD;<8Pn6bj8g6p(st*wc))&7Ao;YWWkL_i76Jq!K+uFek)h3KD>W_PR4`pf0%>O zUyBwk(cOYV!cfpil+K*&*tz@0ZM$zJCYU@u4<9?3bT?g8at^E-@D683$HU`?@8{%X zXa4^C@3f#@hxXhts#fjrI`v+d@O1nPrp~TOkhw@$Zs1qr^Vf*g+Bg%^;@-`x8TR*q}ws4Pp$fSHOmM{CX(Gy zrx+-@YXZua0Y5IsUdgqZTeokQdMS4Gij}t9yq#OOUpjQCbI+aty}!R7vgUH^HA}vk zojQ`lbNse$LwbB1@VIz_}`+*V`8W?8hk2MCQ6sY{{l?;UPA9eM9ci zg2YloPVy4dZi&YNuBp%}z#vk_!(wA6)&2LizwY0&7ZhZ~F`Z;RAfOME&i=G42e6qG z3wqkX7$PL-S-pl0H9r2!R{lb}7#Wb%!)R*VzQd(cF?)AzPftt!{PWK#DOWok-+ujN z+>PtXXGa7Y&9H*f(nxl3#s7&)ln9qGqg6CGIr(n6;2^(%GV3>QNxgAP)+KJ5x8Ir8 zwqxfX7cKsI{(_j8b3Ck^oE+=jv?|pi7^02qH>BQ4;RnM2xXOz_QsSp%P2kersu3u0 zz|E9qQo*QVlxNda)W}UvzJ@?kLW75kFSi3tha z`t%F08ZqkeUahM~{x zbI{OXwv4=|`VSGG6s(pk&g~psW9X2luU)&$F5XY` zzwg;7x=o|{wHr0+H+UG+TXZtCG!O#}m&hk5Y#vUX&)c7Vwc@9r+DAtX8rbjZjmvPg z!$yxid;U_x$ZCt0Ee(sVtMJr){Nz_pojB2_S@Y*dKij=`U#qD5=#0vtQB@rh9MI;8 zCr0&q>aBO*>C(O1m^Y_>`0A9_QMKM2Gw%6!XLabxlX+y{RzB&ldtMFYu56qC#OC$y31n?EjZJMhO&-G zyL)%pXP*a_4M0|)Rzug5sNN#fjN}eR^=BOcHz&KgMB{QNXuJIUMm#h6?1c+WYSvt| zVpT}(x>}uX_1ZPZwr+=Zt68sJx1mE{e)(mq)q4EIDUZ;QvVrBW+xYN>7mgl2EYe2~ zQdx)yW1qPc$4cf1uuT0sOKtl0NURc+P)JCo52_2nBUskr*2jt#)rnY4POCXPGb=DC z6tgsZ@IV3dGsZ~5N-^VwcVM~Aauo|OVTfTO*lNqq%plNqnX*BsRD>E4%@MSBS~GJp z{R7J4Q$pqD%yd@D?w&r%&yk3F=uHw;!-%c1(*CouGV~q>qn8gM2~pGxZnimJeY1Jf zw)LC0!PgM|3MWIS!(zFg<>~DMeu+7jJ@5X5RR6NU9v(V@3^#4&gZo~d-Z+?}kkVx7 zlZT{~#4yli#UL3kd9j-ojnJN(asOUGU=R^HBrFOB+ICxBR*uQvgsvz`o$v}|>vg-A znidokB!9YTaoI(M7d^O_-lIXC1J|zUj6OUte(Zp$)@kND zf(P&byc|}@5DTjjjso^{wN{o`$W_YPtoQHV4-5)H25|6DC}GdZp$(qiWI!e$f%}ea zJC`k4vTW%Jt0=-mfDtX2;X{THu>gMEV9(=OnBhX$h$z57UPmBYemU6e=VfL3`T5Z* z`lc)vd0Ko!b9kBvNL#ON9bcL_u}kkK`H5I_b3Hvh@gHRKJ|@RpMD))F@g%v98vo4Xwex-h5$mM zf@_Id;pOK~J0;YhyBJzjEyL3jQ zFuW3jNf=)dX2jt#dgJJh=8wv+_w-@olnfO_uR;v4+X+j--GOf&34a1#Wy+Mx&dJ@i zb617=7-PD26lLQ98322(SRg(FtjH&*Z%;1~R7g6?$nu3CwK{j5k=${FlmXD= zZ{2R#s8OXVRl#T;1QjBR_(xJ$hy>xFgIa>g$UY`$PWSMhR3`j-qmkbX$g7?p3k1Eo zk?=qKu6|y%I{x;}H{N;+Z02emSBdaIR7o0X`IT{ntR4SCSGZ1sa!W6j zK#NQBv*%lKva|UKdwQ8XjRpu!cBd$V?;Z0~w!JvCw)FJW0RIsDxv`dryAg{uUQl2N z7z%EJHIye{9k6V%X5YARqe7(!Pm?zYGJo#e#b3=H|I)-!&%I!EXdc{8L%TG1`ItPt zgpZdQi}jrBgIOImKvO6XHJc!aZ^z&A^70KN6DEe=5?~2;PH7K1a$bI3{LSlCDwHPy z8|T3b+9gZk?LkD28h`l~ z4@Ll%Ty?^$2%iEWgC#XGSbWA21@P1=n3vp@-qLefX35hNXp}C=lM*x)vBCf$Tftw6 zPeN_dAWj0}3Z)q>ZcW9Ew!}Z+r@)Sgf?6IBSuQX(C9W9-utrr4mKz~JNz||{aJ-ly6+iF!u_wCiWdZle^S2G*OjGJ)q(v6|x zUO>@W|J$lw4QutNSFK~U3gZU!iM@DQk2QzbqRZ|m`ZAihB5MoM6K;2J({gB!E`1wD z&wBecjZF|%`a>WNH_eg#`#VNfdZu^R4mGN5`eilqA0-rQ66kYSx2#|Lc>P-4>ecKP zQGV>ezOfh1&_b%E2oHrJ;Sa$Vr!_U{R;Si2hCbfy@yF`CKV_oRE^){O??e@}-%X6~ z)S}4{hO}Yb*;8L7G%k8MBn<=&@@d`gYkJhH+Ot8m_SMUe9s2au^Ct}j_7C5A^Se(z z79V}le_byS9(x-tkpTcq7i!ZCP?RwP4H-L8InmPv8yPTxV8KW1On^#tmM^dcLPm04 zlyurJB!hsaNGxL5u@uG<8KL79{!yGaQ!dj8(bQ?fsHKR0O zD(3=)uq>@ybxP%YR6DR_c;-AJNJ&{GW?W;-&A6|~BxxkQpyk5f>GV>+>W+-R$Wa%G zMT=x8qBK-s z4AT}jpRiEw^6S>EmYmG5KK&R|YkWJ=bcxH#)DTP|Qc{72PSvYZ`}DbUE7q@Hw`|$z zlgE^mjle&UIODx{>(#GwIwt1l^}qkV=%=%%j&paKDr~Hq1VSkr794Zw(%Q`%8MH4y z{utf(^~sZ0FIjXuE)GE?I;{AV!lg@46WK;khscGZ&xNP{yMN?YEpV#aFzezhf{^FR?R%>r$1cJ}e@TSh+f ztcTIFTJ7kl7A<#d--W#}svfJ}!v_y)EY>Fn49xQ|^z8RE>)#WHj-UbwehIfhB_ywL zczF9p)v4d4apMPflTq6oHuB8K{*fjh9dGo)HhmB__nw0v^HFuWD4S$C@=u zPDx`WZDGGoO=Tizpg;n7x%bkKZ`m~B+0n$ju2#ElRFmdAcI;5VPJT(ZWBYe$^0Imj z7?kH>>OEjEs^QTihai;^L>P*=vxDix@Z9O`1l zs!m$>Y&UnKuWz}Ci1fRuvNx%G0Ksu|)%=|N1BVV?xNzaOHEXB7{T7o}86(K#WRElE z%&Atb+W@h}np!8U&p{V~6);^j4Ke|v4l`#>Z_2{pqvBWYNhqc5kXrGW5}Q8h4zfkEdkDJd!3 z0USc%y_=9wp%&fo#%`UxLEoy8_tR3bG1nVBY-Y)S23(qRa}ORoc;fu!Rllrx`<=Jg zgoC9;od@0(r7nB3X?IfSOOCDPCA3S{g+=Vm0t@&-rJMSXrSX(q}(elH0-;uy3YIm333xf}C zsm%iLk;8?Od=Q{4IyecCxe64Wd-Pw0orNchul;YXa@8iki9ZhNkP}?)LRb0hf@?wA zVhDD9;N5j~@p+27(B1V_;dwFBR3-l?S6uraw=Vpou)FZ{pZfdHFZqW$N>BWKMxew2 zH!OzCcBdvU8}QM2dy{pHY33xRPkv6eWDdg>#6yQSwzoGa-sLMeaW7w=tlS(>Pz8q) zlQNV@_AaNtJvHOzO~0UEFHawIV2Xpq7>&RM$6mQGwCQl*Ej{A_EGV0Ga@J`x`FP_F zhZ-j;vzz8YMuy4DQ;ZTtx7GOiduL^5a8oOE2t*1yXlVCY*)!gqem5@OGbqRl;b13$ z4DgKdr6bb{4T00RcP|ZG(R+FV&1|+Aygl>o0W~L`z*Qm5u&S~ATLgNg8 zj`0Ix0F;E@(PU@6{nnc)@d;j`(#+Q2@rEk+v%og#%-%{dpuxnS|X#_M84mM6zG(tTCF z2y^A>7lECaWHuqYcC+}KsS>rSklfNEEHk@pW>tJ>qb+V%0__rj_;ReUe5f8McI(3D`hG-;BEvT0R_g)#O8j5RbgPJTz0V4>M=dIa-JVjzyn_G{v4)b3WQN z;sJ&{CpW81*|O|^0;FIrNpFEMpRfw>9;dEdeg4H4-W>nj{gh;($eFV04fF^8ynMXT z3Nx~DU@f@RnxE(E>yIxWnj%$udV5*0gun@#w%M&&8TZSUEh~s5`-*Ir3D@Bj9=2=G z&QsUo#!eVN<=HX!(o+iv#;t?Y!75UeO$?>I5M~;pYve*|P_f;TS0*r!%{0~@bjR1n zhi@^2XaB%S4AZ>a&`=2^LSwZgU=!PFcT-TgJ^OZ_xElN7#FwUy9g})DneRfv!)RqX zivWql!rr3Y)aT!X6~3b?K< zK2Hs@5&!=i7covv$(D=)gN zunYW2fq(2saSf#({@x={{7d?Kck_?lfpYSPvmiJ)7{I%B{i+owb~op_b1~JcL_qtZ z>S4Pa5*C_t?mV0cWy!PtAntnA%9ZecF$^JdH}hv0ke& z+U&7s&Q_0%f~Lko21C8D@Cxzg&xyq}8Enkv%($CXD^`MFAqoeuis=c$OI-QcT{U=h z>Dq&0m(N`w-X4g~Rx8o8WnJ&)3<}3+(iwj%4gr8$(dC$!@)6ZA(tsV35{Y#uC5ZvV6pTTqFP9jT9&$XSlgjPEII$rwgm3v?B<~-G`cuKKs*Re} zEtEmK;G@_DQnNBmDQpr2kX@WCf?bj3#h(Rvm4+_E_7K9l8YAstO(DSlrx)kBKlfr0 zLla!xEbiYQ%~hYI+DA^6{#jy1pu_<;+Mg?|L`ZOO-BzvU&Yg>9z4yTWYbQ?j?b9be zC->JiYt2?mt2S*k-ku8=%tsYnxNxCYg|Nm=8Ui|OKf+%#r_h!`z#H+mjvYTWYu5Yh zy49;w7lUXqX(K+hRudT+iD@=Udb!}>TCLlD_uY4Pi~aDveK(FDd!kows@=M2^WBuY zZCbU&6nxQwg-*L;(U0@JD}*|%L~l^9Zd%IS z*&lv{^#wCsSeAkUH0U~Q+kgAj*XEquBm4JUKXIaW?>^XjZ{D;q_-jC9q zdGqG^RjAnLv4+fiRv5xz!#)={Nd9{>;r5YZ$3OVsBYywUwQ8iNrF}T-{ha(fNz|?o z25gOP*Y4YIzqXn45A4}{?ZolNdp&`qBW@IQc5rBz8qy!~a@f9%xhSs%_u$kne`2ma*x&A75vD`U-#>!nOR$*oB11g2ew3&PZ&>K6;3 zORdG~t&}P*?7D>aKOv?%`|z3KVvl?f)Gj*Z!dcf}oOj)#=w0>+U7w|ZV6J*E{_yZg z^`*jP^{GhLrZcV?D7}5?DC&3Nsp1P=?fK)T4_#9Fu|$o4>t|Dfx{N^^ZZ>3Q%q!5m z`N`~yH?DSwjC|wSac|C9=u@c%!TCOYd169Bg3jdg$=oI1eZRC_rSPAZF8+4m%3PhV z1wH_WK>V}a-1Bh}e~>kxZJY5!2J8HT=55$frhIue29+m<-s2{Z*5_cENZBHF73@Y! zEks`Fu_(wKyooD{OZ}9MS-FsdZ_$WFDOT#U>)myxNK@s6 zq!@BX6)b#Lcag>viHAqNjArgr~u9ckak_VkYQx>Kidzsh;7HIWs;0c*45alEdF_~GO z=-TP^sc*FI)SZ1s52xk+y>wr{GVTVv(z)1~lXcI}$BR@7T;{ap=4NIF1_t5bC~>b8 z{mra|DJGc<#^K_2ExVF-vw?`&&X3-CJ1{tS9Tu}*Yr^}(N_ZxX^TT(h zhJ}QV9Qz`2jz7f#Q9^AM7#z0rw7Y&pEA-G=B$%nwl9Po4DYa1qgBdJv{K^D~Ket{7 zwJtdu#WqDUxO)@y6$=v~-h~W-L`uAq)UQjoU$$=#uTUPv91UH9iimZO{4oE{zs~Ko8SY*q^e_X5a4i2(lO5$?T!z)t#Xwh>;C&FXf!(?&uu)6C> z2SAJvs?>*@Au`~C2`WO+ zaHlp`G7=1#*EI8&N3V=?@rz=wpDqCBivy!n( z>m@w6Y?{I^YlFh8cJ9%KEm8_0!?*`x&u(LsbY6i$cD+F;A^Jv44=*BH5m5wrKx{lW zzrYZUjXavKAinY|L4YphKCV=fVvo4C@IwhdRmIx z|K9muTKoUe5h!^$>uT$xz`i!~ZtI6wk6i5|^% zqXaDihEN_sU8#iPTr=l~IVj^HAq4s48oqI}Su7vSm|nX^WUboK{hoeW@8!b^7fo{K zjvXhCpQu&4Hpq_Vjup0Y*Cyo^Xh!%8?oK2k?46a;g0LuX{`lSZpjWvH;bLVX@kmfn zX_xk~MoqkZiE}Mmf(oBS7Xs`6L|RJQ&}va#=_u{yA}NJ>D?OrsI0^AJW#lVP@fR4c zb!ue@Ry$PBGltk(-&Fa4SB*-y5Sz?5~AJ;P%89^+21J zZipCQb5^Jh&dY_gj1L6gkl~d>Jbm1CM7ENxT=eSp9ojc-*4!OGA2Cdnz#|w4a&Sk+ zSjhg;<3D5s3LizG`i=1Ahh3U1_LPh=D0o5}nj~>{&9k*|23oHi=WL@F6k70@jN#DhY8!ki4DpV{D;GFOsdp2*n za_%(zGj`BUY&Af0fQ&#ysKt^QtpMLf!lDw730M~M5+o2G#8!k*N}@{3wxTH1pgvtF zz#b8BR6>VDY4GP7y&x<9$tGn+cE*N9^K&!OrE7GAx(Lky>Py=SzWwUU&UI^!=+mQR zjmkTJU&HEyyHG!05qFS5WD&#i86y)sZp6NV%2DDOaS)tVquq-uzl1QQPCh{9P)=_! zdKe&@3fLW$u|h&{YZcqC6t(|XK%w-e|M?@pv?^H#svzp7!>TbE4Q@`G2Tpycs-pVZ z^jK08ARn+2)Z-)^L4n3%A$};auq6^FxQf3y8ab3QWCN*qWpfLJl~~v=UwM>UEKg8Q zA+>Q_;Vax@By=psWF*d~d&fRG?(gN|+y!R3yf^8*(^p$Kwy#j!C!|BBN#mxFI)$a<(wo) zBMu0aW1|fLLIY_Mm}>AS&~qDXs0lHPU57GOffoUeIe_ApEn9CsdgRR6QzJ$WpZ)&( zR478m?$GNcp%0j=cqD;cv}gW7#K`_C;LcareY-eo5mjmjj_hu9fikp_R$vLPlx_(R z3pb^(yHwkjp)Wm3N8leupu~Ajf+L~>3WN!G$r4_|Lb)H?zwf}F-T4+rgC;Gy_JuwM z${j>HUAtoOm8;i6%2yaVY`CXyz`2;S#}Dnto2Ezao^|Wibx7Q0Jk0fk@Mc*rE*W?j zQYT7pPUe#DzrK9^=3~v9Kl#*CLRA3!V4K}(@-W;_P5pVr(wlMDNHP3Gzy9Hss|yw5 zaGW`IZ148Xd3olht=jeI-W>rzWuSEO?f8vr*LLaDnT(O!wrwNDT0ogHLq?AbC|gb; zo)~@{+P60+?T(L^=bl4{rca+iB$O4)7M;5gQ@L`LVI!XL^7S)Yt>@03Ik0ypJ~&+- zf4o+m`b3~cn?HBz=#EXBDBrwUGk9#;cqcx7{!dFLy!wi#uRp>IO_}}G22Gl2Z29a& zMn%=kxieqss)HS`iu_aE3AA7l^1}oB(}gJ1Gng z-%tVM2~P(ZN*RUYa8oc01 z=6v_kkimF?vwQabi!aQ4XBx3ct=Sn*bnE=X5A({GFCP~dN9d1~zD= zdvs4rNb7tv^m7jd}*|^tV7iAyD!{uD6V48gbyGr*U>ktM*4tZB`bg4n$ znvGgX<{$tq@_={J9k@oubqVi$i_(o2&+rz1Rr;ZH1WHGscmzt^#6YPd6*vhLP>T6L zb}pwmXWq;i6W*OMdh|2yT0`vyjb}}s@cP6t>wozr>*lqc*YA)rit&OUA=1Z@3+Et( zAEczLS+(rw-hF+Zej11Ym>do(W+Z?(8*~EVT1}59`@Qt$3}^u_f8Q-zHVqg$T;V%N z2d5=-I_U_uX zZ&z^TYMnZDc(QY6bN)x+VZlewoi%#6rzI!;ylTmjT|4^pAN0dFUp4OD4f$kuGlW*J z`{_$#aJH;bulAO`doUuTCxY7YdvWAuJ(x1-<(wPW%2ln#f6(7WfB@;BHs8t`MBYWDNf#(y2>0@(QoThWl^&%dP&xwtFajk` z<3{fjjhLmoPNH1{JqfpNuvTx`y0unkG}|1_S~Q23O}rI%@$8vSJ-V5^$yEqMK~}qE zWj>fX<+X;<(On@i^78Mer-D;Z+hDQ8^bijfQBF}Y0iKRsx{~pW#N>?|H@?E*<@DT3beLDMYQsNKa ze&cSGoHY|CzOsJVvd$ekuK(?K53;)0V8BH`r6iSCsQ;L6Kmg}rZ^Qw$9ow{uu2SX6 zE?q&wa~IBMXJx$o=3DinYIg70OOunEk(vtX$HtuP(4iBq{_q`5TehU-Ah386(F`$c z5$8Y=IT{UalY92;IdbL7kRe0I^m`hUa0HcnL8le8tIvgR(DN9L!2Od{2%{BVpl~S= zKRiUGpG!yJ|9b>V9B{i#;#_9VsESUf)z{YxfPx6IvWuwK-A+sZyvmg;XE2!JZzbUF z$37wSE&Fvp&iO7eF7CH&TTjHql&w?|h(iqrvkdSRVuXx+G<%%jl&oViA7qFMX}4d$ zbg6p%`ed^edlkt%?1ryO!p%4XuJ@RKxVgpOhzkl0CUpR6XDFYMZ~72g&_1zUeEOe|BT45oN&VpgsmiQUPr zLq|8BKezS5`7P%!*KOGJ$2s$FT)DPk-=Slt&IVPEL>zHdg=$eLiK$2&+IRi(c`Vq; zH-=;*SO}s+2xz4a!+fwg>PDVKAD=-_4aA)C)QQs!pxS3F{P}_6#V_`DW#1Faci~!O zml6`39w7?;;Z?{p{eS<6OD`-Pfs!@?B@Vbj5n24Bj!K*;+_4RTAz@+B|Bt=*0I#Cz z+P_aPX{68+dheakYv>4w6zL)&HUvbmE25$TDgx4bC-hzfq=epUNJ2?S0_i~d>F0dE zH3xltp7(jK@4c?KsDCoXaL#0A&)$1x?zQ&4TD=AH=HH8peUNl->C(kz8a6H!R-t|8 zu6JYNwr|`}@Z`zWi{~;k(x`VH6ckdjWa;fYc05W<1eIunOx-S=$Ob2>mAno0|)lqI(N2f_ikm%RS2zFXTidy zcVgmlvNF$|K80Lh5m2*E!$m9Bu9>@V@#2LzpX}OqfM|H~5$c0ONczFzz{>@T!4I`t zsXvJaOoPUaRQ~?!wrn;T^qG$y9XfhUV=$+sXMpTMLB)2-AWwq;06+jqL_t*U-j$K~ zP`m+k#?D>3Z`-y#=4NzKLc)@Hb7j4t(-!3C?%uxLX0tP0VlWilTeW!MqS&}w+1c4k zSFTd|2guUIyD3pTUftWPp7Wa%SqJcU?qobW}s*W{<|~q%)Lnf5M#!x z4^Gyh(%IE!o8EW!{KdAy!eQMb`n7HTC?oBsxr?3pfQaY%jeh@=A3u1vW3B3MzxhT^ zcHZ!juU@)#qjlA48`iH6t5?qs8j*#X;^pMmlY;8PTHpu)JQGV8Tr#v*^YGXD_9v1ZjtZ@u>VXCHMWRi0Y+^THL6bMi;@?AD`E z-GhhrJ9Qo_p_qC@-IlGUEM30z>o2!$Shr}&;;zlYn>K3LsA-cI2M-ZP47@h@ANipo z5EPWfEl_1J7^cjazI?@s&NZsN)URLSy}J-P!$*yby&c=8dZqR2R+q2az^OIawWeMJ zUL=R@h@Rd1wQ19;NfTg}#$vBu{bBsu=_wDqbHxgRy~mFn7~Zj6$9i=y-@G|<@j~AK zGqLdu`Hvfy_HKgK_`8U=r7Q@TDzO0WKoP$Ni~IH;f4s>+WKYpnGy_F5@L!(+dF}t} zNB`}PWNiPGs|0f!%&Z=>B))67Z8)ULBoJd#J9Bd$LqHb`DsCZ)h2Rs(4XvX5OmPr9Ebz6n^1U{S}!z*D?jg%j#3V0U{S;>?1Fe4dOS2p5g9sG zkduuYxY2BOXw29KMIXkm5ms7piv}Sn&PxQQ#$$sC%1_TEbqN_oq_}~@N)!17c@$F6 zc&K_#Fg;X_$LMzCKYk2+vl?(n0HY*>&B-tDEf$C{VoahwBkNo?M?pR-481SOPw0}k zDmqB-DoK%KKK=;5TnKI)4qXTmt%g=xpB%a=2@8YT` zSIM7w*901#sSV#VmhiY=3Gf_j)CQ-__kEn0B(!Z|^5Xa@1S&~kv2 zk~eC#u~?AS$KRnb5hV|uj`a`lO^CfUwUb|PLIZ@ zG5Hw-OIS3FXgF~NrN{>;*+lp&IICb4q%cktl67A&6vfw*o|$>$RxGZ8Rx)y^!CVgJQcxKv z(DD`CTuI5Kd6@iQ>w+b?d!ze;xu~gbBOx%ZeS7!DMcQ2+d*$)IvJ&L$VJ6 zbdqV~)~Nh7H6?l5qQy`#fHA38Fa}B*ES-+rtz@wk>Y0^*Mq_hIiFi6jdJ$3nuaZE92>{zCtbb!zX}u|pabQTRut{NDn$x6Pv8MKe$|1OILYo;^QquZpLU6n)Ur zMC0fIOss$;8XD?aaqetHs8cmj5`}2+8=)GF6*3qWf#7&?$Rp{r z+Adzz5EMl8xMxrPxP1BYjcb04ueg{%DIe^ygLzKK5+RIT>yrdP=fC$=7Ni2e^df zQ|VFVBWk%;^3MvnAwmF%v3}#WLx&S0+vVc*y86t5&a_@%9^iBHC*L zg2WLMcN0!gz@4uC=k0gWTs%Q(ixFSMO)Df{OL3B%mMfBGtT0j3JKM!R*H^`vpZa` zjvoCeD>HJ#x}@afCe2$695@hs+q7Xr%_oA0#EKX3rk+%4ljo#>U2O*&6AzySjJl z)vQ%3kRC<^EOy}_!lqWoC*0k%b!T4Aqb}W^Ytgm?pbSXCy#uLEhwbdCGe?gcE_6C^ z3GdmbH-#wW(jM2&ty`{LzJeWczzh9QjZv!c4sY~8p2+Kro) ztJNIy<{RJ^VTxckP`hFEsyel5UA}q)q4mluuRhMqTC;XpdPYja#!Uwd9PDtaEmrHf zGf}6bj+F~5->X-j5@pJ9376Hfdd;f1xVT33>p5&zACe`i)o0F}xpd|7*s)`!s-yt3 z`^2de6nuMe&=9;7`@cAF&L^LoJ#)HM_Z~P(da;v1dvWmh+wuDcY2u&3(TmzLQ-3^+ z&)y@&44~lqKc2`EMX=iozJHJrg6aIvpZ;ARe!oi5hoTwyYtFzQp69=22mj}L#iCw# z0~F6)8`ix0>dU*fY;{^Kg^wQh?%L(jg^QKSSD5qNcN>;2HhP>}HmrGb#IQS8uR5P( zzdLN$%P;niy>Z!Q$$zVF@8pDg2Dc;n{E1;5+FUt*p=_zr%YU9dvUd+;000LcLs4@q zzm%xFdG6GxF6|u_i;qb^aZtbHm>Wi{eR!V$+FERik{5i`%Oh0Ig4BDr-W~k>?U)-Z z0&w9b=EtnktXZ~f;_yLKUGUMXzZ?J7^eLa4+%_La;U{muIc>r?(cL|c>o=}oAqC8h zZp(({3x1q5v3Yn?Rbl?nUe6snb*yHc>K87arPcxjiV#E$-s#vGx%TZ>M(o?M-C0;@ z$xQ3pt<%*jm&3}IpZo39Evr_Tb(;PA_ENX3Wbxqhr;d&o+>bOXT9@PPmtR`+!;G>e zOCH+4|CddX8mh4oPn48&@?;dF#+in&QiDg8@!&ydo!TaUUqV2Ae0@VImd{E{1zW|b zjT@NY4HS0~2^O?fC>y$d#}2Ax zEuJ;Y-)LO9Y?(o?FCE~wd-v|qqepq%_GT?wO#Ajbe5D&6K4df)7R;Gj=yZo4je_B= zQE}+(AHHkRx#PU$Yq^0#2Rz@Sbt_^EhYcHX`S`K&Wy?;OI1z;TdFJ#U17Dar^-GS) zd60DEm%T65s-f1yKgfC>FawMxPo5kR5n)m}`t*7J*slj_SFb`SCIREtyxfITr;hvL z^HHyk26#zpHFd(ThTH)x zPX-RaF3QBZ9XPbObF%SK39#ZQZp7cl}XZR|!>jiu7Ugfvx{N0 z);&%8hzLrU11NE^x3ZHG!z))}uf+MO;b|>+1vYBk)~z>Cx~q2m1}$5)vOClUgSlq4 zS{Z4X92I>%`s2?(lki8arewtm=CIO9_me?+l6FwI3o0Dd`uer2*Uz2Yzj?C&h})go zvfbzpKFx=L=9|LVvcFBcPA-kU(4lJ7to8Zx=f{nG3qM7w6Qm?1qK~&|+tE(;J*_6Z zc^eeShsh~%v9T(nPm}Og6wM(tkswT*^~eGH(Oa*-_vO^)&06935fBpEZ}ix2$G=U| zl}VF6ty#a3#$g zjI_nq+`L&{g>#eFx1=uIrQn_$~ph=%L}`|0k?9PwD}u$%MKcc z6#D&I5GlV2jDLye|ImEVm!cW?E6>0)2i%g%Rfl&QNkk-zKNVj14v@i6r#7s=Y|R>A zR}6DnP|gHyX!K?)RTHJ;mR4&t;XX*HqVgv(lld1bM&dy6BOy7r#cD}U3k(d@=?u7q zP^&@7N~)?|uU^BtwcmK_ZQL~!JExh1dfs?Ka0CsQ=OaHiFSvLKzTLPU-K14>*(Dw? zYCp27s#Pf|DOIXh1L#we9|GKgfx#xDkIF*Q*=)a{;^Zj-n-sSV_Y1>D{5)@7>u!C@ zRH%xx%qNpSA33zgPcvt~+`HG|8@G%`BSTima4^;#5FCPkh+eC&Qlolu!}=@MtOH~b z4;%mwK4IM3ZNeKZoqNM(wX_OvPLHTArYgv_*(^Gv2{CaiCbm)STJYY$J@q+(Vl0oM z6l>IB<;p+2dDDi|x7o*GD|`~45LUhtV+Z6FQeWUzhWMM#`v*vs{r`63tlx-fIam40 z!M}mEoTs#`AimPL7op`o{&ub38vnNv;kN^d{wZk~;(yOcu%Sj%kAq|FJ3z9A7GBTbNkBm8>r1VeZx~5C^0}~0!bVq6=lT3M5;V`>~>qP zUcGi~-E!%~347L~6_gossoJ-1Z?{`RLrdMfc^&he(`oP4>-qhMk32|8F_?_!&R@*P z%Cy@oL^t9i0%*IPxD+}U&7G5wn@vD1$f`tnuR$Z-YPCjn8Yvug;){H0nHf~S6auXLBy~>%I6#wfl`&@4P zbg0Ir!W+F+yY?!7pAD;*8eE0T7tZyoP`PPyiH{fR;vWGNJmp7Jc>%1`L^@*cjZ&Iz+N0X;K$}5x#ytqrN zHCC5+c}sjF4$g$_h-Rf80VeG_-Z98;&bLh~v4I3|Cy>3zD=8zI4bS86`UcIke zy4=*zKas9?AO=koXQ`6GhH>m&V#CHdb zezkVpdZZP!lCp|8Ae~{>{CSWzFLdnOyF*0QlgBgX&efTGLqfx5EM2jC@BXgg;e(&= zbLY+-=wy}0;4%46pFKAyxcKnl!&6gJKl$Y2X0_`L95{I3*tZ)t3J0b^UeFWe7jZ!) zcnTTAwpXfE`}r|%zW4gr@LF|0`1Fg%c?C|5cKoFGex5tGX`On70-3me>kU5s?|%K= zvN>}*HfXqZ?fRD8dppRl<)&_aKaMH-R5SyB$r(WL`+xcQWo-Wx@-FLnTrsdu;&0%RKt&WqKm}tRYM9ez z&CSg%UaB-@xnRJ#Wex&PBtxEfd;>ka{lJ)L>SPv_>XEYFZgb}3+2UF^DTAHX+->#4IYOG5w$KaFHfh}ko#LyPY40QH#Z~> zVeYtEXJ_Y9gT}|tSA6-DxESsn&5~O+Ffc&L=_-aoD@`GnJP!K6(tt{w5!V%FK@*uKgaM~St*-U0h@&U*cJ4xvxAxmudQ9@Wb zr4T)%;g51hplFaRc*)QoqE{s;74xrgQl$vmhjA%+S(qImbC^ROU5J0KjtCSGhce<5 zDmkJ_Qi;j1u$Xc#WEkR^jXyDM{Dt3gBRNF*EmB7|+0VQYTdT~j98k2$49G*m^eMkZ zWU>{7ynAQk&-wk#aokGXTNGV+3f($=q0{9);+_)2VEb z5CQ{|^;(Dx6)Bu32LmVE^uv}R+7*&m+`kIsmHAHg(&3VNfziNG2{R0zw_`1lng zOkDn=1b-tiz;h8HN~JR*Ys|$0XxT1+ArK;2VyQlff~+<0%Re}T41^p;ls-i=L3+>} zc=8V}2^r3QxNp94CB|hZIS72YK}RjL;Nn5-mLIVqdgD6w0kLZS$Eiv2)`SrzOi z>__9H_0yRo*wS0CokNit3a~=)OExYFHHiK%u%LwRs2p~wZp^@44u=*NOD3HK?(!Q`}qn&L2lD+S6j*dtmqZ#^$RRoj6T@JLbLK!l@54&IBq1s)hs zCxHr?Z^Z$Fe?WiigXoD*ZH6KYBs_++gR&stPXLvu*n(DE0^|Zi!5V-SfYr!AuqZ0% zC5j}A1HPhQ(l2Nbe8g8Rn0NK;X`u^5tHtk8I1vu#gEC;|Rb&c5} zCQlH}uG$s3DKGO8N;G?cQ9zP&83PA%WrF>gBHEV?n=2g_8!UfS^3B5EunUl<4!waa zva}%mWQTw`R}%7ECIH_a;&=2!j0yqbo;Q2ewX0WTvSiLFoXpV(SFIlDDUwgF$V^F@ zN?8waEyl*ZlUv69MTKV{m!Z0a+;C=;v|@4%X3931K}imF*hsY z;^~ush=S*Ws>BR}-3mYeQc8&z!6yL@(cWB6@I)5)YW0sl{&@KCVWA#q3cXgU34=DA z19c#KB(a~%Y2XvRLSGhCR9d##uADvPv=n+_7x)I{L|`Z=Nlz)BS(ux5{>&LbN6|!G zTvF6jgbt7=g2wa9u3dL;#V}e01Hmj|^!Ykt+Vttuzf(e3Kx#tTRg{{f#Id(yjGdv; z>Z9x@KYZ}P?b~q#ddj$HSmXo<4LGBy%3wiK(bE<9{Wu>H6A0;N^}4bXSZzLur}#_0t4o1%$O`)Uao=JP1o}C}Uv<`5IM5&XnAc+8sN#jTkbRvv~{{7-w;_(6~h@ zSJ)D{UeQ)G1Ao;Sc;%5jBn?R*)C=8dpK#vq?;fuI+Xm z+`Z%bsbAtI=vCE$B{&3P6`d|I^1(u&$^jl|c0x!1VZ8n>JMPus!tPyr$GrZgAht`= z<`K0hMmho~#rIi24OJT73<{{zH}DnQ5!iLPlI}!L7|_e?%qLKY=LcOLEffBlTqtzX zzRtoqja8G|sI?estaXZ(o`QZKc z+%6luv8Rd$hfIPaL5 zXQ_I|m~aHABZBn?S^{%f4KXaY0~0gzM7|dTg�BsIV)zg?=J#iQJ61W6}+#40m{C z=8Ytwb=XvPNpn-&UA=aPLmZJirYj5shL`pabap46SG@nt{La3}~lJnetce z-+#Yva)6O7!-HOXo+wIi5hoi)YVn z+q~(-(ZfOh=F+7?&zv~6W7FE$n>PxaZnLj%@!+7#7te3myy58KL)4QgRl1buocsv9 z2+E&~I<|Jrs?(=V7AqE9vP8+P8`p#1cAI16s+H%@o>YF$+_@dsv~g4L_rShAYgR44 zbpBMu^5uPt1woj5*ZU9e-&?<8`JL+*tqxm`#a6X?&Fm*xYgR1ZwsGCHE0-!%s8~E8 z;KHe>Es^VQM_)y2HTd}j2N%C`{@nJ>8&4lSLdNhCp`mnX*P2ze>(sB&sEKgg&^q+d zYMcJe)HV?j5uG~28O)tCJIFt9v%a#cY4zb#8OBT$j zUa69wuixg#O%9vo)bS%5*R3+>v=u5<5|&1IV}WBKO!(z?K1__?xM8i2!L(w@;^-@v z>(r=vCpLD`{CTO#NmVPBQ=70UsNIEm8`rI0y=Kk5+jkn)tA{QOVrHeJEL*&A|L)yB zTGcNH_O$cyLN0}y=tY$ zW~)}IB5_REoUdItvvpJCv17+Wn3Ry>*Dqh%xiPY7!v-e3E-vo&`t=(cHmFy}UGCcL}ZMDk*2x^HWaytoABtz`@k)v$+ z1x{<>E6?`?2U$0`C}XcCX{+QGMAwBCHfdE?E}U-?PMD9@RahAL{g0!E4DuA_-8dWd z)`%BJ4jbyR7hXAibV%PGYPZd0Em%Ke+LM%ISpUe?E8iMA=;DbZzAD!TFAqQb%U<{o z6m?i*avL&!IA>A@rru9jHTAnsK77~Dpxv}=!P~=!PI~LDkO05ci{?z9@*xO~O8({> zudQCXs8*HA`?v4-aKbxgl{z;yy-!4^13Pz>2?_do@@KY;bf<%ayJV$R*blH<3Q&-F zSIzLh8vFX%RV&RV!%v@nII@4Abt{(_5Agr|wb6SvMZ%aQCnt>^H3}S8+3lmdcYl(a zUaL~o$mPqou3Ce3|M8@Wh55O~0(=_Qt8L9leVmnrA@szF6AvCd0CK==SqZVWDcXu= z;4eG_EHR(0jaL$VN?bdvwx!d*{qXCr`i~q&)tAp*wx|=QSvQvr4Lfn=lCMrvke0Y( z-HKz!jtm(z;6cKz$Jvi3PMCmax7BJtd-FO=>_=JYJJznCIAo80m4R9@wy8aLW3Gpl z3med{zi&`5OLjP1_y_2MGG#(1P8biFJ?_od8bx#%{mM($Ld(+czWHR<4+CH5t9Du& zHfVhQ%;_erJJ2~4*~*nG4;eCK@46LljUA^8EIx0}&mL>u(bLCNM$@DT6Pq<_28@34 z`Q-cYcOO5_c;}sWED-e`?U`%WP-q{eKiCqv?$pVs7X}R0!Ri9pYKUM+Z0HLj84L3g z?^mu|6(14t>QJfPe&hAmUmYEHt5d)3okzX*%Ea+wA0+nf*`Z^?y}PAKl}3#gibKc^ zO~XbF=Fgf51C*4Mc;?ip&fR-JWQ(m*)*DzaVMRQ8VQD-wrhnJ4R;|jx{$G9i`LeCs znzm?GwOr{blRx>;<+^&Azv|QzG5ncLs>SW6+*b*5DLjU^90kd8=sBh11 z9U?HX0@E}moCydnw~es~<)ZfSS+#m4H!Cmk?vecm&cw!#O?`b+OD0@u8_~Uxzj$H z7+$SHkFMRE7CSL7=*kA-WIR4LkIClI6Os7kNAESQS*LHuuB!a}?9>!61sxs-IAV5Q z7%|4K(syXtXwi?~cx>1eQ337nHm+#)QqpC0`H!a1-i7gL9o)MNC4%&w6hDwu>`GDl%B}*c2YBW@-{TsAxU#K;ftXxy2@kvX4q{W+t{M&fBXl*W4L5r?ka`nDB+Thwv zB9iYVn5l(k_YiwZH62)DwaO&eild3zRjF>%0z*Jp?FK5np=QhW7M)+EDs?@1F0Cam z@%AlH{<*fT8&?eN->Dr;_SNXCH)5~1>e|8K(-l0hM;ekg{VJ9E*3X5r1j5eKdozP?1+kiQAN z53myd4aL!zI+JEU-?(crnB$yI8yaXr;{CE^%c2}Ao_TIerd)l|q6INAF$WL-8WtMb zp<`#75uZNc^U;N+O>tatI0-Lee|maqnKI>AYS0J01|J!rkr{+I{7Iw>fR63dDXuUI zT1j7oNWd})dUB}Z`vRx0#il53?@TQAQiDP@FvW5hLQANBzKLG1d<1v3RjXEIrhYkj z!rw1|Dip+I`uY1uMIAeP{u~@O1GnVnI#M3u%p<;fN|zu_yxghG4Bb#XaHLte0{(}J zhq+#l6DPNF{(`yp?%g|l@V7iX5g5W%<4H?+DQG>$A zk6ZwD=h5XwQI)BTU$1) zcje_A+_O9S+=b3PdlP(LqGajh`|)|XS(bwACy&#L2L%QP1?}CjD=Q%h|7Vm|#gh)y zRfGXBsnVFtzQ%IpZztS=LB<)6zX()wMZs_!LRcP%w<2Ol{6>&j;`;*C&JRx1lsm?W z9B&)}a>T*TP3zAbKPoPKcDqKhxCk^WR|E6;EE}>wp=(5hINh4?rp;8oe%m*1VrGaq zJ$y8Z+HajBI^4q~dSIXJamL*5zvZH=6m%NhoF9I;8ykZLOx{6pY^MoY*BiXqOGHW5 z>D><29x&Aoh<@n)^z^jiC4!3;3)s7NPkQ`aI*&#AlaD@Z9}%%;!@6|~7GAx45r?0` z+#GjiR;4PHxoZ?x%gV}n8s@{e{uOF7n5W}W237Q{Xa@eo8F=P^8;%!kk#`kfGp+l$Rr_V~{pH?Vi(IH`QN`$uR-FIZa zL6fI^+cBc+8=rnQ^}~Zi9V?ei<}k#Pzs4 zO>5O#xo%^bnsw1uZ8$m_4diZx6au_~Fpv-8rb+XbS1w#cnU(SjYNFrKpwWB@#>96{ z%8yZAQWg?W*Mi|>Xwf;LizL#L_)01T3y4enkIpc6_H1~3L3v^(rF^Am(45OC6y=@M zb#`)3>9h`vn9>s*;Ei8?JAL7@RV^Ac?K^N#TtcEsYpmI@aYX+?@4ot4+lCEsW$_6u zh0|ktO6L06vx)}?@`gAZ_-i5{xHQ!=aY&T0V3{QxpEN40mDq2?JgnA?diAyIH)6tT z)mgq~UDl(WQ%K;J-8js6xt0_`h`U-|ryC_RqOW6!HNSAO;UJm&<9*ewES{Ffq|%;I5^`xp8ljDBqT^D*#+sTv;#YVwipl`#F=RB1v$AvegS$YU@pgn-Byj( zXeTHV9L5hERo3l(keE26Tf~7QM}tDkg3ma9|%YS%sC3HBa#TXmyy$ZFjG9W zWv7?xr2~&LANOrq@A$14LqMR7h<@Dq+!AF8lMPgMFAvqPTU-*qBK-J}14qb$fr}#5OPV z=VpQ3WQid%o`x6~v-;;LZvm$fOlZ?5@p;4u0V&{3%yIHvqlD2XC#ZFzc z{jbl!v**Vx-XG|-@ER!IqQrr}lGsRZbif-Z>?sU^pD)Qb1Zr45vztUyR)^cy&je85 z<_pPdBpZ~3F%ppuYkl2J`U6Hk$Bole)t~$ikUoutOyj3!y~03&u+%p`G{? zt=2#?RTM^aQhte|3vejV!OKGYx!Kpm2O(W?=Fn*U{R2qP1afJ029O${=Pwj45TJz9 z(5THO{1$Z(=ONOCK%7}q_(nEBI64npX7 zM#j%a!G|BY z^b*tK9hn|yM$Z3NJjwt1i}<&=z*FFUisB+%)v6*t?my+$DQe%Ja^?S%i}Hy5lg5p0 zi+2~a6LXr&X3ft_Nlp%e)h-n#NB|zWJ=qy4x%qiv6)VDBSY29!L1Om2s)QKktT_qs zx5Fw_Hv9M!&?i_dd686<>9yt;SPLGLTO*)YF_b&%!3h2c9Sa=DfI%2SCLd*G1qTH9 z7}SLY`DyV9RPuEieHH2*_@$s)A;k(FWfv4LT}H3h+i@y1Xi!E`XYmAA73AE#mr$W< z6#$=1;feyQB>Yo(tOfb@!U8l#AA{b<*WaNxD|tBVTCJXxSoDBG?hKsr5NZ;I7vv@; zCRC|i2WCSjrDYO(MJxHiry$0MAnmItq9nq;Q z{3Yv2e#x-15}nDp{7phAS#~&7cI&u!&{?I!a;J)vZkdJO zH)fjcDR7G%EYi4@N%+%#J#*_C1`LMs8uR*|KXtN0v!+A4bnOw|Y~HLNHRO(RItCAZ zu~T^X3+>x=?GzCo6D^V3prgFUE^Le!PMv67t?Kap{UT~r-?wuIFS_Wm0FUB2^7WUW zKNlX}tx8kIM!TS;?@3X47p{sO(K^CzR)*QheQ?+YF4*50*sliF#MY$PBo z;Xf+rcK3$$yESUkwr0(jhYh+DeM9=opkX5`E5YE<92af2wlXztKF@G~^wvY$VFs%7;m!v_xRTCdjbt(#dD2wkIi z-O$NByLWeO*q}?}hOKH;9X@zqT=Y#+arN!p!UY zhb}uWUO9IwX22^iPyKqb&TS!5G4|HYHZ9wftJfHx3iNB-D!kgd(CU_giLbpf?we_s zVsF3m?UZlc8C#g24bG5yi2<>3A$^&!lQFjc!)odHGyX++bWz5=iM>PIIYJC-2-_d+@M5CwJzz-;f0bGbGc-5{KrP z0R#Y2rg4W(J1<>38-4f1k)vl!{aR(q(`#HYx30Ht-8!sZLyQVa8WWOtBak3hTpC;M zgx6nw`|GbR#oT)DtEu0PAD^F_jSzyC?OLKz#i(P4@sI1$turMhNorB6TIH;?RKaB7eqkA0 zXQGZ#baBwo5e}7pz@Wjvz^PLwl~-NVUd3PAV>asjOvcvX;pJ-Ae)2d=*5~|5On+Qf zy=TYva-pT$w{3&=*QilrOmj3|yLK&SS_)Q5K7aJ-CojJ;rc~)LyzV#=n;u63F|>Z^ z{25hVcE7>HF$4DR-=94fFPsy_Sji%dn>n4t_u0p6tX-pKv*t~+9%bMnM=LPMct@iG zqsyXHB8r+c4JR{)QKzd}wOamzWOx#dUOVZ7_g@)3resNqSaObxSr}x&a1T1KDmQP? zpdogTy5Ea~lszZq(Sfv3_-R2;jmK=#)v8&&Ws7EyGcvLLiX)}sZ^QZq4CgB?H*V6T zM5$6xK-H^MwLDCr0h)P#(uZTlycQbDsD&a2X<5byfx;M1MyYah1`Zic=^3dU6VatJB`o&u-bKm*#bsN;W=;UT`KU~&^6i;>{*_S| z{rcO_fKYqS)CM`rc(r;%Q1KE#GCXhPsk(mM+KtQ6`IEkS>E#>~{<3~;&J@z2} zZofHmwAj0#W8t_JxVV1(X7g4p1z^E@hwb{+Yxi!PU9@!BsP{iq8H|*OL4$;Rhkupg zmIAaKDfb86H97u%*&4MC20y~}eayuID^z)ung+PCN+2H|rcutZX6Ic$b?(ab%iA_? z{`l)}p-8}M^iGNr2*69mZLm)5uv>3ky%>A-^73U%Uj5)>Y@38dih0#*UnEbsosdkl zbF8wM6CDqe;zMiKHu(kQxiuzVQ%J?~nQ7o7O9)wS$dB4pn3r|__%Ana-P*c!>&KJ7 zmQ;h3Q*hY9XtA|Yir#yIvoQMFb-DD?Wv{&d0U5ZRMAW0?O6fkL5U`BaX>U!)-K;)!E7Z{64txo+aOKaqWr63@+B zwz+cUO5Ya-y2RP$DPckvq{~Pony2H^K+#q-1Ao66c;sqc{8L~Jj2m&msag8#6(HP7p~d>{r1$df-Y!+OO7rH;v3vsC zAiFpQ;*sKV8T5Lqc!WULh)EzX4?jph39&Ky!7by5Q)sn_^EZ7{GIr6XJvaA@FQ;VO zy@zu&U@iV5)IiYVq$dLxPZPydSy)gwapJ`M_}e8aRc6$v|8f8rjp(B_!D94jIY(hP zR5>|0l<1@U3iOjhfbeAcfAGZV)A#O=e(UYYZ@rO|_7F@2=tY$l5Ej=!E^hV@+;{xc@w@j23>-3M;ETDB zpTNK~3I#)PpVPbSHc7P#j4CAzOg`oUOQGn_D64LFZlTRTs5nTBP$7*q&pVwphn3rN zb3r~a?F`oJ<6CI6(v(z0P-}AYtwF)1JRZ@?MQ0{B$m#YAD}VgdxwE$u zMvtE~_4QYuq^2OM%7=yBj*X^ff@DZkUP)T>{;-fh6teFrW%5T*|2w_BMSXpCXW*Fw zZd6Jo4Tz%R>9k$Dcbodvm#BVu57QboZtC@7LpcsAUVOm7L3eIOr>AECO&HBUpc0{F zA4bPwm($s$sBq4mJC!O`p=_h~@d10Vba_Af`1yD1-ZLXER*L*eOb2Kwgm2|46*JS) z*(0>F*uxya@zCMJ0P9N^E&{QlmO88y2Y_8hKNjy7bXq$OCz{YQIs?@R|UAoi4^{ZE?Er6gRxRL+q1a5C0l3*zdEe;zo zd??EBg$oyi%t0gwQsZ3c!R-RyJ}N&F1VmTMlq;Wd^QN_+Kp=(?g4T`aw$9L{Q)h%o^v!Fy@kyHsnS#QP*btf!iGc76(D(jH-0`@+r~L_WWD|+by|yx$*I3%9mrjObDH2<1=!DrIOxg?$*5r zm$`ZQDl&>Rt(B@)fu(2K>8kg>h-{D_*#xzVwxSvMd(OZ!2iyWsddfDUIJ)jeM_aR= zSe`tY_5Dv(nzm{Z-YP9AY3te*`B|i3dayonL!)*bOO^=E&B@-oeFxreZQ6Gx#YyCf zrACKk!-{2rtt@>!JCx=hg4r!#B7jj)LF}SjPIj zZjBo8w_{XpyFsU3{PR!b)OKb+TDfqMDlnvZi?)aq%j1mf)cel-$JT-;SsAH0xsPm? zygfTMxLpM;TeVRIglt|s$6(7_y6|V83gsI#ZLTxwtxmTCof05(*$gUM`n{NIr=q_9 z`b#wPn)Msrx^!;!;`yu)6#Novo>65d5qa9(Tee5(YgQ~(1^BjV(-O6}fH=s9i7tG| zbDlg&PqpRcSo5;UC<43ErcFDQzwd_C%M7l<r}sW1_!P~W{5U#(7UGO2^wID^ZAL>O&!Zm(+p_k3TA zdR;UF$_zYnz>R)JDREw3aAeWk`5o%iYu}*3u8T%muTGuQCr^o9?s6Xc_3*&)KhKACAz}9W z?@a90y2Yiaqg$6R?b@{Q@+C{JUA;Vg;`pTd2_y*oX~BXevwvz-wbI7O4NI0RAz!sc z08z|<_>Jfdx`Nb)-P*MtGje1xU%yp5c9aS&vuE2@#E>Y`ur&x4bX%3{*rELc+qG?5 zug z`k_&!3KaBNxNxC(T8pZS7BA!pYY1CbX7AQ5#*BK&Y&5OezCEmT>FwLMW~M&GNPq{q zSH0zosDlUkv}@C*Ufmm4uFhVtm;yCChabHAZkO<;jC$jeC0)atE|@nrCi>dc3FFce z<4rz3Kh2r9c;TD|70Yg3w|eoS`P>mMCt?7QCT2?QDacIk+_vqgmxc%W`L5o!qh#@t zT=2w+Q+4XrSDZAZVCB;epxytbaf@A&pQ5d32L6UK@W)l;-?01te_duZ{Lhh-y!O1s zq_|^s?^YKSSV&G3;0rk+&L5<}U=5y?{TTb3Qv8uaab8g8DPHSo5rf zM8TVV{Dg(&7{Xrg06{w!6D72Xj-*bI8#%c-0RjFJ$Ia$)W@crFln(V0x4;+DhHkk@ zWrdVrP{MViWqL?L4ZQVh*WtTqGwL^QMzkQ>F@J@qhCj}?6oRLI{=Rfo{40b(66qqm zgXCJ$I)sH$<>y%Z%{WIAg+xvk&!Z;=!Nmh5Q6MD0B*Ri-Vd%RM7pSw+K?**SLd4*( zbnn{Zt7$(qXcCUIjCg>c?Tg^hTCFzDCF=_pgTG-h0OjHxjn{S%{<{~NGsm7v?4p1MwjE*uI)2_n!WYV5v|Dtn?vFP7p^6^?_)A$ znA{A;iOf-4BpEd~k=dmw!FMI_$0Jr%(nZRMw0hi%FgduT9T*?E3cZk7<}-Jnv&DN- zaorTX9@$8|D^CD*G^PJ6wv+{j^j?C$yhrk&s405H7aHR#^h!LCq)o4IQ4GAwSJLsa zbWj4vB%Def6lsuav6Gb?;R0MjZQwHxo3tkPLdNK%T(Z~ch~9G7Xot=;K$$a2 zd@=(PDJHH$++2?licfit>7w#P@#e@A|8y)uLXkCIl}WCWoUGo>Di*_bBG)Y1{x4?W znbWv=vY%oEkEfOjh~|I*18*aiGoERk9!sK%2zU}+@X!j7DO<4sU%4A7u0V!fUA$B< z1s5t7eN$d-Jw9gk`em&*OFC8EOb$d-FKND}8L|5`M z%DV@IK>`3~6w#GD*Rs>kFF?>$O8QCe6sppI@7y^~5tx&IgtP{=d53A%`{1D96D8Q9 zb(=a32y78obWZW&9fzW26Q2H54^?X%c7R6OBu%(BuIxF z0f;Hymi*=vc_b*xqdXPznLSYU-dime1VK{9i#dcj@2AmgO08^Z;rS!X` zOCn*DV2CK02DRbSFQ;nFW=HNk;j|5nDt?`R#--|n-y!sM-L(_mI z9bkCWW#7AP>+Y=^d^GOc*DjtpdYJPBUJ!1KfGv##S%??g;5kjZX)vWJAlV?dDW15ijxzY5JW}ZFRVC;XX?b*8n z%7e&zN9t)=6!ILZc|hf%lvk8TQ-Kd&I9_;p2iPmG91BU@zXgL7UE9SwhOe@n7fFNn zngA(bio78}BDhH6s)XCIyo0=5ygWj|$bP{t{>Kl*>jcf^G9D&vnLn3TQeJQVD4YU7 zAt(jX3l^i=I|UF`cH7w#Q6Q9rL*gP$%UnsKw?&o1?+qEjpMV%)52*MBA^0vwm`N5{9b4luW_0M)>D(>w@|KLW=SxKcU!W~+MMU2 zj=8Ni1RF8~1Y*44g-j=jSdkV=@T3A9f|g7nxWjq${NlM&7tfz_*lqjv?OnWNA=8L> zkff9;s{Bt?(1(JXp!5V_EbPw+rIv@!P z3ET2G=(W=>CG)r!+!KEA%b|nZ5gK7XrJnJFWNK*w=OH8Bmu_du{d-&1uN6_JJhwE> z#Yt|U+!xZ<0NgGkoS$@wHj!!sh$)l~0)h@WoYav(lpvesuF37;Fi9Rjhvm^ytEYZ3 zdG5@g@NeY+o;aQX`cJboS+xBXW`H;G+1g<4KF=wjPgJG*G)eM_zJHk#`XaNIHs2=(gHTqA$_7>}HI-y+Ap? z0CTVw>~cB_A&5oiV<93e0bPU|wmBVEoySJnLZ{0L`~VO_raPU<@weX}(o5^ero*Ix z<#1v$(XrKTmL;;lXL%!Bxmz%Z>GgpLa5_v(0rSF!c4#oY^U((@BO{TDvZ8m`Oez;Y z8b_fdf))BoTp`iWu|fcxJgmY?Go>EajISr}-?4=RFW@O%f(?;@vrs_V@f~>bxPJJ0 z%AuV*bx=JbsWD6o{E5i6VR{gmCyFzVg5+Dr+(6e=WOUrMWt(2B>HT~!FE4@E(8;Y5 zo*Q>Y=FKU7JTi|F5fM?>uakRS21zy;CYu!QqUFN+i>%^#5aR$*0C=(!*~H3(Kf4)} z$VjkNo(Ji$Xt7*`Swc>LvfOm-fG$na^->GNn2Me zd3Yz5$q}K*0*kqkt~38AF@p3(+n+cCV9T?$3Bf0&3}i9BHF9mWN)@i%x{0Fp^2k>+ zGcz}?TT9@3czE-E0|t<;b?c^$RVvoH6BiQ~d!t*oZs9Fku3o)5IWfLpzX5d{wNMml zb^Ptv9XocVrKQxaS$*ix!5-gY8#hKK#N2$8oTxUMM~ogF6y$&X%GI6Qw>zx%=X&&M z)~YpiX9y@)piK>{4sa*lyR&8Mj_fCodOr7D)7EYG?A%VEP@mp?i5bMQw06y^(jorm zPaZ}MojY$TLRg+XdHnqOiz7zA<{LmY928G&+heOaA(Knt(oZ=)oa!mIBb}|pU;*JYwpHIXFPZS**$pFYh+13eJbkQ=_u00 z^zPj|q-2;p`U<5hOp5_vXTDgn*uS6t&CEGJ!>&XfJxrMkUtj-Ue?3&UL7hQ^2kzN* z=-jz;9U|Jd>llF}PpH;R6SvxVa%gWs8}GmQ$2pMHki>6WL|>@y~s^D_UqBq z^vn@M1`iuCZ0UlZKbkrB0#kqf{Md2hxS&^G5v2ESzy164YtcXdS2G|w$1}CD>V|0G zML?I?v2x{yZ@hMJ=N>09;E%JO@7nR|<#Salm;3R%shd_UH@V&0)~*{ja_GGqS1gaR z-W@jRr5F3h-@4(>&v~s+&xi4GX0`j;>0`q?wU4=SzGCST%jeA+{z7kbL-=N@)fh#| z1T@vx&Yl?4y_2gTC(ze8esI6|xSOa>%(;*%@B?UaYR{Na!(G-~yx+#Z*gqlqhSp~N zX53p&OD^6CX{kv+PyUenO_Fv*v4;t%;i9RGeLVT2kErZxay!SpI%?gr#nmg7JGg86XCJ<& z@z_%m<0p<9>BxDkvgC~F(K#nA*{pIbp7q05W8Xkq!xeJomtP_T5H@IzqUq6Vh;`I2 zltoxrl^gHxo*mol*|?!nnKCC29~#=GX=Z$kEHaQw%!!yv3&)Kay70%DN3@?c*@!PL4O(U&d^ZQuU(waa1ULKiKZKXl;pDr+8shpDDDL@dn6f&F`@ zP5nBwM98UQ2S*GUi2kUA>oX8F7%d|b3PhKYN%pbiI!zm)fL}auWa_xr*DPHW#-Bg_;N||$uUok!B+&2k*GKK$yw>1$-MDn_ z)t;RT^B?(|)Z>Q^KyLt#m(5@B#p`ckMt_`^HskXzP_Gq{BRVtBg&mcf9dH~P_qKH_ zUw`RE0-*Bq9)CDu&})PG#ay}Q%+GzZcaQYgn=H0w{xD;6Se{8ONhX~mL-MuR!n*LUyk-J@R~$%4LF^JdehO{2I~ z!{f&Z3!E`+I)p=`Bgav@Dp#yD`-kbRBHGSdwu(1?_=_)eZ`JbnuZM>Y9(>{OzA9C# zy*Xh59Lmo#zwa~n#i`#+<=sk8PCk5a|Cl;;?eaSj<+D?dUi)@$uh6tfr%s(JlrFw?^@_3M-VQLCuU))&xU2Si8@oU zLWS;~+xPF*-IkYEAvBaMD`uVcNm}Zrjhjy%Ki;cX@8v&y|KT^^4IDWNz4UW?;i)rc z!8NO07y#g?T!jiFM!xjxngwr-d)weweD0i|T~^DX)2GQBHE#Snt(w)_uzJP&UrbZL zT{P~i=TApR9a+8qX#F~MRSs+G)@=#&l>P~|jf`{0-?<%DHtfZL{mlVE=yaC69A1y4 zySG34z4m)|`M8$Gt-+k{Ll=AW4f8R8sQm#;YS-S;TG*}F%#A^m!JY}PhS>WvyOF!@1ZU`T0h0U`iG1_0z% z!2jOAA8#&S79ULN{|VV)z59acl)~_qAHwJ@%$3j;!n3J;eO9lOXC@~p{@DJ5QL%T-zCO9B ziD#mYJU{T|+24QPb5LgOSVlBIxCd|1d2{CgxvdWEx_>UYhWHtzTGOEbRxrgVjJodyi-(4`anU6pFpQtsSgNi=xyU^2+lGK7iv zp9tMwq-p=j(EZG5+~A0qnP3so5;fX(ox3`y!=g6CMBjRnc)wYdY7NSlo$<-1(Pz$C zC}pP6hPP@>N^6hKRHI&#mhC#bwPw3pTdRJPl+<)gIng(7wvXrpfyo+zXAPr?b?CXzA19!&|xpZIxB`DP9ogK5O*UQ80giyb)B;1KyAD=u3nFR zd;EkwtJXZ%wadO8JJ2+RNk`j*aK?zLa@T0y+$W$I4m5XT??8?AYS*rA`SSg{bfXKT z>$BwKP5$V!hBfPT?$%9JV9iK>^e`m@ZMs>r7Gf%OtJ=2dfTE4*5o|zxM(qN3gr*jX z6oaojF8*$-4xQ8{U*?m#z!4q0+`kvk+s-kV{#e)FyL(sV=U=6MLknZq8{2p4EdF?y z%fv}pyflUkALayKTDF`wdzQ^wfOu7@?VpW%J-k);o8#ZHVhz<`Xpm(Gw-5&w=G;Z4 zNvvyb=dJ5kI&|)8rzW!27+S8pPpPnkd-tJytsJ8nywO`Ellv8dFwYS(Vq zs-?*{0FUnKHEN|kNM(LwZd^aOKC)5yiuEg2Ik|JsnUkj;CMD5;qyy9IY+7yewh?%Q zFk?K(xbo>m5s}AQOp!cV8g+Q9Rx&WHwoZfkZ925a_|Eb58#Z{5@(@G$`4dr{JG5<9 zy~?6R3+<1yaIRqvC%pgOm0LIWZ`$_h=U?E_&QGb*<#;mT59u@an)cYdqQW~0@1COH zf65FzgBrJF%n;WRXh)TrQdpSGnR2wkUT4IH zgtj3oQtXF1oiV^K@j)^+M;vX@P4d%Hiv3F$h9pQ^8huP%gf6P2o6D4f|zXFG#p}hz~DhcyLUfz;%IJqdiS2acrmeO!NrRu z&E>9LyEgC8;j^ctUL|%c%A^Msqr)AqUR}Lv#TVno8vFv!IMQM6(iblp!%E*%!e)G^yY40BvMw45rhkjz}fH*)gT ztC#E5Z%BhOQHTPSs%oXG`}Q5!xpm9eQ@$#+wO~zc+qDyOu4M&Ye|g z$?4Rlo!f;Mkc11axOQx;PoQ(|to zy-GhoXs{XcrdkSY`kH*`pbSj;Gj|<@n;GC%dz9MDxD~<5a0yXh`BzCi%b&Rr1E^8E zcC-4m-*|ltZl}a!sZD0+Af_-o=W&?Lsx=s6V`HV~;;AY9kcWy#36m>*;JL<4#9-#x z<|c}CrM?gTM~DqE4{P$XA59uP^7HR!_Itk1@xy!Hd2=joIaa6h#*G_@V0E!#7cN|E z+p{NNo|c+IYvgU0MwN|;(i((Q@|x9u+grAWJ$u}}aqaxY3s{Zr+=+{hj^>5nmE-lZIqX>D@P6|+Z5Zf8S=ZtZfF0h^`-Og+ zwrxFiA}T-U(SrGN_*3hSZ5=9i>9S>FVxwJ_LXXYTt5=_0dk#EEO2(jn>HMY4j7Rd) zi)0Xn=Cy5`);RMrj{UOd=BZO% zd-XvDR7=7t&{o~NO$U|6uy+3J!i<#Nn>VCHUjt7Z9tXKfaC+0LbQtTB67Ewip>5|* zA_IUSkLLLo2EFy^ryq^|FL%-@Ksdl-@o_fwsbln5CWkDLWj_lCMeRB-m!oR2!dco zkuKQQiiq^4paP0C6_6IX^j?yXgpvZJm)qa(b0&!(x~uX#3B%l(GpEk^JTKl;jz}6=iv`1wviJ>8Mhva`wfz+}vC^d^SW7 z&dVf$T#!g~nnlzr4jR#0LFi8h!|;+qrDBz%bkOCqkwNz8XTJY-%(+v_YKITbhU^NZ zY&$Qa-9&JEP?fCE{(G=$Pe<@3T1> zNmq{V+4;>EOU!zd2;DcXrMxv`I5*~UZJ!R37Wx%=H?#RAzW9+EW<0niQ`}}jSzWj3bwIr*@jvi(C@GyMp#>B*I z+O&yIhZ67+!F2QY7fs)yV)J|8t^;o78`!Iu5ici38Ssp-JbGQF>NUCz9sc~V5skxZ zy)bS%lJFe?D-id5s#EuO_wc{7_hph=E;uG>@qJ+2SQci-!+mj*7fSxEE1oxRi9Y zO}z%MJu|xJQ^VlUQ9FS%A2eu?J>`18egmK`;AP363QfB8eyV%-$y49ZS*(-Z`*88X zr7a_ybQ>`6OzcJ2zj^Z)w2N%gyjhF74IB08+aG>VfsM!mU@3K0Z}bJ>#!w2G`1ZSV z=Phhpr(UlCkDN!OU@$)Z5+=bT!4vLp7)v|Nh?j;wHDcnFR&DRM z!R^vr;E9(BHgMGRw-UHA831Q&{(8ZpxYuE0F4;*Y>GveyC)Tf>q)$+c!!v+p$ z+o0a!rAuZm|1zLdS%b;?V3*DZ_U>t1Bm7EI(wG-t`1s?G8`iFu3($)5S}PsP1>t|F zi49hn55sK22c)i4(6K`YL3V~1^L%(~1C}%BC(;UG$_><+-e8@BpaOwA(jmJjkqe<` z`2klAki!FCNlaS)!G|SFl%U2UnU{#mC}x{<7BhRMbf(j2wTt4PEIMeZ-C)L!s!P}I zU{zWlp3jKKv837Aw zr!6n-`iIb=Pbk&$6GSml znGK8Q@%Hv*f`PaeckbNTAFtuq_@m9_!-86@s}1_}>#6?!0ZdDb+;D%22(nyI1)(Xj zICo{GXIU&JBHF+eq9qO1fZ75XvOsXjsu^i0DaAZTmzbDHj5O_I<0O-(IXvb2x!+Gs`I-CKjDUy$uZt}FLB}$YS^~|$ufl@X( z%I7KHs1m(n4TNI_EPEBTW9$)(!ni;`kVGNi!s9&e2pyEI2#3XtN{cWPRO3=YY_|v6 zulwPLz_98rzGPdTHy#L#M$9bb+vDp&0#@X!L_XiY=D6#CoAo10V7^x6?nCS#R2MMI zU@~AvjSjmD!KScTG7V(4$xL7#0wRJM;Glf|z?_;NYSg%5;xQkE$;IOnKCGEP2xQ=c zxT`Ql#M&}dv8M(dgedSpM7vBHUKK}FQVLuW0_n8m>q!^S4jIsI=Dhh$TC{{Egoc1! z=r)KF93+RcARSnYY+d4(f%T#SA?z&+Z~+*2Fo8}HY6x4;v=LSE6T2{A+5f<=sSYlZD3oU{-HBpy;XY9h+TB6R?X#9$BG zcVVS5=V1X)k<_6&8f?P`oh8f!HZfTfP8#8;cz&38Ndjln(P| zQ9KZVp>BH~`#j8DKR}|%ya4I&z-mEgMSPiHFC(X5ipfG={J~ve*QidizUb6 z4?zol+O8NNvS$uqeEKn#t zEfba*C+UC<$-oW}mK6zgrW+X$LRAoZM+{hm76R1*pww9j2Gnpt@1!Fr8$=W2b;?>w zLcvq0)LCFxNTUW(KqTEKq!T*lAth0A1WcjGB%uP(Rgc?7dbbgKSnMqY5P?OmX$w?Q zuuRcJ(t!NbR_ZK7K%g0Fgh7v5c+qMWF1#yvPbnNi1w=JQstgkIOOXT`23W%#^Vz5f zO$bdaP&L6>09c3~C5k;H8!ke+9C#y9D;34`ORbb~rqrJvfOzwHU@cE6Lan4YiWYZ= zsJ{r!gbTsGUsXjf0+No`= zn%{o2iiyWefX(5HhYeyi-KkN72OHJDw_4T59~p%BRWOBVFLOpRf&WDpqIb_;TqIt; zbbqVX{T}YhD^n)CE(>iweeea~(CK#V+SRI7je)(ow~7c~_2mjEuU6Z=mbA3E%5 zq|a;CtbMRy{q~LP-&4Kn6ORmJ*OckXB$mo>fPE5ZhO%{6uO{8sqD9|sT{|{xH2L-M z%s}c46bj_HJ^Q0}HwmlKzh}?;YDIjzdKKvrs))vgqB85&e)nMg`VTZ{(4uaGwoUJc zJ9TQ@X#A_=MDszRB~zWYe)rv)_Kg~}Z``m&c=bmI4m@-ABmgjN%A^n9|A2~uZo+;7 zD~bcuBqV$#V#W`4KrgAQEMEwoz`8(&f&0L>V5`DhDx*O^n*kOX9+30=gg_)o3^B7 z+h@&~YIWwxev=z#8&C-Ffw+o#>N8{VOA!rg?mcmE@y0b9mn=N7|A0lj0|{7o{EKaKF}yWpc=83$S4Fm89kQ6*uhm3-PWBjg2S{}7)-A$CN=3m_&B7YJJQP8VWKh-#HNOMRit&|Rsb{8H0jULGV$p0Hf{m+!a zr-oefMq0+cA2$zvbTDG#nzbToHfy?L#}2p;s)6-6s9;QAcJ{C#TGsMf4W(@R&b;<28o4=V(?gL7;vmJ1GU)w*?7R_30K>z^7r zTyL^sxKaJy7TbQ>%sN)q-EQ6P-8*!d>3#e4v%B5hdi7>j5dl)rODZCVV3E`1YcbcZ zUbRKj$fT5H_%~1sHjKfOi7ITFCo?1M=kGTRdGaab*|lob3BUK=pEhrTkrFuYc=qhx zt;^2r)3-Nk?cRO+lkMJJyI~w*-EsL3;|_ZcQ|9MwtyQfWvBB8$ZJ!BLy zK?%1f&}yYE%^F1}5m4A*tPm2~bKszlKl}iX9Kv`?djT1KpzHyV%Rwu&LmK{D2lq+J zpWuj==66!^v@o~AoK!Cr{>s0QveX|J{Lu~k-EytUy!EwTO4ahz(vwE;`ldsEsBB5V zkzeP61hDj~lD|w(m764$`>Hnz?-aaL^#5Jd10n*uyAF|fxLFCJfd^v~UcHv+gz4PX z!ow5dFJo}Q20d&3q)S)I)v9GOTTy4TSgpmXRK0p71+f7PuFa*hvI(iz=VV+z5ViT} z(W5I@u9`Syx-4(SCq3{4f8scG{BV<&_uzgoF)1GLb4XZqrBnPvh-igFkWLOQITs2u>0BXv+OK@X=NB+4vhzX)vmg^msSZRYq>&VqrS3>}LSMB* zT#f+k6$2)6=ste z1;L!m4ER}G<}<<5(^HLRD~xBJ-RWa6`uh20Wo4)>I_4$n+p@ppNy~ig)rpA-7yZkY zvEm|D=B!u(3D2$AOprVyJ(bciq!f@Ph)+P^rTB|1one_hc28<*x{rSV5COs>VDa}0 zggX{S8`{987OQV|owBlC9sk;uxHz9OWih*u-8Gi)NK8>k1Hqdl~>1IiMi-sK17sZ zsDn-)TBYi>^Jg6PTn}EuQ3at<&^dA@6!tN0Q2oRDJsSuLPUR$eaNhY6=* z%gIGURvcy<+;+Xkw?rT+$xeq2+h2V0XQ!u?2=tR6RP6c4mK}4><6pk~rXBnCM8}OB zIr7yJ!&6hQBPM}|m07RTMMp=6R;+>o6&fi>8MCvqd5Ksid0E+I%b@Yi9v8LHS$zU? z^AM1tz94!P*_k<7{9M`ABRA1caQg;V+Oqq=t_znQeRkaVp~J3UO`;uyM-ee4+@Fh3 zHF|#^e>lvXtZXw_s5ekt@sed&JfQu{HkY?wv8*fyO;zLJ|RIXZ0R0No~E=Rcv6%x;!$%WI`IVtP<`RGbv zl>uuiEN!6-#2uP^-)w2ut{t6p>iBW9DC?poBb+zo(x~8yp>X?g(a~lUg5CBrCys=M zRi-# z3BX&a(q%~i*%1oIfz^oyZ2~vyLuIX0fM_BT5me=oLZt$}TEP0YgJklTf4}gXFSw9Q z&|TB@7#Asv6*k&3Bn?W_TG{(X8#Rpw^2{Z&?lDYg5Y3lh^-O`K20a&W)kk#?it ztaV7iucH6&+8)UNQ0|`gi{Kbm)u<0B8Bn)P^VzdLaAs%hkJ=f1@Mj{PWTYf-SiKS> zz*enV>3l59=FRowq%N8_+b^_IL}WwJuwX)|Jw3Quk6nUD9o@HQ%H)Z3VEwucc#hzb z*^OOGjheOME?xjMfwMZTTQ8dVfmd$ou5CY@JFurimrh{i=C$8sBwW0wRcl><|B?lt zn4H;*=FTcsF{E*1Q(hNaIFLk?^6YMBPGZ7^6Z>|)J8_)OV2P+x`^M#qA5NdBw`ZfX z0psj6dIgpZifGej{_L6V+_c?0x5OX*xkLAd^9=eQR<6!SN^a4jh0eztC*#K4ocVJ; z^A8P?Fuo#s7M#OJIasf==Ut0WIJ|%NJ2R$Hc7r;#)2>|k$FwQVoNPc>n0vj>H!vXL zzI*3<`kpH%dEfSp(ff9E>-I2eHXFWOm7I9Bc&So#+q7LUdzLNZ#_sK#FCN(2vuAho zf6Yq1fN%#*3|x&T#a=kNH|otdCbL+mSFavnCoe^xFJ7&d&dbkBhe9~rC-j5@aWf5= zf_ED7LKV5@U>LYl2+<_g?z1OAaxH;|ebBvp4T=I*UCT<#fwh9r#sZYQ1z5=lQYB*` znTryL;?q?fsD}K>Q-OQoYNcAg92%dl3Mr_imKA3XEl({urAy`IzM72EnBV@Rs?u~t zz2uT=t2ekTY=WdvR7&e4eij}D_X}?nUHpxEpkUMY)Z`Aibo zZbNQt$i0!-rDf|Uh77Uz`!8I#u}s+@o{CblOeC+`wd?HOv5oj=sGv-mI{nP~bCKcU z!$*yrG;?Ofu&Nn3xf4e`gMWInkKenW%v>;gcEhmBpMUYytU2>wNMUiA#WJN?0C?&1 zu3hiYy3OFn9>uc!v+vfHDOc{qu_HUTZS%H@(K*QgI-6TRW%~4UXD>7huldYVBVM2R zN%_!9nRz*rMvvLLeW%&m=e^l;KmFk2M%Ai+v2x`nbLJUwJfiePR1QF;!G1lhU5l1O zhCFVtTIa7_SE5YN;h*u{z0=3rTX=pJB4CvlrOco$O(SZJczVS2Pi6-Nmq+<`>ZoV8 z?bu15OrG)PnRDlxM${fT;+ZKkXW@{Vr2{fzh3%@aezS!%*xA^vI*%zKh?A%d#hwa-L*q-Crn)^f-%N)d*_Yc5o^ z@^5JMl(s9BSFa1HP(Smj7ir|q?=V_`pi?bati%_1<6v`<3~8EDrU1KS#}F_3_z!vF>1QAm+7yMK zWrGnxISM3Y%9IB3@WIV)o+C4dFfk&%krq1%#8tYC5Ex7}72IVof#&v zOyMkkmq0@JR<#r$ErZ01teGSihFe*Lq7#X*NCe%`QAsTM(+b{_;?TNLg;5q2kb#oA zF~BTjBOqRsK>k388Uv=q(bfS1T8vl=;xDwGqEz z>^e{rOrJ0vle%bpJ7iCCmqS|$2qja6a@O9JXMgHYtu7!ETJEw}jDdbqN>)}{>O)OJ zO6rKZrkeSuaxO~zf9e4l(Yxvp^=QFg5wv2JfK!fQNM^n8rY2B;Y}snHDq*kA24I?f zeA!+z0wSQ3a9gN&GoNrq0TPH&94aVEty{oMB9;+lSp1*hX$~>&gvnDTO_|CTCsK2_ zmpQOhu-@Ve_OMgO2ksZ(4@$$wqQQ+1juOG;b6Im+*s&DREz=vxH2{iDB2S4AKviAh z#|}$wuVQ7&weQkZgwtTIQXLmf3@=ljOIJE17#AUeAh7+o^7IeD+>;$uQSGppiRdCZ_Z|$n8%rHv=l`{ks){$FBR0O^Fw4yla+cRMId<>d2Gp`Ah$_e z;?a?C$bNx=w1}CE)IHCwD-m4IZSsZzR}yRWePJwux(K_oXqT8wxgLWJ-AXSrA%>^E z@zx{5hF7RmS^7}?V4#U^#Q6z$O+O%{C#*fh5}&%-hbn6tF+Ks^7%mXhjc$zSfGdtz zal6WGFGHZ{6{Cjk6fz}R%QYy>o6uPNjR3X+mx7)`1!w|8qh$R6G73T5SGY{A)V5(E zY716tN{QknQQpe`sgRMr<$aUHSLmJ}nDPa(MESajT>J#g1?+)Ob(Z|17#1?EpD;g-?J z`+mc^Yw0Ni20hA#93JwR6XJzKS)eeN*eDZEalix^4#c?JM-LoWxqMl}h7Ef6>xVY? ziqF4j8rj&VM5*hkX?+I_0@a<4Yz$kB-T@#Z2dDvYAP*++g>$FBTeV{RE3epbm(1Ry zaPL${ocJLThHJ+)3K19V%8{WUj^YGC1c~Uv)D(Wsfzlp|A2>l0nxfKhA0MDr*l(zd zEC*D1OlD#GRGXRcN|lgz&{q+kPhtXYz{WlET1tz1@691@1f?nVUW6M0@MJE;2E@ep z8Lx6Q2U>$`X>7>>uy8j zNCxK*oiKY7n;wlz>0xS3_AKMB~GQm4um4 zF0iJp9$kL)Bj{6UF-G3&Ld{1}Pp%&ued`V%MnqvSB zY>&rv;@HuhTesqN3c&$#5!jmySC6Ao+Ya@Gf4J}{3M%8M#XByW1 zq0o@v(9nvf4sO!3AO&gFb?}B5=`NNl1u3a}uY6qAkw+cR78`X6Zf#I_}^eP#@_X+`ikIhK~y?QJxS?Gar;|5hF^+vTBC8kuWhR zEF<5A&jv=A5h!~bP{k8QNfhHqpx8av&pWmmi}{Wj`&^^Q#(Q>dGnEV&`RwQ>E$%hC z?8su_{gg!_5XpolF1av(@KcnC!zLsc%t&z^X`UoVp*5050wVrDY4lBYPe5FblmIF3p%*)(9PBQ6KE`Hm=D0EkIXOPKIr z89qe~Pzu!e6vND$3U^pS*z@eKjem8)vSkk1Zp$3pzIDv$!)AlSq_eMIz2coI<1D&7 z(LMw=U8o_7gDY4?&o*ldiZXq1tEsbE>ad- zEATeCB^X+j*NIS?5XO8rov)(iA z)mK(8S$sJzM&5$Z(R_vPptlf;Ulo0GHHxnP<~`swY0{*>c@zFc^@&#W*Ihr4{XXvxO)>-_zEDwi++ z!`ihcjvlR9r#9dV0a&_d9_;GIwQDXMJ;XK&qW}Ov07*naRFvm-+{n(ZQmwkdma}5n z(q#)5$3{ojsZ|$v!ZGWf9ov>ITCj7+PQ1C6EL8@?WBHz$mOA&-nIV-b7x(weN=aHc zZ%)d`AwTN{AA__0Y2Vee(^adcJboH<0p<+sZwRZC+{BIy(=@%mXw&3 zot3rns}(V)Ph{ui`uY`T&-47r;|u0|_S2?K-d6AOAr*j<)k_z;Y_=Ucwtl&M1PFODw{Fd%g$o$#$`vaBeKD~y%a?!Nv~lCO z*bB6`Wbwd-^X48qe4tw8O1}PqWC#kv)e1k(=OJhJo~@fs96o$GAz|_S`N>z}>(!1} zw`R@qFP0GrBCJv*kOyycJ#ph z0mGhAAE-z<1b+oh86VH8&zCrJb2k0>!|Ih^n+=`{<;pHvuwd=lHKhXl%LkXk{52^l zar>4Zmn~k9dE-WC#qtC(!28X~<45a8)G&EmJAT@{BWhQR7A>>WQnzp4hEn|Q-Fw$> z{Ju%!CeR_-EznTAfKEsq)b-6b{>99>{c--w`B_8ZXis*Ioj?`g(}G#sUs3iz{pmls4u|R!(I&^@|e?UPl%6wws()16UTJf>39cpuj{=7?H_%a4n64RYdOkAHHAz?(}Iv#rzLOZGEC&cUc&K ztnm6oi6nGAyNiq-xOO=~mv)`V<@EgisI8Mn4PUfyo{z8f!&jbvqHp&V3+9o->w_QJ zu<|RD&H+!l^qn`U?X~!ei>FU{;hAAR2G`a#tB3XL37;y$CB4oLc(V**#h{8ivTy(7 zkxwt3zaY@Z`u-~~5ANIN%O#5gtlqCa@%Xw`U!fs$WZ%BQ9ojjvGyTk_F$4OZJ$Vct zW3y+zH|3dO@ZHH*F1<5pg282DG+2&sz{QIuyULuOUoTra?6F6p&z#M=k^20gfrIv*Zf54tu3au)xWE$Xqxb&t)#ppYLqg`yeE);#(@?(&sSy0Z3(ucDc9h7IFFiW2 zN`+v=GEJLA9zArJYRAXL{!P`I{yuO0Q9-L4%h@) zQoG$a*2s#FedxI{_jm99*pR_|)GoFHbF#i(zQW?~8&E88>Eb1m#*YJgKpjozj0pi$ zBg<&sv}t-=Y|Mq|vCofXs#x{53m2ltj2S~DgCAs9h+QHN!pF}2`^=a!ksnvCT-yBO zkJW0_mX)(?LXrn70nh8NjjPw88%yMeJ9gZ$ZSzx)K00y2ge{wX?EFxtwQE*YZQQ6$ zo7TZWeWfIhv=#Y*IA zK~k8hBCl|D>(Vjmr)`h+>b`yR4`o9`qPA`6_HftzyLY#5dmp5QS`6sb^QCdGyr-&0Cwr(xu~7n+q7>l@&*yDcm@vYyKl#)u&}VPV@BHy`VZfGt696Y z^XJcJDLft>(RRx zLym-+wT_nvr=8QMOm5V${zGls^zG7_2*Slnl{T0~@OOi0s?cAlXmae-mC9`_x|xthkkwEnK2zSPD)JNvSm|PIJ_#6C4Q+vv=jJ#YSmH{NX0sA1>FAA4i!XCSa_!tXv^o3r z?$dYRAY^{*eBez%ZO7bQQ~R!J?On%lBV&~SPFU%cD&2F`v#*XF`}Ckk&+Y%&NHhY2 zwbp%YJpTUKI`8TcjWgq}dLd&(E{ibSjU72IOhk{)gIb;kA53Pa-r^!Qq?b1_vv9hX zW6%g|P_MFr$+qX}c@%Gm5H@b2& z4*#^FUE5YPP3Q5h-}-?pqc6L9`~*zUxh7^NqUb!cIa4A`;;YX>w}9G6ZciRP^8NCa zwJV0zu2?>5!@9ju`@I}yx0g@zHtlna-g$a+y}FI#&L`;fKJ3!5_5k;tI&Ztd&!P9r zfn@2O=a21c@<4}lqcy`jz^`JMcliqO(Xj}|-FjjVSw%4Jc3q0O@W_CH^{ZBW>DlMO z+su>%Gug=NqIv;l%hE@$PmGJJUbzaJ33wtz)v|um77nvD+hDFzvq`OH?e!(fWqMh| zYSg`$kcd^N+T8=SMQBc^g*Uu6+bgh2%?3K1_tk_1D;p6`hecL3qCz9{U!?b*M)x*z zn*8lvzG1a%)oj^97g!?8=u^FV^(&Vy8|=1ohYo+g;zm?_9ff_s*!~ONmhS zd+uq49~YL>OtQj4FQ1Cj!;o=4DA|j2N498@t~c9#idC&!r}=$tbM%&6 zop0S*)o&zSvJeP(>reOHcVA@ns*7jMv?bx4Mr;LMeBt?XCysAewd&;w6YO}h)SJo$ zmxU}PC*y|Kgd_&OfUiXuZny*f=)qt9q-90NozVk#AIB||SA-=-kKvVMu?}tb4d{0xCFOyR9YsS;G`p12n88rBYIQ2NW$zw~w;8Z=n*6NZ{^)3| zTeo5T>Mz-@xR&R6`Sr$@Y*d}v@%<)OBO8-tW-6$av0vM zXdmuD&_iIi7NpYnL5GW_fDWlExK3lB8f<()2WT)Zwj0&0g-0#;WrBGbEdd~*sIQ6D z3_T*s(Lf$Mgm}$`?L%QvG?_8EMCSp$X_wm^SO%bE)6nCvArZ>Gnpg^RQ~b(GBR0v(si{ZvhrQG#tGo+xc_n$VAW+HFJ0c zR8t83N#3+U1SaUmOLe6xbf5&?k*raOq&}>2ULN|?Ywx_(yIbd|?>9X2^a#mH=RR@l zI6ITNfPh1X4z+k7h+HmTy~=o%E?b7LOLT^4yn<@ZMaLc71B$=6gWGpo)0m@75WelS z)WmI@zONV*G-}i+W>-x7d4i-mTmTWO87wwQ!?M#^x>7}Bxu7*)eUWqh>XLa21ViB0 zv11n%VKLA~q)3!^v#5apG0aO?KxEm5`yYI8?S}Q2FJC6Y@9w>OFI~NYIxs5H2*t^q zr8p9pPd|Du=lYdR8@@ZfE9(9Z9f8}3h#Dag;m?g4*|m3ng9SgYW`s6~b2g;PhJjT-msj! zbpDJjH#67a?C{V-8#iu2YQ@r@cWwij7JWPu656EsJ-Fm9Tej@EQzsl*nczK&lI(Co zt7d)n3F{5E=|sB4sNr=>kD9hP#6}u``>L!<2{}1Jeo%v>AR_LfI0#Bh{9)Q6y66rw z8XGD^C%w(MIHcQ^J`Lb0xq{X*%Ab8SGxq!i4Mtgd(0>BvkXuHYm4kQ*rxL6TL<)c_(mQC@O<9#gVL;Lq7TuvxfvP8X>E#}Xib1gpp7K%$V|O~?q{MUzM{_A^1zyHoXaMul|@GZ=6 zw_2+XWs#RM~tlBC=w~HSgaQMdl-=xEl< zh!Nq+Kwp%2k(-E1YYSF_YSuGtjArZ`iIVVe_g5G)hs4s3_f z6V+V9<)q~M>ehRGr5Z<;-Q2Az^9va-M&%`NH zVag`G`PSTpi<;K0*K@#t^VG+n-xjs2=l$&)R<9ACm^|i%m)K5(F$CRUco)A^aJWtHZBaa^ zLerYHzgfSbIEh#9+42Oo$(}3KG10xF>&1MqOTp&fR)cs8Y3Wx1LQR>dcxmm+)Yiqz)hR!oGb69&CQ^u%RQn z^yr0cBZ{F1_U+rgZ7ciCtS3avS{QEmghcQo#!zqpLNKW?kFYppl~=A(r3VHKe7sxF z`|oWTRH+K?a8X>GKI1Lc89n;+@7ZTi>9XagPM?8?f*n!2>$9%X8@~JQyYVl)AUuIA z7yizeR8;pr&;zV973c8>lKvf&(6(RsDv^aMh=u4HmQd$Pxq1ag36D!4juTlmD&9Jb z-DR=Q+8GYiB>q!9EbMKl<@$KT>T;W2L)I*NWgbzd20KJ+8>Y^!W1~{m)QCu#{CZEO zL`Corr9tih3@p2uoDLMGmB3p^^a6q^n2>m&YvR}qmOR{hh`tJO z2t65Tsd$(&8rf4s5aLQsO(}!R5tiT-Z_wxF=D_m?1V{t}J46GGLzeeUV2^?QEnz>E z3b05m6cTBg%W_kB@St9>o|@h|D+x->%_e%g$zt^tk-o>7o}M01vNVmx{4pmx3!w&G z0S^psh%}b2vr!LtNo;c@_=DgPf*|MywS(am-NekS0RLiqY=qOX=h-sTGE0^zjjlP8 z8SF-+tDU;+>|8&V7j$DD{JqWUgKY}V1`y-0m4|LO)QH87+MtlQlb)4dFJHD`!Rl4t z8hm_37e>SmbgvS)pdByFwhT7(QV69mqNu^sT||Zysk3nD9WtO_7b3>kgJzNLi^ zPE?lIwxI9PbBCJI9Q?YASDP!B6Z^Jr_4W5Z5dIBnmXVc(n;iq8QI$a}ZHI1156DaT zOAJLFVt@*BeD7Y_%eghXrPNflx>&BK4qb z-$1?5Y9p*O@aRNZ!nyZaZP{o zt^UJ?hgJ#$*=d!Ze;}j*)E14&qT`P4fxC|5X4%XXAnZ3gSQeKjH#03JF1ljn%2q$z zBni^7ZRq83Vt}J3I8CtK`~s94e4uLP@X7*TvR$Xo|EU7qaJ)T@^h zs#XcJ`S^Gl%*Zo^%b^rRq8aHqnd$IVe*OW(mPA-#cQ7$UAd7*!Y@m^g&XAgz^zG-1 z=gfR>=+ndc4jKUZAaXHc?FSD98=RM$aN&IAiXkrI+TeXkP(}isGmouay1cCPOEEFQ z!4)jUOMu9E@Z30j6?RkS%DPT89I9EYSTV3qr1AzVG-Wl-;sddY>`4MlZdZD03fFMV z$bG53#oON-9vObwnRDU7h4LX)@IZ@=rXaOgG8M}E1QZubNl;s! zxseN?s+f_LPX84zUK}S@=%@fU7>gp<1-TIEAmR`pcG$AhQ$_6(h8>7Snql=NKo2Tz zkk9xF=R!g&n*IFfQ2+^yX0Za0i~e3t#-%f-Duq>X`v$s9RvN4LQPGvQrKj3)5}?ze z6v%3Zo&btd@Xi<%1Pvetn*+5#s0-^WY!2l`L3YlR$)E@=QcE*%ofQRQdtsp1u%zCKyZUiUtVSquZi&K8f)6A_NX0H@zUt zv5-F=bD+Fv7WyXyOU^=UVBlq=RW0vG;y}rUZBN8|iZCO5Be5_5P-ZDY&ILyi$6e49 zcgp{F9mmc50HkCtxLxxW%xw`7@z`UJw~44Zaojiz(r_F)d)6mCyL9f{xXGELM`7c{ zGF|W46Sb{nc#T25d$x&)So_UN#KUk)B92f_^Jabg-hFjy4Cv9VNrU?M<#0JM$x)L~ zxK7=qSI2d19ND2!gBI1RyfAu1PU>}UR{vs(E#g}3Lqd1Ef1jQW8`an4+2?%tal59? z+Bc5u7}?C+()`=YkDs9L3euO7`J!oOL*ob|HkZ%_vj0_b*b-P|I)TECt>Th*yW z-B_%PKm~~`;ONm!9{)B`Q88>4hVZSkKa3?@)`VDG+;jT?7} zY}CGfWXFd5HGFIO6pYdL@7md{N|ix9y4_c+`o`~8({;l3(rmpRlaQ#bo151R@7L|& zwzVSGNzqCa0eZ0d_qZlbdc8}NMx7!XG_M>oX5`a(2!)g~4QcpMvi?8bd5bN@pLg$WS*7wo+Ep{0%#mwB4}e0b4A$W&Xt%6ht^a_2%_Ab# zfAdKa7UdKCgTFrh!A1=pY}AOJ8T-uBx#>5cd=GW*vU~eB)Ek`+Cv6uy6(zT3 zfEChGeW@6oqp**m6;F+#g2Kv4(R8v0#u+%R6F4I?k-Tsp& zo3?55#-#B;jBF7h8-=Z8KI%OW^zOa+#t85U$14W?XhuNW8$LDUEI9?5OcamkKXoN*RxLzeSY$!pHH70Ipd9YM?d4qOv5=8 zD{jEi?aUoJZ18i_CLTC_bkvMV)5ea>%e(<1DM^Mx0k`|XUVVN#dGga$tD?SL{q@pC zG)SJ}WdmG6OAq)m=o;SJWXrkeP0_Kt&&F@qf0&x}?$*gGFJsJ*N1mNF?cmw7V<(TF zHs(QEym7MLNoKHMEyV~-P@>9|!i9eN-mLKP$<4)^= zyKYd`Y!Xo>%lx4upD7<&3DIYG&06MCB~PC?X|Y&l&ibTJpFYeHcIRMlP_o97O_!a~ zr(a)W2)%muXJQ>Xuon)DN$znv)_$|PW{cLf>NP|R+W+w<&PPRMrKRyeG^3f!#OlQl zw#h5d-=|q*!%DU4TuZzvZj!)tnRnt!Y5kgS%LSD|ivx8955i@i056Nrx^?S!x160m)(RgDglwfo_D$@X0Lujd*6UdRwgK_3PJ`u2};e z@ZCGM>THrZY7c$-^+p76aNk}s)_A&I+qda5(|h(CXmjei_v(qL;O7JTgr#KELMxrVli4R=%WYMs1ezq{{FqYX;<%F zJ@60SyKf(gKCmYWP(o7$=67$@PF+TN-@bhj)A#7z51=}*YZti4ph%yhnqf2`Ql{vp zjUy}9jYv#P1f1_{*S1t(vGw15D|iTaOJcg>x2J$$UI8d`@#{lls`Ky4P|~iO53sf1 z5R{ca4wd1rQc_!R|35#0B^q z4sisCWeY|TF&EC+uU(J4_Z|^Vd+BqrM>ptfNIwl;7K7``rMNP+B5>E?GMK$B#xhmI zu3nB4ejFL4$DNX#T)k#39$}4Lu}V0s?6u^SiX}=5udb#e++R{c+}_RWc17)pi;aHo z{h5#hJ?^KOnfRRYxQ-n;+yb|PCJZuR&xH{K*%mBWI(+C;@T8Y6#Ru1`tG8HP{ImL$ zs~U#1zj7(4c7)Z>Kf`VC^$jdhsZvVPRhlH?0;2(EhdNho;-$nrTQ=<6 zu`4zv=A92dVbj=UGDJ3Oc_Mx8NjQ zs~*cq19VMiLx#EltTkxXFc@B5W6~vk0Wed9h`I|7$zNS7Ar=qviy(j0#&0fB(EQd? zG<+|(SCHdx{{M#({pOoRH~;5);I0F1+1nDw6-*AQAu{Y zeJy60#|R3+QJp?5Elqr4usRlPHUp8sF#1!^Wlf%(oD@>2s_6cDq0jBh2xBS38N3E# zcuV-f0|(Auj82P>uiLbRQjZgvrp7%Hl>OZKGfkQf0>{N_MZC`HqPA_%h>z>|@WV{$ zwDcQRbnoB~Maxj)=A>n$i+mguaC%ZwQUU`55&I&4z}nFtd;RQeczn@yL1Lm7`&ijs zycnAj7hkJc3j!}tH!5#bQF5S7ny43TZX?PudQa5$O>Dw;>DG<-Yw78k!~0wnvE4?ytViuAG*>z)?k7Fs())s_!8-0BOurMU_J5K;HR3(ZBloIr3 zFV0Kv@9&#u%Rw=YJ2rHjb&ldC{Ndus06`?a08#hu(yiy_-AA^bjw=&f{@F1j$RCve z)LP(4F?)`QzECc>B2gE>UieOuhXSvWEL-3 z+UCf!yAT%}K5)NNzbf$e8vf-8i)IQCg| zW)P7pIkm8MB3f0uV=(b-eSNj!%ZDF*3@#Ra-OtD0mW%5?q_SSXF3?$`Y;d;di5v0M z=j~^)JF;|?kDUuDANKsn(WnIC;|~o= z5`k8@e`V_pU1tQqJ_h}=g$o{fVyMx#n8EDh?Gu=tlZoyoBL)4-$jmKPytHU<3mNeW z^eL8;od*d4cH!2uvUB~5l|m*Y`=&HVw4%K_Jlu8Do`YLXohuzw`nfUBun`NWhXz-Q zkGaIaC~w`+BmVL~eBOZ(<}V*Q6`E@xt5S#oC4W?#zgv!ipDH)WP|E?LAap_f2}pBM zATp8H)qPT`2FeY}V&TD}OMd+7C@A9gqJN`4+Pl0`_^0kb@%Z~y#zjB>rakamU&-IJ zDSxcmDp(g#AeiOO${g|dquxgSoKHWNJ$fcRvI-*CCTZn zPrZ?HBQ7qqa%JWn))WqxJ0v9Z%!%X34hS?BfBrPG7NYTjCZMV)RTDnZh2zro>#T7+M)b!`B^0P*8C4@e_`mY+sWBmBjSexJqHwF_f45sRQ_+Ru(`wrWx(p zwM#sAIwd*LjD@?>Y}3G%5J9qtBGX6{w z638cnH2g^;6#W2HX_0&n>Q7o%cxc#H@Kb}me5@BXSsoOEyN02hHLhF(S?y`zBhqVS z6=W&@Ybh1POX2zX%Y~vV@|nnSQ?~i*0PRKXY0>$v>VaFn(z~JMXbZETh84ACWjy`( zz?{^i*IpZkPoKoNXlG`c!EVpGcI|rNRg`sek`ps8U&2hTWy_X2|9~Y6=6gAuOBc@b z3#rhcaih%ily6skNyw4TogO~Db?fncJKR|*3unF4y5FN_j7PLBeqdO{2yS~$=G6Bk>6Zuda35~Yv?i+v-zcwWZp;Wgr7W56=fVu-Tri=~wt*2AL{0(W4q`8_Rk zKEBJBF4EgGm(Baow^H?b4IVz?b*ZxwHL4D%EcgiMJHO zZ)^M=|7T~UUX4AsbMuBzXMNPXLx<9(OCSDu-@n0UsP|{sCq;l6eQ5Gzqe&`ov(HAym|iaoaJ*r^$V^L8QC~HEqUEHU*Z^` zIllV*d{-=)Yj$VM|Lnbh(9p(Bn!(MBpF#LsqUspD2tA$@8@+Yoy4kbmG;ZIiM9H$) z;>VmnSGQJ#_MM9yOKA-Oi0JpRzJl9U3njG*5&l%Lvha_$xOTF$fFPs*OlgHC^j68p zkTinTtYi4YVFGu6SI)?}ctkr({d@1osQlt7{5V zVjL|iEjJQqoW6EeZxo&Gs2;fcWZdj+!HhXw@iEcI_C_IF?b`l+nGRl_$sf+@@=)Ig z8b#_{Ik1c4M~#GEeP{XlcCA}}JoBSb!=IWpV}^Hdg}L+Rfh=cEpL%QTnELf18#HJ% z{DtQS_vnsCq=v2T9ryYqEZ1ZL!X1jVg)Kn+mNlz4fA?)bxpF;w^cwKklcGzfM9c_1 z5iZxSUw`=*UyL3zmRcZ=_;T5@$!|<$I+FvrV1V=SXP=FFa>$2oO!g^TcJ{(=v4RG9 zXFNN$cEe`XDu>Pb^s{G&J@vsGQv-u5&Yr*6X!6d5m!+8k8Q_s?>)P+Ou3zt4HmG-x z-u)kY3^DyT-+V)mSr}t%62-_6%ML)pVOzOy-nbd>qCDd!G#a~t-ahYtI&18(A)if~ z92iu7-r}YB(mHVBz>G1YBkR_wRJHoMpU!=8=p&!K^@e|$67!dQ0U1K1gqRzC8>tec zX!j2r9^CLvvGSo0b??`6@RMlKty%Sz$;V%|7eE}bVkPhd!P9J?FJ3Zr>U3Eoh}^|( zwVFSfJ$uy9Cq9}!BcNP`ISW6>;OOLuBX2%8wpQK9>S0x8e)93C;ZJ_}#zf!p73a-g zE)5nkNw>*D1dYLM>o;uMxZbB+`5xW64}AP7m~4(+yLS$L>~SGWqH+gp^OBm+Y6lRW zy>N2Nl@e<6KWSv!Sp5_M82a&wRhK3Rn=adrB$ljH2DDsms)JGBO6Zi5O>5b52F0a= zRJQ7^{PfyO1+S};)Gd{kmd3mZ*{xo$jI@kdOUu-dy>J)&3A{H2kFq7PU#h zw@&M_^!lhPx>NCq4XOqaFtA`s{Lw}WZ>Tnv5O=U+R;pAf?r43+HXwU<${d3jh7Oy} z#@^r`&9@+(8PyQGou~+}z4jWh3j|Yu2nrDI8e-}K{|L5{CL=uy6GVitzztQRbU!~o zQt}FE*g@qfVhprpcuT9cmXZV*j;lQ7jFbeK$=h$g-Jn4OY(RJy1r^Q`&|V$9%b~Rz zEN0EYo+J(DCa;rcR#vvRx3^{q!izU<+<=SMN~A0;&2N5IJnHnoeKTgvXxON+G)>5w z=p(9}HE5EeHDKZq#|`{`lqyA6Ajv`iLaQm7cS%Iij7aIyJe6AAT%v+b@KC;OK{r@n zy&gAAAAR)E`t|Eg-rkH?fzgUcb&8JP>j6<8XFL>BFIM>&@v=TscQ_+cOuLblmXg}_ zo{(b1*ZA}E>-Wkny8q96;AT4i&r|-7KP_m860a6wfZ!m9;5q0dOxK@n2B5%O-~yJ= zAR~x@DxQ`J>Lu7Eb4>z^#f;|3pU)mv4&k?X^3eQ(t=GRh) zPzbc{c&4C|f(7a*f-_jjK3<6uCAikIRow-95`z^kl^;CdGqFt z8aC!WX?R!s!*Hc|*P9aOk0!0duPlAeh3HV~b((sQmJ*W5-mF$DtwgDYYA7Nm={*5$ z;f{GTGduf@=`$uxnS!o5JpgKpW+85CijIHU16mKN#bf?gr4d4zQOLB6EZX=u<(~TV z{%KxC5C45VpqRUVUv__0PKv3@=dH9l$aH1a>UHTl-QMfEqvum1!%G|S@lS++KXYif zKvw(#lo~Q>7XXStqTrVlB&ea)<4@(!q8XSLzT4>q9t!&j`XT8AZ8-QPnjxatp-R^v zO)E=L5h;wHJVkMvL#vfm6YdwLB!4zw1^$G8l5#||L}a33xdRdS1Jr zo>~L7pITX*xruN;U(Q=l0TEu)KH?MQ+E4+cy+F2@2r;IJp>Has=<>IEfRRz!jeKV6 z^G3waHv}^SQs;Vf>FIgt$;l5js|alue=$YJf29X*pCkX3V*h=G@GWU*Df3h|b^sdz z0eiu-u_{;CPf`1mL#~mKaGzSqZ?4EuU28dTM%hE*DVd2yxE-B2 z^hlIcNpZ^LqP?4+gUTa66AjAM9r$r+CTFb_&g@2VRvn_9f43iak-EyOv_<%6SwRSi zRYgm|>d+brdEwAXp*ZqaMF@4#>Ok{VHRQhJtVXJEcTNbNmYveYrhDjNJ^>B2*;*ib z-YPo&RS(E#B>(8%f{_Bj31%UpNg=3gkjMsYQD6S6p8R({|I^VgfLy1a+KV)A@R-;T7L2&*k2%KY+O9>X(cH|my zwk-Aqz%^!Bq{gI{`}vs9wyPJ>R|EhhS02MGsZ2%0EW9W7HPRxqv03)YZk-Sf$y4nY zAQBgfBpa4I6z*K6u-xKW&h#q9i4>VFP1I>b5P@si{0+Ka_Xfj3Uy&ORXfXmPR4O%b zISna1q-OBul34gWlwL&CB|WSxsH9}bFkCbKP{rF9_esUJsKym)`DiL4T&ot)arBvV zpWDIeUSW~k;>|zwCEPxCcgG#omvqvtz|0mmv96BOm$vVa#*H32M{M-KIBbNM&<%U zTU#_*9 z@Q_EcImuHFt!~5-6F{T+%6|xHM1Zail~S!xw9P$X^QA?SsXA-zExf~1Rmc3NTK;Ss zlBXv6=YlkpUYJASyV@N2-vXDCO@% zgz3U=SDjXL`p@>j-=5>vW}b8sS2urOYF=_1J`H_{%Ef)sQnCX|6Vm0jZ}7Gr)=)#c z&HNU)RJwgAy72>e7JP$iRf})6bauj;FR z{XJ-z+?{7_SZF(BQY4Y;JC{Mg*$p9&47#+;+-l7$0^Axiqn0msXIK4KTP^edzgpnG zy)adMfPh2eu=1~k5f*?0iEz(>Df)`d$3nwv<822a7NWlo(L2we7mXU zq|}C2A)V9|?4vBs8vIF)u&7?x8>yK|qf|?O)vm%eX|zT1pQZZe8Kw5t&*j7?k*%Qa(QL;z3dgDQN+3NV%v*O7~ahJ^JzxKEtI5g0a&;Mp9 zDX8Vmlc(U`o$$Xl5E`U{uo}qT0_cCHmD)F;^T;M@?TGx8lbX zhB6oPZk;YFQJ0oxLolb-_@bM}^k${4tT?n*Xgyi*y7ZnhkSw@S z^#32|0X8AEsexEY7O^@lci)PMf_`2)4S^OM{m zz^x6GnzQ-1#`hy1lUlr?IRIPE#N|)CnyQ1uD!u&p3qQ7-i!Q9sKdI8qX;eG@^+~!y zweb(0|8!q9f>gImdvc37~|Dm9} z>J_c`3VTfZNms*YnZ7fJ^!jXn+l}I$%)tD$)UQYCcQVeueQSUIv!2vm_)oX@pV#((|0(;pYO<(N zD41;8NU6%>_m~|@xa~hTO#k~0{~xd6-yXR0VLHE?wIwo&fbuN~7K@UYJY&VjHn?a? zh&u59WA8iwtthVlzwNeHc=X;B6nn>t*s!5u!yYwOj3w3>OJa@Ps4@0fK}1Cn>Agrt zL`5t}mEL*%mc6_8|2Z@FUceZCn!k_0dplhA?at0lIXnBEGiS~@vpMK#Afl42RCV*6 zZFIKPfRVk}dL;UBMYek4$2b4&`qxWwe_j6nR^a~;@iX<|?SKeac!CtXCMECw)$e7i z+HWB%*%HuobWS9-)f4|m?d#xeJ|)qTS8DW1!w1Ln*b``eiHWRbMWny#_X;fo0N*B^ zWQPattQ0^IOF&#cLoF8(`w%ft!Hgu$uHMdOO#;o2D?}H_*3W8y*B8O@e?b8TQChv zv0^~#D7=tx;OyXpRg)fHp(vl0sNQZhpUr0yocz`ls3pMt6JKh!d+|qXN|?g&C;KI9 zM2D0jn%luRZlyl{Tla$PAV1An1K;@*^?u~=U)SzGJ-Hw&aQ8P9!`IR`4W6@CfDS7d_8z4c$ zF^5Q{iTO_-o2359pdZ-kF~MRYvfpG*sLeKA60*mw9#A*7-@}en*Z-S}0s`vgZ^Z@U zD-cUf?4?+}6kug~Bp^=0?8y3mXS_v{%^jDUt2h3 zWTBKN`6!W(DJMgD;e=pz7G-~J-a+~ukpe%VEH3u-d}tZdO)KH4c!1_B+!p@^sNt(2w|tI@uyP@TBJ34kGR85) z;{RSx%YtL=q<}0L%S=mp1?~o75oM$wsj~>wsF5HG;k6I9x5Wk^>^Y+GS$L;v!zeY3uhtq%S5@K6KJ6<< z(K!!@zQA-r$rrYR08eMAHvBo)W zwQeAKd1=e6GR(EU@#fGkLJeAl3mf3~KI5=d$%I)f??tyrYZu8PgEpcGj|92EfdH%9x>%L!7 zxo^8>hqo(W5z?VMso9%w1j6vCzqti)by*7!H4fY&w-g|@@Xxo{Fa}Os(y`M%cF1Dg zS8Z#cCNqqT1SQkVtyl(A<624q^};t#u7op^tqMh)jWz&7d`XdQB-7ltqgG{xsiD5LEyLfRdy3;<*jC$Qr=--FG@fe}#7I(Dt3L(s$jI||T8&?ZS+M)USNeKKcZgQGfh3gdu?ZiR$I z`VoiP4cYRwY=bs>hU3bbq_MorNO|dKOq|yEKs2x|q7lYQ(d=fr!;9p{b#Sc36kt0Z zR7cXL?XhRENKLYDTK#FU2q>^M1!U(CVwC5rNaIFd%r665)KX-Q zB|t?@Nx4CA&3Y@Kt8IsEF*rd!h|3m7>|nrW3Y&!zJ)bOPXJt~Lco6Ca1@f<2kI+cW zB!-L4p%#B`n;RLN{dW{F*^}DGXPl7+oeS!%uddGa9hNAwtCZjhDdeCf#6s>DF9Gse+sbWyUV|K z#&h_k3|zMu+6saPJt3gLmJ|@_nu2N>?;}=eB8`|;Aqs#YKS5jpars;0=q82yZ$gbX zP02rG?89#m8kaEI%yjr=sVd|bdaG$Ob7ZVCgb=~8)>1(2)~tGkf@Tx3kiawTRdZqi zb;Kr~jOIa42q^Gl3NTq`EsRbBhOltW=GUZP8o!ylR*6i*81uS{rJdR0mgcav}DST zq#KEd7LF*tHpFpBF9KVH53kjs#(~l_#48^^W#cSEr7%`}l8O(PFOxIHf-!T34X@SI z4jL!l6)zp@8-5y^wP*y=&;htjcX0f)EqVjB|MLqv#IL1*GXD?_QhtDi*zM_{&{}$z zbR-E1yK61eUE{#5smjr|Z9gqvBP^9H0B6TU5l- z1uO_n`7abuq)@HNCeXLID>WCLvk z;YyJI=x0GU8z030b;h>vpT&$5ttvGvAq$FVPc0-BlvZ1U2{6gm$Z9?=G}%&uf-_($ zer;u7@LVZW;k6VkSj&Jy5R8?YY!S$Y%BlLPSUFSpPD7=R6hsu3;vqZ=0Kq0(vm$a3 z?En%o@CaygPz7mZ%(hhVxl$AEX4H;=GSWz8#8aQ`GDvMMRyBtQRSfZROa0UUnXNRH z&(~ilkh|n0AGXXhA6dswYVC+r$KPwv{9j6eAEAOUq+o_Hwdg@+L*fRjQnmZNP{xkeb2R5(Rxtoq2)S#}%1 zm*Rz;^%)XOpV>$z6S72`7T%X+_*BxUiV+I;Ro3!H0+>pPQno;<)LeqJNU!|!h!!$< zX(JTI!GH`Kwuw^;wQme|6-)~#IfI?EXFFogtTYCEys4%c$$W#FkfUiiNtc!*0ab&I z8d~g}ZyEKFw2H_1gMOzx!zoJoav}lZ(HP;ARaN<;5nNsfiL!968==QJ{iw=GOsZfk<}N1@*u<`QB3Lm|m$@g$j6S1XnnVt7AMJlMiKwnD8Tlp-9RSSQp3j zWP*Y%`Tr;gubPFs9JJ;Bl9CZazz<&8HeBQ=mJtdWR4g2JxaGU)kYzC*fTQHBei_;T zs6bc0NLd*NSp+i$0!E8c1iZN_2q;(RLAi`2GQ$v57TDb?=xRy4{8UvGC<|(*z%Q+;#ND2hG+LP}BC!=#ZPCTOQV68hH`Crj1Bg zM62M5ZHGWQ;c4N1m9}}rbR945dud|jaqtx=*8*`cFm2e5Ft+&o_dVZ9r%Nn9oJfRV zI5iv#B8z-N|40qD9YQ}Dp*dLzkBj8OOe(a6y9d@}Nk|0Dg7*l~)q~O+%9S5TYkGoz zrUPCf$04q354@B%SuIxZ(DO&Dw};IwQ|#yVJEBLblu8|1>eovuwhgCj=MD5xdVde0=Qws7cjpF&H-VK z+m;0=O3wOW$%deCPfmz6sXGr!Y&MPQ;i$cmm$KUxzZd)BQVr+y)nawe)Hwbz4zUx+vzkL)<7U^v?ta8}7U zgfo{Sr2^b%#WD-U{3Q+y7h#bWG)YZ4S0(+ZS%T(@QaVlDJ_f*Dx8HxAZ@tD4!4!d& zY;ZE}*R!TP)awd(&gTXWYSCe@l|L-Hq5F}BA3yhjmo7U0obnB;uQ|FaaCGy-56tTS zDguSF3ofbOq?M_J!sLR`0B}<911AgKSi%@*(i>JUpV_|;VufpN{ZkSFHGI8-DUKR^ zLuG64zUz)fpG*)|n+)vex%WMI_F3mG`)cln4IA3*xpz^c#w-gW*$^(4B*+)YUIvL= z?WIXd3=X_2gw%MarO8YbSVgr_*{N9n#y=h()2Fv=BZG(pGpfK0f4k$&YJe^%!y&Q;`r{TnNG~KxG6HFFw+F}4B1huZ1E~D5)kyM+|w{k z6NpDXo<(q}`If_so-yParn1Og)bYOoaM$q;5Hx=?Uk8YQDS{*!<6OnzG)BctRut9=}l5egu^`fH;JV_vI)5wrJWXcEh6AUwd=q`b{g=m$%t% zuZZv7c1{m!_NV7wIIP=ADL~RgCl1Hu88%&}5YH4Dd)UhYpvWLVVs_w0Ok*GnaY@X; zK=~;vB;xyY;?6(cI)CygtEAwr2Or<5(;l;D&ak4fNEn3|#)A|@0Nbf|5-)Hb@@^5; zhkVx&>4q2a;{|@oPN(pk9#PT~Kar~E*qIbMk6~fc(QK6Bpe^GCqZl<5$XAfM$(5*p(kzA4vZoCjW~uWmi!DGO;Pw_DYXIC^5F?!A*H)F~tPDydPEU|dpC^PxMS2JeB8a63u+aX-s%C(EZ#Cr+hML!my93Hmu$I(4%{teSS&noo&;cH*3(jvF?3$OrEysw&&JZx<`6->B`*7hZfBE1Ofs zj~>{scj<=pyX|wpMHgSzpyS@3O_?z7(}}j9nm2D=JmnpA!l}pi=xG)<5oTN{r3U~R zQDt6I_qym$wm1OppCn5VSyw2aC+5P<2`*@HgV?>KjL;EBcl&n42h+xkOqOl@e)1=V zR|)fK_^J7GKc6viEbQSCM;&8^V$AM2A2)JXapR^xd_Hr=(1CAE`4qLfD^5Sj*tpIt zDFIT)_3J(E-G0vw9T(MRe=PB3{NBA^AMn&;M#gR4s>P&%?@t;z;=>7(Pv~*>oT+zz zF@EB`_uhBq4YxFD*+!{~GqAv0;}}`QB9q3Cd*#W;jZ}G)2K9!${?deDL;4LL|K*oo z-t+qlP=6_G({|R_QHIN2fp(AcFPuAn-h&sP%haLOE*DtAgjG?>+&7030HFW0Kaku~dsV?1)zxtXhSARLp zNT-Thw5V9If;CHf|3kt?`j(q-STt=+IA%wk(7-;geYT?XsmCAvu+OV~9(xpWHr%Pr z^ueDB-CDJ~?1BsW_I~x9Xa4~Wo3v~(WxzXAfcEM0=FFHp>W$|OHy&-;Dp6Lwh_WKd){HPZOsGTQBAnebRA~>mw6v%PhKsGW5t`uJj4Kxo7bN_e&n)6^Wqy;AAH!MX~fJhg|^-E4}W}r#AvMl3_m${#-t(dyk%@! zjnQG>F|$9OI6K;;C5kD7`oEV+vY3Y~!S~8b2R#0W;a1-Hmpk77puemGx~X|ze0Ip` z7hLt_3O>(+bt=w5$&TvqaoWHgg{^SNhZ4I7-)CTwEMKr7WJOxE?%&j^YGGLd)l~{8rRk zh&5_9aqOsO&6^(9^|*tMIw|ZWCXF3oBFHT+*k$)UL{|>L&zSoi*2Q%q9rxR}Y1`K8 zR;*mQaDGw4dPg03Sf-%FwWD2*IHI_?Xvz2Ag~fV>!N6L(a+Rzt`k6=Wxwq>gS|pa|rVArssx3bE3SxWiwGZ;5eGfa*EnT;2-q%YPelLpO z1qB^;-Q6>tE=L~KxJ3)>ffg^AC#^8evrj$!cjsJW*afs`#^k9>zy7LX`Lcr#I~?JE z=fe*rPjtdpEn6n7@H3D8<=$fs>Eb2W%FvrVYbI(p4O+F?=kO!#l13f&*gIXiv10K; z7CX0uX_*qqP{c0?LV+JoMgpn?6sS`aP=eHKi$zgGY29&++O(Uoj_lTb%z$_1fAdYK zsQv**9M^j1-QIcfo^sP_(zad40}jvFEGHt7;33lqM_?M!_lUDVX*xxPCCRG9nq@0Q z#Nj8GFI$}P-Db@iq4q3U8&1nst%dbAty};0aK{4wcJ5zr$`B zI}$O?bX7%RLDWbk4cAL-+!SuyY~_m8c0p03uwJtk%?(rnGhV!+ys)rj-gn=u->?=i zMT1baBJ|I9`#S{3!A8`rIE-@fg_dEcO%gPM4=HthuEZR^_mAK2@l zgGqq-Phm;(zOTPxg=35dDHIf8{6TDGEyP|(Xx#{TRZ!TiIPhFdUH%(z+;#cw2W`(& z0A)9kb8C?|vEshNwmNq?s$%te1iNl|`F@9WZMo}S(fW-HE3$6Y`UAThgU=BkpfQ>2 zJ88U2nT~76yl`PW;~aYAF>Y1zyKfeJ{C=OIAAj)O=W{C8Z|d6ZIu zo6Y(1t3*Xb>H77Vs_GWaTlMYR7yC-gW71CasBR}DU4OuUffGgyd-cWV^r{}h>ux8X zQdyZCGjZz1^2)hi&l^2y+Tu0qx}AP@(H?suz3Fwy`D2F;nmuXU(+}VG`YSK(vfEyW zEMSYDdFGkP6DACPx1SfScU+IN_Bo`B0JP`7_S$Rx-+OoM*WVeH@w6T3xX*q@QHc?b zjT|$+tU5V*;>7XOKds-Qb;pCdgc>z}^VL_Uj2k<+Uq2%r2d|xig3c$N9BI~K=%^88 zm6gLMOd2_PCN)g@R?>8=Sdq&bD15yGJ=sf|DT9Rk8&Q&y0Iw8LAWQad@F$=^-K0Pk zAF_>ytrNvGO#Qqx(9s@r1F4;8{eAW?XwuYm!h0Tfnh|bbSPk|$?6`O;16~{9Lc=Kp z!F@BFunOXKaXMUlaJQ4syY(-IQ}q6Wcm3<0yA30J?w{^EuWS$yr+K!ycGcQ12M>DWii_{P>;k0Tdv`nO%4=>;8?jR_yu_$V z4t?og=bv%*)i>V8gYUoi(!}@PGn{&cU7Ro@C!c*@VbfOeb(@~J?N96Geyt=JINLLi zIpx$=d+(oKzv-UeUGmy(w~ih%0yV;f8U4p=Z|`+tccY@>l{;?zd#~Tmc;~gxKbuk1 zq}83T_BHA?o;YaGBR5=YtX{T9*X|cye|x&P@dbB3Kr4p4{^q|PdDN(1-)P>x!ic0} z4X(WJAqKD^ufKZ7MLj=88oqpSrDIVj*-^J{ zH9fS(<^3sk3O0;M^ULOLbQ`s7jH5`n8Ck{81wCEe`U}e43;xz-3h2HkJwbN}K2&$p zx_{hr)0t=W-2JHTaXVVmtrtP6+R`E4CBbgw)=tL5g$Jt1rW-G?apqyfLk&}gT|d1^ zxkQD(jwF#^n3-6*l8-0>A1#n0N^(>i@Q-?NG$l&cEzOwW5^QrK#R)pPnT{Dr6h&OK z9tJH%MtsAnrRY4DG-~NZ3e#4k*o`MD%Stz_M5(%7v*xae453I=IIK9LjCf^LNxcSk z*hmp7D}ckIXhG5rRXQ=2NgEi+^3^N(j>Zii{PJV>VP)Tc?h?<56=dA_#!cyXyk1dp zvR-{OUSO6{$Br7+b~0YRcC~Gp^&2+Ggo~=ILbSJ|nMB#z<%XMRSigR}q&3>r=%0u1 zJ{zw{mX(IX;lkn)IASt^k#=k|R9#xWZlx8*%CynSXd@Wrf^^b~S0^`Z3a68g{q62A z=6>C3p97wL`L(n!*j%x08O(c8<7Rfapvoy^(+pM)&>2q%a+@(9x3r;6;V?pW%>Oa7 zL_w36vbnE7XpBfLW4&ioEL(cZQEeOkxR+>Acx~oN0~z;^-+6Goqfh{r16>T5Ft{|1 z>I5SXnPcjOD0Q@L04y>ZNL?~UJQB);>N$m@iHoR0BWW5NsTGo%lso&A@n20H)v`eWEq%LBU!%DBVcpM6IVEm5 zR88j1>LbA~rC}D0z_5~`z$wer_ha>qom+r|tHfQn_%nfaNh4yk0-BArP*R35rSk$S zT(7_@Y?8E{a#dTIZa5r_ih`!rPR-p+dKI#MAHjcmK;O5P&zsw6*EVaHEuB7Lv|$z< z*ZrI-qa=x-9y`tU^`Q^W$I01*D0V1eIpSRfdr!xxG751514kLgA_u~X=R#gi4!Pw; z1tD0R4En&1Ndb-2V7G>EeG&>`ui^ozOP4HZ-L`EQp)<>Nws^|kw08Bzb?ZCrvKxzk zw74=E$D#ya2DL#mY`~pxG*Yh+NJQ~Wv;Xja!sg{1A!RUDG;IvK!9Hb77)mBG;gI;u zBNbjs06mQ{zH3B7M)}(H#RcJrU6}Dwrk7f>ba|VdcN5lGd1577O2U-fcB~XA&a)?bBX4G;qqKNmV~A5R5A*KKhJv zd)@RW?KOq;(jssfS15RMm1c>PSQ$GMSL!l`sWZrkpFi$!A{6GXI$;x(u$?I{-LQJ) zidL=K6%^KktO%6QMz5+YU$tsY8>k%(%Ue+6Oj)%%VdB&YAFY(U8n?W>*Nw+_KRv-H zicUKVzj(INIa3pHSx#9DX=Y`4X$bS4a7?_O(@Mc>p_BYkVe?rorq2EiIPN<8>T6ZQ z1qAVcnl?WD`1NO>cuJgK_|_%2+}87)bJJ<}_CMdcVAL?9r~r+Q-{1b{6MLTjX75)g zJoitagqceLqY%QIt;j2i%K)zL1p!o^jE{ zaFO=N3M6NTKgJQFMLuAgY(oP#ulRVBEyR~9Rl??-0U%?rg-vMV+SQlzJk6*sH>x&u z=zQoyFT83Nz%M%)JB-&6@!<_4%Om*80w39>2E=COA&T1wX>c!s!zmO3@h8iSfH?$F zCL~Fb4jh*u2{iZLd;5)nk3M8Hs%NZUdq~goZ@%YojMZOz;UA-4c-9Dq5SjG4|Doeg zJ|)gz>6;f^bKUt@_rjnb#+jkeXFN3{pt0}zkqGh-nGy+s8I5pg4kek;xaq0GhJN(X zM=$p2=g~@Dp!87MsFKQXBqORubGcKu0|X30K!F{P0)V9A)THM5lXyJ-?BD*{>*`BK zO&oL8Er0C&apQL;PyTqqxGQhy_2!e0BvaK_Ty^EU)29uXIb+E5Nq1rh z6N?<#<%kT>DM4^8?5D%N8}c}MVJ{uc_?#f?M&X%X0eA+3ob)2N9Kmac!S2SwHcD+W zcmhej`0Ue1pM6%F_U#rfT-d5vv#!S+P0Fm!isBnVM*&0;2;lm|MHU?Psv~YS=P|bD z@)L#1^|lNT0Cg%JMZ}zTHQb9K3JkC8f+oNii^cx(_+x{|kLy2r?847JA3kw%DjZFR zqt&<&%ou2#J8nGWCL#zPJl9MnW2!C4tntelLT17kK-1E0Sc*u7aJs6TzQP>K1`8mj z3@4eYs`x(#4<0;n^z;4t%=qvETdgF7Gk%7~F8rcLf~UeA{weK=iR4Udl;oApK{ z)ESm2>$|*UvodRJXEn!pIR+M&1Fm z{1K0>U|fh?=R?48*ZD56V-i3(;w=J~GPUP_WxPp~7Vxe0>Nh|T=!NaH6TbfTzk+oL zt`F;S%&U(*xqidOCT(_t2ZSN5ms&Gm(8oufd#)WTHZ9;<_*wA!s~Ho^*R6^~3a3t; zyvOdlo^{q4!$*vsGiT2JhaTGH$YU}_)Q`tMn>2CO^eH7x8=cNvU?AYRmQQYjZH%jgf^-mSwb*#0zxZ2kWj|>eA0x9jcdb& z;mK2{bn3L**=L?Le$3dJpUpb((1W{lJ=Qjh*w$_C)R9xCPVneBsVNefYt<^$KHe-2aywufC#hpEu!c|9RuJqlXM&o$>Dv z-S^fDPw&2SYs9`~Wo7WcaHlxYOS{RX3%+Zx*PhX$V#-b67cdk1nL7FRq zZ07y0olcB=?%CU}y2?u?zxZ_eP1pV5k2hSCj#qs#eexeKx-eoH%jVC&^X$`;r5jo{ zY4r6sb3tU0db61s*&%rlfjj0h{;D+_#!dX>(Z~NDjzo_;=2!#|Hj;7QkA`J~wIKuE z|MOLsFPuNWw6yfWD=)sO*X8rS`zBFY@!%yFty{bhCx1^oc>n7!{&Tku?N%;dx@prU zXiDWoR--b3^+ipFVTrBEQm(&f$x>teh64`RFO9m1Va)t&_Pg)D2f-ZMFp|vZ@p#A&_T%khYlX}x10XZwrK;? zPb^!xBJDVP?YF7=^_TKleh&vNx<)Hhbjgw`YB66_!-5TQ+;zq%%%GbHR6Cciwl;vwEI; za-<|>qjzip`qT$Tb>aL4yLZ}^?Sol$BpYDRLqkPC5K*v65!6{Wp<%~LHQ=0m+0_?c zeDR4VpL|o#vrjnxyz8&JX8n>ME>WmsZc=+3IAbmdb4}XY-qAYsK1jDr0a4lv6-FC-=4!!lh2djPal1~O3ru)~& z9>u8g+{u%^oi??7>sCv?`?m7|`=8PCoRgzPE-EF6pppK->!Q|zexoq($i3Z2CX=kL zy78tPx?Xfa^Hwb}39YWEeC&aHjfHb#ZQC5!`5?w8=tkXWBM(_vjYgwSz4!(q^tH>D z%^fqg@26jdBe6~EHqM$lz5AKxe!JkihAo<(diohgy`~s0CDKMwy+WgY!ICB4?}~Sq ztg=s*~HA9fIQyMgg!-_&p_o)klCV9#8zC*Hsr@ef3AvXPk8AnKz$v z?z)vLBk+EHdezc}w_ktV88_T`ELJB-h`|@O6^mC5?DuZZ^Dh+RLsrbt$`(Zrz?Nxx zrsMwm+tC6eT(Hy5yS6*%;BaC6q#5qmt{rYh;Crv^b?JDDxfx2 ztyS$4afJA%af9&{+HiOpKH zrZ#KWtm=I5fh`a0eAyXi+;`7k%QmcGgERC+gEk_-vt$$pquUU~VDapPVdIB?LQ zA#c3)dI&>AB|?Wh?6t;Z0Eby=1VQ0gr#<#O^zb8)JNqGLmt8t;TDuZ~#FbZH^T~$; zPCx6cF(Zc|USe1gI~Zy%V~p6vWPGIIW~!?EPdDDswta`|Z~hYl6*mft3qBk)u>bT= z&phY+doH`QvUCHDbMX~G0})r*R;PmwG9CO_MRwn1&o=uX9EugOYP!qL9agVg37OC8 zdA4VTd!F3=U$4GgT~)?Y6xVF0c5BwGLT_;g>yg{Gx&d{zExm)%ep3nv#;TxCrc9Xv zzjwj!e{U8RUUl7dc)tGr%UKpyDNDb<;j#-)JO8ZTU42blbfOUjxLC1F9yJ16k0Xvg z4x>I60fB8GN|ZHF{WCmRZvYzn6$wZvBq)iPA{-PKKkz22E5?s>K5@zU~I(0It;`sfDyQ#6C zOyEmz+_c%6Rm*O>$7FY|X0G_&j3r?z^de@4VgZ;wv+eBFx+2F2QKw zE&)(jVe}#x1Z3)Od3Gqt5z&Tiq(Gcd7@l&}xN(SmFTV5=qjAF}ix!KEJw;4v)p6Gj z-+cGA=i9Nz#jcv6Z zWhR4C%9pH&9(VwiFP-nLyrwK4Zvmu_m`7Zi+@?D-%tE@1`|s%viKjF1qMqqon@Q#q+=U zV%7)!-omxFsM(16I1X9MZ@S^yL{-JDH{ZN|{aU6J)z$3mWAuLG)gKlw+*DfD_uco5 z!omX&IaG!>v5w?1L{AaaD>3Orft%HXl!*0FdSttvFoiNU&YU@G`HB@MopjRPhaK_5 zg87c;RaKN3r4_B)cldX;KQ$Qa$o=&Cf#e402!95S`df*}N_UmJ8D*bHG;Hw7?5{x&Ur$>)#+M@ORg^S|fe0$O} z&my4~$%qn0^9M*}YVV?`W}pWMKWtaBOc#wJLxIUnhYHfDNSo=tZ@u>VEuR{N1?>+w zq{n%eAV`Ogmhm8yX}9xErQd#CUS3*Quko+}{Rceph*(mZ)>X$GRj^;@7yG=2C|KDj zV&si51?4zwaCi>Mawua)(4)leJ`zFZ!~Sz^~6)pyzt_rJ?H!v|M|~QLA|hP;zbr!K1Q0Q%NK3@Y#M~Q z{>(E)24h7odF1Kt-Mdd8GiKzIk5JK8M;vv_Kb|)lGy}H9rbbO9M7zbtA7PYL%#>rt zsS?wfjGe+iEqkq*_NOyuKKshwnDjK<_YS`7l7LWu>$lmQ(ZNmpD;8x3i+BhEF?G)@b z_#&*U=oG^>BJwbBnhIQsy<^6Ymv3Z;FyD&d78TIsg+<4$EH5uF zXoTs!S-oNH#m5}`=I3*9+eF)y6vW4*Cgdkd*R5{Uuo)5zY7Sdyp3~up4IExKXg0s6ogkO)vG~Sy*%ELdeNE_=RELLlI{1w zpMU~85CwoOf<|qPykd0j8`V}Axr0-`1q`NwCLAOchyfqUdSRplA!Xzazne({{g&(` zCWD60egrJG33DfFb7Fih8?yjYSfFsp0KkkNp+Ve;ihLN^amEXmG)5@F-3)@~Od1I? zbpV=ilyvn?fBs{S9!qZyQgiZ^v4r(YX$cA^ELAeBAfhqi zctK$Zk{xtqh0cuDb7CbZgrMKDvLb%k?RSb1ztZ3qw9i;D^9)giEjq--0g0VR;lvP$ zyo_oX{G;ZG;3$GUI{dRi=9Y4x!qx z(u@INn|&{-ugyv*v~ckI%Ibk=#J~YibobRUUH?WAH}<8Z>HlRJW5sZCRWY{1>!lWRnteqmf+H z`4Dj2b-oMyFA1Q&NHzK8j1TjdbKKNgvz!{ibt44F`QE`Vz(TA?01fT~Mr6&USmYrJ z*Uu6MA_xde*+4YTrw}y*LZFRs#OX9%s1N~*%t7j=QUV{$;29wZV2!L|${Hq82`QTy zJ$V9O!Yih#K)86MfakcP9x%UPA0G0ko+O|k_*kmJWPvRoLk5;LQXyhIHze91a;_KT zD9d+obRs*)XJCLu2L@TaltnUKJ;{sOBCC)gL0T#~P&HDz3@+kq3p}JS5nXeF*ggY( zB9EYz&|b>QP6d$MI2QOzhSJ`o9|~FQ(nAhR2)HSyBvm=!U7Q`+B8+s(xe7A{N|bV$ zF0gtjCZjl86bi@-GGzHzmV9jnQ6ZbWA?ibm65=D!iiQy!AtI+X3};dU739&}MZj@G z`yd1q*zqVJD5Ji4s?L`W^eIpf+<{w#fd*~_wK10jD&SAC#1R>^0#Ghhivb4^OTi=Y ztPF~vn)N}fWk6(13I%QyERvfTbMrXXk)k;$Pl20)#vqi^(H39?{{S3`0meffU?yz8 zV2@y{2v{s!65^u@%tDVPrR>dFAcVb+r+_(0#^bCyQi>p&Y8pq_Utvn|a>ekI_83LL zV&QFbbv%au%-adsZ~n$hi7DyN)m+r43P-g zStm{7qe_*-K!Pz6`4mPmCuLc|4vA`yPRd_?1Tvrs>NGe=KMz7cfgO(mU|Alx)d>pA z0+(Qpu$f?kU>+Eu;0)kX-@QEL8A)*qO_PQ1lLRCL5k!bBCLf?D*P9<#SfTP1UKR8d z)ek`*1w*PDTr&3;E|N?|m;hq3F%B{lJVB9Ak2uuZpkw=~*&YyS^6%j#C z&g2%WdXk}>lR>_v7`A}F#E>Zi07C+SK?{V{Uoz?S2qx9TO9|o#xL1`A)uVoZDS^8X zN<9lw%cI2Te5B^K4U!@jCmsOjFbW zfL;DWCy6Yw32unU1E^Ao1^*Oed!r^=jMgpt)T@|3vN))9(*=2WthB~wb0B}A2&1E~})Q<|zRQIrlP zMVUHRko?_@Z)74oF!}IT5Et=`YKE}RLS+7AYefw$nVm@OR4V{jZV8hO*0S)b&Pl@q zDho|aOWu~G&k+;o71@f!Ng=t(4W~RMWwAVvucKrnQXLc*vxRaiTP2|qHPmHsE-5wX z)I}XX=x$ZVi4Zh@+fhI>iTU!j{47ktW+C&D!AU@Y z+Dd_dziMq3ofLnyK+{nUv_K6=gSRww8 z;)2V70=1a}0mogN-|qI=Ho@5}0&ixwsjB9eIp$WsNcbywXp;Vl)Y~U%f?5R>_+=Cb zIPPCYry$jDM1h>%Mi#men22u+B|kyK1C0pTR#9{qq-SD*frj3XO8$5LTH|$282+Q= z!DT>!T1bI_@)|;Vs)j%6~^~66mLL*>3qn za2`;gc2XeVxNGNo-F{o8gd`&77HbQ+$igl$iuJ~}4vP^~fyKB3e;1-_BC5WqFmuEh zoz2n308cM&=GwXSRW3I88&IH@QeZ2Nrk1ubXx8>e0a=jzaRA1N)SJj)I*2nQz*=k& zm8mqhxWw9Aq5Y3p@4RBkk24Lf0}9ke3IxDi8{g`7+$N>}hqDY>NEBI#mY0Y{qL{^r zb+#BnV|<6_)eKIKLc(La!jzgOZ#DQGbf@F*A-u*n$;FYTq^=9=3R-C`mBb-gUp@ied#= zP|6+!S>JTx#v12GTLwUU>xeCH2Im0)OB}D4(OP>5Rd~Z)}4eAn5pr$BLOMqMUF-5~nDJo!_$4}P4P(MI_K?^{cb!Rdm zpFK{11o74!%DN1e4W%3pH_>e8l$~wMA9YF$S@DEC!%uN^L`ohijgFUc#krl7%TdzH zJ|AqvhmU++q9oS|NTV|oNdcaY)guy>t0$*YF24B%y(Dl;2c0>`T+rp`u5!GmPL?Z0 zyxDIZws<~UsN|D-YU{MW^_5hHtp;Sv7bwqm>GZ&iYg?7!hKKjKwA0>4$i^^K$F^JS z)A_(27sRcCq!o49jhEM{HBK~htr~%hoF8wFA8&zQ5Pn+<=-!^|LfKA2SCP+(SHO|a zjU9H>7pFe1-?sc9x1Xj!t&wqW9)WZxLmUcXvHO&`qQ~*Q%(R8ulUT|ih3x+ z=C6G73dc2(1QUP&B0*sV2jmGT@Du^0;2)>jtWR)Q-GYl0P|9M5l(pG0KeyN!fEZn( zB$ONovNMZ1*OxrbKM~;O;`>GRSSwcBz-5GyeA*G;Psw@BO}+JG*@K4dKdrc<3Dy$@QaHI68;Jb?C`)X9Y&$j)`%usOtA%$ z?4|_Iz;1Y+5rdOsGgG~<2fYmJmWRbD2AfEUnFm2(-j)~#SkoB+HPQ;pr?8h8fgAmZ zItn0D4HYks3)3w}okk;meu7V0lIUCPVx=!7u^t;#J`$^f)KUI#&R<=q2pzAkw`{(y z{}wm1IjEwt=lS|m^UdweXI*nAiY(Mk8mmgtrQ%7_D&HH9lX~RADI^yKB zD?)|Y_fm8933C2f3P>*!8&~NhydpWOj%7PWB9#d{>^Lu5P09`S&yn4W0^t5xg8oJ~ zcX;3yFp|#1l$ow0oeV&vZ=q$z5|>CvG2j3uETdY@=E}?B(#M$d$sW>ZYoI z%1YXyPV`Y1D33}u3WHTuxhs2g=W0UzRkmsvP>;*u-z$J@K?hZH*|`AGg^JPVa-nPR z+a6_<6qX+$iIkSNpp^NMmo6%9xOdm%stu=I`<*&3^At=KUafp@=2%?G5 zo9*w6c*^jjM$$4?ZP>7S?V4uM`s^Z2=i|uq988cs@Z^tlO#=BSKm^JqJ}bP_6Z|aJ zs1j8V75rrkQ1J?0G(^f?irMURtmMn5Ex@gSlG7~I>Ks%HB$6OwkOU3E`D`RcqGg2A zC8`}8h&9|3!WGXCg zx6Vg)*lT~w39l=QV{ntUs;Obk)hEdL=PAI;Qk_=19NyAZhmp==MZ=3$tcVoEc4^Z( z2BIg8kR8sj_hr6^Nxut1O;KQn2W|*Mm(p6joNxse@kcZed?U%SNz_PWjFqd_7S${4 z)^(3qLS}rDKs~4*;Ft$8>81+KRBpn#=Mhb(p=SamVmG3K22D-s*b6roJAfKMQ zI;Q51nh4}iN!fEsKa>Z@yZx>rU+>Y{0B!U+~{S*aj5X;elIE`DOC0esnNoY(yf$_8lbJg59fD zuWMFZZfd@6=WPV;aq0G*4^mZfAV`f**TP5?j?Nj}+} zb5CiA-25(yS5PFpcAi1WC6*`^&tLViNJPG>a-piD2LryS99(Bjz)6{VIxF7MpIkmR zi_zqw(ouxupiAE(M#tC0adQP!6}Zg4I3^Nmy(>%1v{A?lP01V7uOm4APbdJh!aL$C z;K&q|PXGlh39gK<#g@@{|MnxNeZR-vyT(u!Qg?#GHlV-`PwE^jW|5B>ot!E02f`1g zcVDs4V1|4I)-2$OKcLdk3%B~(5203cn z@~rO4pq2hfK42^H=)34q0hx}}T4o9&pXMmH$W`jbr661WE%_NSPN6u-nOUi*i8DZm zLp=fqS|zuXsNV<7rJd?yl9c$~BVROYq|Q=>G-ydp<&mthqGVH}!PPZXB_cCr_6;&K zZe?f)P3EtvFf>V{2%jG_YqU;5=08gT`HW0D6Wt{v>-g$RqVGw!l{aP-hr_ArIOn<} z|7>Q#&EJLsc}Vzevio1D_D$9AxZG2!GkESS{_iY)5uzy3m0sO#<(CR52bNH3bT)=|1dgI zzD*<#k~6s}wB@L*%J8kJeSV*oqmN>#W^aWa&sluEZz1P5GtSoVn2 zhg-0!WF??SVBR1)jRfUStWB$srlqF7k>C; zQB_vDX2o(pkpw7M6UXX(-KrHE*R7ESbGjp}um~dr^8i{P(h4~lk){|vst#CV8s?Z0 z!~b^Ay&t^ujw%I6x6 zAOxvW-Hz#c`tiq(9{f=x<2$L;{5hZZJoLcR58S7!bUgwPQi!MMEE^7E4k*i`$U|K5 z=THCjznU{=;@ENHM~@;u7oJ(56ebZlcslH*dcXM3GY{VX^n(waG;%~p^>0|(XxM~D z4+%{MO@;g0N;>A57!U?06+jqL_t)a;^%mAT>FD#JQDKKY|awPcu_caq}!e! zHvEX?htlx^cpSm1%(HQZQNy6v4L&(li-vYpy0i&cBEP0Jbm-TQEhuEt%)(8KaE%jI#V;F16(s zsNQ05s38jM_%d$wZS%F$w}joL4{3oZ(qYBHA%xB#O5m;l0UKX=vQp_KGhPy*21|3A zvR*35^GFjSe%h}rA2Vjmpn(IbD#{N!_`r)Uy|Scvn`AsOe$aqH1Nv_$D?Pl+k(XU{ zHJLVrvQQ% zf>XwW6&I@HJ39(=hwgClp zc;KeH2(PWW3tzs-ZngXX69M{{^f!S6VZceFSMAW^Ff&MdQ;Fn^88hNG66wtPrSsU7 z7UlwZ#pt1`FOgho1e*HeL{cG^xp<|}Zy5sJ@%oy=;VaRFLvf1*5-7S8+U3opL)#_P}Da}P$xZQHjW)_(@5 z7;E10%HLo5)Z>p%oH(gzG`+E`@~N-yU%Fu4@m&vN4goQokO2!HHoQMvdFkwF(~F{Z z$ceoD+P{`8TKw=!Zz__>L5Wy?<71X5Q^PGKaLFs<&FBdI9yOyasxTV0Yo;bO(Yo4~ zXKpB!@ii9!Ygs+=d=J5$ut&`7)R?Qjwp^;gp&qB5#yB`jo^2K#` z|1DGynK<&JsF%)EZ5-3@l{G(n+x@Jw&-vZ&i=4~{Z@m;U5|x`)%^vbG8z#Onbo`*{ zUllfQ>%nz1FY4QMP{a zNADz(31}tUusR|HM;}s<^|$&7P|FyqfBfO4uu!Ytqs_vwXtBd^TX>QYBfWgm;N*Wn z0X-j*X_96jUR)22<=-*oexy{`D^-F_n` zj=$@#_jm7cmhYvSx8J#K`<XvOwv60yUvJv!}XJJvBrcKLeN40!*^lP9yP7>X9|vD zK13{U9C>6{D)!yH`LfA?;dMRgD49yg!lf~8;*_~x&P6x9%VCG}E{{0;(DB2E`fe&@ z`cuY_z4zYxlPi7@X32LG@f2-msTQ?G)+IUrX8t^J_LT8YKI&-kS&pHRk^cPi&ksB7 zFxEoR7^DKO>!YnGieEtpC=gI!yP-hc18(6P?64J9`Wy}{gKLElo%nV{U>b#H&-=?Q zw=Q44yvGF>U2)ZwXB@e|lC+q|pWOZ6&%b=@)t7re_v~MqB5#r(c1olJ(E=)AdKmTy?? z`nbm|M7Hi$Rwm;Wg@wi6e)Z*s^{def6fRV-+o@l_J_z1v*F8B$#?gPoNGB3CoJhE^ zI6^X$qRonR8?BvF3%>h?;lwf{rR&z*an)s5{BZgFZJgn$A81-2Us)VU8*Pzmw3 zhXlZOSt)ZOX)Ba8orK6%!rXH`f8FYJD9N>I-QtZm-Vhm|sBc?Ch7B7vZsIu?T@-87 z$cV)j{jdbvNDNp2#I%nUiwJ6vkT3mV$%5HmxaHMPKmYPu{XTf_qmOU=^Br#5A2ej> zo_p;hoU&;@{p8>Ky#D(9Z@<~Fu@vAx;NU|Hh7HsD$J5XBe(SaQbHA!CEyYqsxd;!# z3wx+qnC9;L?QayYge5oq;sy=h=-pS!r2sqBpkb2@n>Ni`xOm;Vb$jo)xK%E0_fo;pcZ?G4M7$KazvSA)Nf_5eZ zY9oILWugVw-*!8k@{5l@KCt(jMySAWVxaA-|9btgJMKKK%hB` zKpYC8M+XNVGpuO84?i&Kl{l^1?RLN+t{qL7;iFGGQ{;aYRa0lpy8Xe20niU#dF82l z?{x|bo3?IgJF$Y|#<$%6uwjMYf8}3K{pFr;VPV6Tt?Yt=1oa6=5s!%LaF*-~>)rk6 z;~32JfA!@{PU-&oKb~5&XaRPR*WPwpBJItYI`vQ2^b)1)g2G5)ajTAk4vV_J@Td85 z^~&`Y%hvfjzx+hEVsoB62~M$p!s^xKC1G>7cEwQ-F@q`=5X3?~UX5Z2tgcc6M3V{C zY}|#21&ryZJl9QDRlWWY%O?|4W zEAU8t;t_`>egEn^@4w`#UQwkX>)D~Ciw(WQ%kokc*X^Ipp0m@Q2mI^px6P1KT2{f6;b_E(gs`l4m9>PENtTwDQXM2PAhYYbYu9Zg z^Ja}3Aw)B>x+BW=gd79Cv&0J}nlW`*jK=P3Sw_-6Li*Mur z2v0EnW|a^N8+rh`C~(;!FNVMut3{ukJ)Ah^Dt0JRToNm3z|Qe$^iuQ&IZiamAOuTn zTPZi>*b!{nF}1gyuwq&nlI)Bf7FQ%@#&Z+TKKc0i)yq2{eBk);rFao$uS{ zt%YBIu}7z!*DPB$Zukhp44-;WPi6yYKNBr(U=%TYSa{k~Kn(nv2`dt5*wjG+g9k3` zeIf-7o5w82smAjl1*ox4Bej@oD#)gFH9x-p_~Q$%0}2Eb_)iq5bHFXEFHnZQpyweX z5z}zQ4Z>iGi4KARjCFu@xC*?UiImu|F>%k>EMN*gizwShk|1`4;x0;@LE)Q3RKS5k z+<)@`JENiEO6D?~fwdM{j2Nur5bB4OYbK5uKJ3E*By84k_Xi(%s9lGh(QjF_Y{j%u zBSwDk9(8Emq0{{jKi+<)4hcpVGH2=_ejpfN;4=)DWy4S??V;vVTWLR>>aXuyR zu7fzkLRWiq9vMZ24tuWkU?lE8~ZxSG>L|SsoyXY|tP_ zwmy&;9i%>bgiw_bfaiolfq1fE;gJi;Q{t7KQC1Ks+xujP7C~(l1e$eK)w9`1ipV9< zF~KpQKtO@*lmc}QxO3Q~kx;->L#p^;P?*b$(w5em^VI069Oj8P7@$zCN zN>GoxX)IaDNFpSVOnG=f+JW+*=&h)r?knCl%Sdm*1yed;O9|Makz8jhPYE&u-{R#% z2h|ph?;=VelIejYlibu|B2|(+vICSZoH++U)@dn7MQtu(%SK76AOsW$D6l!!1N9!t<&iSjmZKKIh;$|Co#fEG6auHyk^3r2X=w{1w~^t1T(>e6Ap=Bp`+4DOU(rR16OSvMBlht4gax^2o!kt|20A<-|lDAU)X% zaObpGfMfF4S}XXGl0?)<=Ptq;87Q>WTeXqfI#)@P_qHrg1tFk7K!NR;0vbcM;|l)P zRRf`NIJr6NJhbJbwDcf<3E*wT<>ok6#jR`X%cdnOfm+}?x8A7nliS?dDycv&oWF`w z7z>Q@SFl%^yQo^}7=_@QYlB|5dR87+x0Hw@o5$v4z;PC}RXwxA13Bs{ZCx38OqIC# z`QR*|KtO@*lLB=PxYOe6S@i7yDVc8IN7zw*ghiQK*irr?QpS6ph=)<@BaPZpPduRV z6bv=MFK`L&=^q>}zC>8kRI8dihrx@m!LqX$UZv5?#zI{%QEnM&HiQJ2m1F~xm=tz2 zNTSJ5#uMZqSUD0Sb)gj0vpu}@+3fi1`Y4~nony*~-zg9o-dlNR#TpWKS1BvQk2KVZ z1muh>K(=;dT0tu?24aFiIpP$H83lYjc1JY9H$&(WrV{L(hQ>ELo_XwHhT9d~12Buw z;{jHqq!V9(8FU%hPe*7*l#eARxnUHVP3g*}_wo#tplFgwGlg7Qmt3X@;mtvnYZ9? zfX3=^i*nfz4#QTmfEE%@eS;8CUWsX)sb( zVkEHx7l6yYTimbW;6yDbH&x=9_#1S5PZByLp188|tZY0M=t??kvGm6U1;@QcMJwG-2Nn#1O7vRyk$ zhraA@Ni|eT5$w<$Tpmg~igIzHBEy4%RV_y`G3Li8U+%J@aLB=X2}2w1rKp3Vm4o|3 z?vgpg5%{yLL?WpvLhDG$JxUOVD9RbEq{+|*u@91V;SqpMQk0$x&q8E~Ks*(r#4z3f zKxTnu*>c|X*o0m-@5Bi>$)pa9pi(G>Q#~0J@x=&96|Kxv$|I6$Xgg2F1DX%TtPpfm zRAD?-H5XR|;$Vg?MTOLiv*O$_2mu9lKnm13;07=F;aVvyp11jUh3_^&P(SUuDS$%| z4lGdc3=RW^Y>p11r}s507L*# zVdr6pc}Dieb}U@_W!PUm?aLOu=dg0}6n8+9Q-eFfbmX7(XTU=pw-OOxGt@ z4DqwOz6)zhwOv?w;L^dsU4TrWSVjq{B$#hHfELcAh?UxN&8P#n3STe7i*T_v%s!}U zv6+MyiZZ$&ATp>XM)nLe(18ZYY{4|b;jrK`B#@Cseze+4xR~UN!!7Cw!>f39;F9#f zYcyoZupuJ~I4g>aG8%An*(+H*$9gzz5kCY>6HJDI82hX7ES$KEqf9~=KXKy4n-D7E zrhr>i-VWJ9GbT)8RLVxo?1dzQftwO%0}y~wj|?FrEu<|}pPYr?=TZm}gn$A&9tG+g za0|OD%&a&P;oFB3RIDGsA)p4|2)eON4R11;lzDXjcl%Uq+JM`?S6+Bw=Co;!X}@zb80dmjVuy$>-IzbM+c(US08ZKysaJZa{@WCEAa`^Z2zNOAGI>4ce zqZM2k3STUe3^z?+DBx1I@655ZR6qtWGROZS+;*D1m?d*cwj4VNxhR;Mu*|OJm#F|J zjBU&uydi@IjT}6PYO`$+?N37-GLxAK!;(O(OqPTbXUvhB3q@(Zs=tt$y$@9dbgQ8Z z2_%}J#<*l68LzCs4F~( zAHVjmR{&DP4#LVJPzD>}qhY)?5aVVfjSE7=>#(5m7JwTw3@`4_|7Py`Rm*&)`@S3E zF(ur_87y*K&%swADC~bf=d+C~mKHJqBn>u<=UEtB_*F`zjIU&nEA5V!O9)2BnU5rj}#1ux1&u$EMSlo#JKla`NzN({Z`#yf+0TL{DaCdi? z0zryvTdV~t)C(=eTda450&Stiy%a0%?h*n7NFW4ANJ1c<TOKQSG39A3VPWwE|&uieB18&e$R6@83yTeO34cU+uG_;@w!-0TwE<1yI zI8;=>puB-6ZbUY1)DWl>MG@FyuQSHuOff2&7!)&`+0fAe!Z3FTKmdr~@O$r!TfXc^ zw>{g-Wa6&OiLtFPVIV5N5Vy?;D7)PjHRh)kLe=msTBO_ka{D$>SHqdvv=BJk;RP?D zb9xzJt|cCJCeyjHhIVWhb>*Ugy(M)n_+TREL3DW>)dF!K8>|G|^5CJ6NRb@Cxx<$A z$;aa-efkMZG+L+5VfRBrc4V{Rv4VK6WO>1UBgfqC&6_rk8aW&&G#MER;7QyeqXQQV z6162tEXuwp+|;BDDWEbCV`SdJx6xtqF?cN8blMq-K;jE!0nvfqTNo8!78RO|I$W?T zB6I8xtB0C*X!{Q#ySOV;kxJG{Sa#%b*{xKZgL-qqxLYYT@i^^NICUbzjGVY#7Qt}j zh&V>)BXUS;%&A_VyL5?SVJ~+3V%Fy(UYVq3f`QbZAJ1#xKd=VChG**#D%s-!2d`ec zxPRB42MI}ao3?1zq2tLz2U1fW1?SJZYx}QWUS2Qu?q8HG8C{e6-n>Xq@a_rNZQ zS)jCi*B8r{DH9cOZqvGT_}cE!v2)cH%|t&WDs)PlUzw07my8_}aikG*{H+?kUnZ$wA6Y}2Me^Ok6@ zY-|asZ7_I2IS8-{sJS<6+z?VUq*bey4yWyxHR~&sDH|Da`Np*?ZQ8Z2UBBLnRjXrS zZuRKay-fY)SeE>}c1@)U6)V@MbL8-$l!UnadGqYqd!Ry(o$0U_3q>2@3(UGDkzffFZM26ihV4pF#EpkJC7XL?-!V- zXWxD$OO;{L(w33BcG*wYuY}jCR^8~d>n(ol<%asra6n9Zpg25Wy(jM0t!*1Owrk&M z?dmo8@&pbZJn+Wt=uKOX^{d1BA?s@eKrF)SyUhp;>^KTQ{x`YTG(9CAnmw0^t`gQKu(Q9NW5O3oS%gV5LgXXmJ02S1z5;PEUKOdss|#G}39!$`w~G zT_mbg$Bs_<=>4ze&I%0hojGaZS94~;JP5l((wbDjR0E14J9F9GSwjX6vS-`Qo;>!_ zkO4yn4|LjXXO10yG3*6~7kZ~{$-?;`y*Z{pzQB7i(S16#^*GY?_N);@hkQMEc1YpE zutchYuYxZtNR@e*Ql#`1bQj zldLc)ZqJC9Ub%MZa?#*?AHMPCo?m|r3C@4<^qJYyr!gFv|Mgb~_a6u;T#Ow!3s$Uf z`{oT)_zM@#UAcDke?>2TUWjL|2A(;Mn;A!MgCX{I46nLp?*Tx`OpZbKiux#20UEhb!nbclqkRwT)2~#? z;;ELrLIf*|H`Md4y?U8lb~MVR|C+mas(&h8!CxuFzrQz%yBCYe7KJ9Pb(%MpWLdRehV z@YIGHjT+3DGuvU$s8fBf$ICqHtFI{Fo&yKZo;XpiSW&i$fM-xFheMZ@zGUT6Z~r`g z`SPqd`|>x9_Z2r3!>n)22=B7^CC(7E+pM zv3Oy|0J4XQ7ZGHbGHZ7As@3x6%QN%i@$=XHT(?2}QbmePo;YR7EP@z!?b+vVwPd9x zT)uee*xp0kI(7Yg`i#zv8m#{5hYjD(Te0V$!OO>~cUP`n^IGJUmhD`yV8<~K9 zemt*%|Fjx-<}_}IV;Ihcjhj|%)T~>JmhZmzL2^=pp3TVI_F9b^2NfuU^;q7#L4_(+ zI(hOWMhdVYz!Dh6=v{2;0C|*Xpvw&7BKj;p8Z~NMrD^jn65B@);^Kr$fuJB638=T* zvPKRcG31?hYt*a*^k2GgVbiKLITi9t?292snei-{bNUtBqhXS7{hcWOgs8J zfX#d$FpW+e6BARoaA7h|0XXV6Xv8Ksdb3yA>NOfQZZ1=^CUceQHSXWNCyK6!CO9Ry zV||TgE&crSVmVj7Vx`CE1#+x-m%hiU1N=rX^@>KNo8#-^F%GGMvnV>XVo40I1KHU)0R62}0 z=8x2_)FGS6F$~a^uUv(>WQkv|t69H3^-!iv8C`a!4Fe#jlX|FKyH@@B4UZo?nfM@q zToo-+c+~rE&-vuzm&U$UzI=Ix3B23_Aw^>E#Y((J6MMqR$@AlR4g9Coz%vKj=)zb7 z`{c>{@#TUA7)3kKT}cLVL(Q84waAP-0eN((8JUk$;TbG?cTChx zAEGrg=49rT{r0vq#^JbVVkpwIzqE^T_lfB5bY)k~1`ZQThNDS=cMYgFGr`VP= zHB7Y~II_&fOKCc0(foBf5L0+@jeCg7q@bK`4^=kL;*E{LzwQRc$z3%qaE1Q4Wkeu}3 z>#x7o`TC;93PNCk#zY_1Z=f;kh{5Cr9QBws$chhO3gn8odhNo+h&pxa_v+U_AtD?F z5WvJLoDu%kUpDSu_w$4apFm|idi2Qda(C?7Wy_B3QFmf|{ryjzIPoAU(dMv=Rh-gW zkp=n7l)ZB0N_Ivj8Q=xYo%^rHNeSmpo%m(*rfx6xvRceer#(;U&?{H2WoKtUOiDBc z2Nx|~{K~a!o7Sxr6J;?>G@?F>`3%`7ljPK3C{w2F?dT|%!;U=&s~a>^X5;SH5Xe!7zbFTeiUu};mAJ;Ey2s`cu)_v~tM%D|U~FPJsAMaxzlJ9dOVZeFe0kU@hQ zw{9)84hslPq>j;@*e1$)6+vo4jaoI29X)JenKzJ%B5pD;BEY=~3&v4IF>iDljM;G2 zATd~I@x_F5jXGlY*4pz(e6IXsYyH&Lct5>b8)xJGKf(o7g z)ohENtveywML5W8g2@WCVU%G3ln!w+W?&a6l&zK!pu;`@E=KL^YR?N-ll*FW@OP4EGtj4_Um!HpA{o@b+iJkU&o}br% zOtSsiwP4rj(sjDSH&QMqJAy)j*;iaRTJ)yeL$HG1|GzS#J^WjoLhwWU-ZYwQb()^Vzd2)osX9Mcx`8 zDoK=L1I`87p86qi4Bmlb$R=b4^^uWu-zOlz;O#98yWR+Ehi)Pf-kk77YOZzS_zxLU-%22Z_$jM+3jS*I=tVTbf8tfiSzYP|XS3tp{V5<=-OzI*A0CL!#hE1E-s9lE{MetCTx`?49M7B|8V?|8KlqoZlO0`${ zK0o8(3EmGux$)h`f&K3T~M#URGZ}@D_B* zrr(>5k5kejBCciwSGX~9fuSIqfES8tM!GF2>EXk~OGR5Td6`&mk0vW@2GgV- zeSBg{;={)oP%1peA|0Tbf(cdv;sR@Sp%-vNOD1(UqN3vCAHa+l;c*m^t-eK8*&VKY z`GePQ*ns-XKgqQ?I05wxPP;uZA&CRh9IC@1zM8;Con7N9gsNanLTc!_!i@}id1%~& zL^YToBm5Oo9UT_xgtAw zGqPAykNYg))A43yDps__&|xDemr&d}(unfvQU8UoCr+Lf`*aqAGJ2xIMc$AMYBzKi zOOIsDyAjtYA|bP}P9Ygsd!wOrnR3T2Ud$Jae5Ekd9z4Fttmnt`8u*W^foBf5p(?yt zIqA%}bYyp{l7&Zh?bM-c$lkTfN5A>b^m&VHqJ6ppJocaF&gfjZ#48vfS1xt;{7Kei zLt>!G0UB1fbH;?X!m5ULt5vpD$>0gY2Rc(83Za6c4YU!`;I_IlPwd$?pl2r^U3O~J zmF|sd4ei>vXWi1X-WluXNONEg!cuY!Je-~zdp5K!k@wZEZQGSAv3vbWi&NHiqaia_ zXtlfculu2AjbdTd%C#+1Vr>7ecP|_UXhATBtAG`pkI+64Fz)Qs=qo*%R3F@_WzX7` zr;dNqV9SI`FhZNN0^Z<>*tfk^iJ+l9I(Dj6Y43)gyqsCAp0P7r5E9wB>*t@k)-DxR zwS3#q;&1fq9DnJgx6NS{SKSB)l4GJzpwp~#rrZhd+O%rFjxECKR-HBeZNKa^r^n(k z2)k|4J1-sB+O}lSm~L&`l`gVx<#I2(9VQ1U2*tr_p?&L@_N-Q-OSSUt%NKd8Z|ArR zC%l~5GbT=4ICH9xJImtAW{AREf}K$U+C2-|8MELWT##A@;LL!+OX1+fLdGztp{lX= zMrUS3gOC$xCWqLQNaav4WQq=IoZEzy?nDo&QcV)NU3BskqmX@5UKm*11%%0 z({CSv`P_@@5uVaKeWUL4kw~=gDo*W+1mL6Y=A@#f`1E(~-JA$>9#OQOJgQ>gb*o@f zKJ~1)9p|Hp`@6IVDS>%1H(lr``KY>4hCn%4B}R_i9Ll*$MXpFmwXzdHu8Sos6~^bu zC`IZ|9faiISu*$h_-kt5nX73OM$rvjt}!Ery*=%-3s z&z!Y%+pp&$t~Y7gbkd|r;4!)>P{3OeCDr3;*SpV&3l~W%k$z5sTD22d` zOxR8Q_>+mBe9F%$Q>Im|TJ`khYb$r``DN+S)2B}pSmMEplFdG1 z^2XTAv{ckeV$vvDlwz*}1y0?FTC-zUYD&u7*|SK8r?svkOCm*_NTQRcPN`h2`l*ZI zD|YYRzI55Sv!{d#R(WOp=jhR6-kUltJmT88snaLEIW8judRL7egfftp=3lVziEELo zcJ55eOq(@(HjJ^T{Paad;Lt@0hOP9pX)~%+t$yM1)fK;P-?D7k`7@_^nvn*bnXi!U zEx8?ilW_paQLz+UDwEL$g9|Nl)Rlq;t<>Z;2Qn0J$hrFDfQ|whDUv$4`-f+9B2mxg zhWfXE1xnS25X7$~p@mlGf@L|$|1l0N0advhd3fRlCSHQ`7QIu%`;}UnZm;k*}u$Dv9*u#`BYZMh!gOu>Ki2`jc}( zi$sk)a^#>cJ+)h(zBYrod;h`ACZ0HPN(R;VFE#3SZrxb5QNv2rYiAj}hYlVRdGaJc zL%ta7qxeA$fGj32t6$#0nsw@ws8a3W!(=dDu5(9e!8ijcR-jPxR;?eWXP(@?X5X-W1Mqut_QWCaiUb(2_R(b%NzPju_^$_?9kTt#ZQ#hj#4{Lvo$Y zs&}6}c1V|+`eMI)q=SSwVkS*cN@-Me4Tj_e@FwK*`>?sca;__&12z>P3V@=+<_3|APd+ju zq6Rkt-&!gfNQ#ga5g;F>js9LUkxT2MXzoe+0YuL@CpPKhmang=*i<6V3 zoD?Mj0aQtk7(^iPQ8CMd5?IR+wIbK$qYj>;QZ*H#tcYqY2-;d#6CK7pmn#@)MbwZJa}zit-9 zSFKzHKdK&!DKs<`4KN`g!7pEsS|2Dq9IRv^;{F4NjvqW2b1Qo0yoE|fq4Iytb7h8}nS0V#);E(=M_Y+!|Q z<>T+inT;mdrN-&N_OW1vau%z%&1m)V_X-LPy&rdvu21iF+OPnT4xW*cdi3bA^Ovrz z`+4056Q;Dg1jPfblfdi>bsix*a{TJtf(5zPDC?)r7=PM$UuEE3PU zAUk9g{6AiG`4<6Kzl#&zYlOrbz z!L^)gNCI_6li{yA&sCFudyY5tBnow+@|go=Poff3{XY&Zw|}?{h65C%mqXpi$%6_h*A%SF-Tx{}a#@vH#(lpJKs--Zm&EY=_?v3r*#mA- zFkPvS9tZgP$pRB`@2TTFJ2Rc#XRs~7(l6z4Mw!xnY@h|&c>DO#VKcJQ8N}0+X}EM9 zOy45TUzqUG_>_D1sx)kxo}Nxe0)x-7s>2NzSK7sa0OL^?)>qDGEu;>haem<$G&^{b!&@LYf8zb1=S()QMdN2J}jCa0# z*e^2mia}tB(9#cY-9pPYS&=#dkK|0`mPt@pKuGnC-{3|h?-K+O68Q;~D>#$j>V)e8 zwc14$CI|IIdlu|g@v7+m8$8mY(t--62@xiGD^Y7P$g?V{Cn3cQlxU@bMPb)c)t=0a z=eKfcS0tnA$j!-<3-UD2YS*=rY6)pqv_c5ZtK@QGK>(^`wM>fKRpEHtsFZyfTK@jf z9@m0tcjTPUZvi=bQh~pJSp6b~-{;}K2=n~X|5Oe9cW>fa57@@;dI#p3m4kIfA85_(IZ|R_14Id2@m4Pu?0^lWX~i9dFZMB{um?c zGg8xlf1b<8&I&GAfLBVwoN;BP zV?JTec0IgxKcsjl_$8*`U?z|%>u0clK%N-dv}=p?TKL5?U?JXL1s^fzmQ?lm3l;$6 z?nGT@J2ajP55E{1T9VCo$T{m=3KuPMHzLBGl?}!-uy}myR*BG1#t#-FRly2JJ=;KJ zvk3yFwU|4$??g`xKYxMEdsu73)(32;W(2!+`XWV&0R7RoZsInU)X!bGRHAelXb=Qa zrrlP!Sh4%zmpqwSknb!YdKi7XWSR1sUeCavZG@>VS(4Lc-UqL5CQXwlVk(OCbgym;0$)n;>Nl1=zgHb4TBj@W$ z>G-V*D&9j^M6nc{OtX!Tlk1t%oK(q{p0th{6I-nnr8?59(w0}M4PSGj!Q?2qie z1}2K%4;f1^R;bimQn6*EQDxOO~!y ztF|DEsN_g0$|EacVq@-{J9lo%jL(pg)v8rvo1W-`@Gn)T$xOr;r)q|WTsA(TOdq9v)*`O%dcV0TaTacY0H+a z@CXfoO>a@w0Sxcm)oau~eB^|amFard2X_ZJI)_~x7W^-Gmlw{qEn@4sisBJMs!0_Y7OyT{^fdzcc|tm&wsL(M*c-*4Gf zG=KiB8#m#$RD!4_II^73XqY&C#=?Cm>e z-h%mG&udt^2zpR%qkhV?nb&W|G^tSTomYpC|7KpnQl;69V#eq(2M!#7_L;kI;WyuqO6aN; zE5BW|SUkGG_aNnfTQ<94iD`0LkH+;z3?69l3s|^yN4`SETsq6KQ>SaysSBqADzgHU zIR*em+9eB9S&<~zC&EwzL+;Cr0C#1+R9cukr>VdS;+#~!@Cy=&Oa{oxAPLDwCQ+3| zGvff|x)8xCdZkFeTs9o3!ClhggeMtUXg~QSU?&O7!X?Ho@cyhQhUeo&@CG~YjwpPg2{a)VkAEjp?W~ofL3s+gq*Zglg!O9 zcU5ZY_Vcg5qXrn5%KQFPkAIY}66sT{q@V-H4matKA0-zkP*88MIPgp&ZZlGue8VFh0W2JK%9JcoDJ5HZDOY~U+ipMDQIO3G;a2_Vz7ji~#6~)!#fb}*72CKrl zaO?3fIiXNUh|NV`7YEtc!s~=H11#U2rq29#mh1L8tIHMJieh_!RN9$&rx9`YbtR((r$r>}6CfUVZ(H}sN z+s?XBC$@oR1N+2VEI4`*Ql6S1PKV5IiqinW905H+Q}lF!E)Y;4O8}SPQj4-7hBSC) z7>GeYP;bcMz7SDjgXm?Jt>rb6f%+#1Fvr0c*9k-0VV4K>enf5vs{^S3^ds8{rN$P5 zfFQmG{USkR1*t4^6?q_8#Pz#G$^uZ4^rC}nl8AWZii(X8B*)TFWS@-m0Dra0lLkoR zB5Oz-3eGFUM3o_+L?}{89w$2DK~zYDh)yZE0Jj2U$+3hHm=@PEG+-<>9X2kDD_y8F zB33d&GAxm*Jd;aV&nrz*Lh=pNOO$$Dwow`fMH8 zB?D<<#DVhR?Hf=q1YaAMA}$@=yO$1W@%6VGz(*F^!S8xm^919xM6a2sDODXzfOrO@ z9V}770iMx<_6i-Lc@m@pY*@A|BjvH)YW;Nj^y!~Yb!FLbt>>~>1A{|6CM((?iwHCH zdheiMmdMjjaI568dixhBVmAqD%W88vHlzXBLj^OLvhmVq^m4F)Bq|bGlfhc2QPaxR zYY1qY#bc^CMUkc6D5&PVg-W9vsXa=B3VuY5{Ub#Xo53F z9Uf717Hz1OZsHEG(SLbV!nR^U%Sg!xFSgsHe0DB!~@Ix zeFO8#nkJz+%;eVN z;w55k+?p|I3Por109=Ye3nw1^){Pss|MCmqqFfIE{y3|`pOlDR$q?Yh!cf6Pg)=(IK|`O5n~GphVV}3LtwH2$n%Z_ zszt9CbybA3)VTo<0UI}LUcnoBd8qIRwezt>it(Cx0duX{Qb-Uw=_JlZ}TMFkG*{PGOm$eHh$T(Y0;uZv}93f z#kUd2hM}D}N0kGZcYqh7wkX|4hK;J9tKK4hqCLXl>U5ap&3^Cgs}Ywm!LHY+@t!?< zysTC_Im=jZ|If;AdJEn&3>LEu_+Zan>_&8W;SkgV6o6%%h`3!gum)^mAr8F`MM!$P z3})qJ(d(zpm_`g_+l^}#vMMYp|8R`OdKn-{a)-Q;o-9eQrV%w-R<}#3aEpb4N=Aqf z+JO=Q$4=+Hd-twfjifdx;q19z?B2U)(v+$A7$IF^kPYl51A=8T)<>-sO-^8d)C3^} zHN=G$hTqtStX=S}gb+nfbYU|q%%y(Fqk%O=%&1Z>YMV>_`VBaD{CG-wTH@V1o98c7 zF-nxOGDy6m=r5c)as0?3zz%&7Y=h!qU=e@kPUMwvG-nF~0j4~~b*yOVQd4RfCpeZE z7kib7Tn}EtWT_E3k$vc8=0KU^xLm*fv^f3Y1A$ZkzCc00h21jYDi!VO+o#vAo!dnr z7x5{=RJJJMCofT&2E|9ub<=oefAX>2p3MM7YY$X`?61UPdNRa^ns+f?$T?RcR#4kQ z4tN20E94zBm~*XSU8OV#KuzWv1{@xnU23db^9{$qQp*d)%bpFpKPbv~xqDsZG7NhmU z4?hM<$cNR0qLz*0ZS3*bt5^DwbYMInK^}mG9t(a`MLTnYJ!X^HrpQq=U*L>SaDhCh zV{bsbIF#>W(TCuO?0B~g}gA*$+E+PyFh?r^K zw*ArYD_)e+$zB#to)SYDlw3ReCm=9!GD7OOJW)=jh|lG+8a$s(`LuYc(8`r7LYwe5 zWh~@ypxaZEiYtd-a(^~uQt1*!t5vQn`YN{&Y}s~;iD_7;(ZYQJEi=`RDl4_YKtf2E zt!VdpLyxd-BC8Mz3aA;eFnYjIl}l7n>W|vjH*eYe$fffL<}62+`0ZlJJ}=fLBn+uh zdm<7={>$q|zvd9xLzN<@9r%f1EQ9MCE(6O51|jEorEpd*3MCW=Lq$9B4R|CDO!dqBC9k9(IFIqOOc)*e^U)SbHGj8rG>6u zz3S?f%lGcbeq*)_d1d(d3zyPUGhXQ2{>0$}>{Da$&U0}8p^D{84;nCVD8@vL|sTC&aB^y(<3B-Rd=KckkCaLx;ZX<(22V?|x|CsqNm~ zyONV0_UhfQOqJ?*@8h7?3*+f@o;!Q`_~Aq8_Uz7`yH=`N1)Or)GB2IK@axVUsNS91 zwLfy~=-?5r`uh5^hx!i-mtMaWRkd1;!a+fRP>c3$j~qFc{_v5v)$8EVBNHcmYPUOA ztzHumevw^Z2EFt$+v@=P=+PTiuV7H$rcIk$x9`+$*ramh$`9i19Y1yi(~j!ZYPRp# zDbU|<{fd>>A}-yFz5T7#tN+Wx*KXcivwGEvRVrJI=9^L1H*fkmHa50OrAmGJ_4oG6 zhc@@cjA`Avh3(k6=i#H2-o1O5Enfl2fwdnXjwVmx=-_v6N3B}7KIL&zr!HMeX#qw>)@|P^wg^euH1~@wTpA zv+DYlhzBuIUzjXIMvP%<;njRI&Q{9yoBQQiZbp`}g0pW$T$UXPP!?*}82TL`2N3=>6NbjTk;W?%v%k z+qSlE*Kx!8_4)h*yZ7oHP^ge#pg^I3oyaI;smF78|L)Ax6qD8K$iV|uDwXfkyWh4Q zyN(Y^q0Lj z@!N%HSJ+KMk`tk-4Xc(eu3Vw~rAwEiZ$`Ci-Hf%n-+i|{_Wr%@-MiPWQwMQ>KQ8Xn zk;9iRoU2)*YP$~Yjb4FUwrz7|r}yjgA}?+K{sXt~J{U4+;EMzL{V;#t)X!%fKXvBh z@snd-A1hKO7ontF6zKW!x7GkUz%zCz_E_Lk&BD-E*u3%d@nc&zuSfB`aQNWlS4XV; z={tWf!&ehNeC5Tkr9Ujl<7@qR#IRr1tTcM;Dar9&+qF4=>O|2Zg}#_GbLG+{=u!d# zGESF8DJ%c=MZ&#%Mz?+W?3u#`3{FjWm=tq!P}>&C@o@-AAq*9{eE(kTn8AY_S((=C zwBeoFJ$~?j-WVBvX;inc$SaXP7VD^?{g%&}k&&Fl{sV7~8TI{_vkCb3P-h?1X~}H6wQ%tfIiJTeb4sxeGWi zym;);faZ% zF$;46JP0Ka_PkJ$*+xEy7B*4r2<5z}mS+klc+50ik zi>6Jt>UH;HZY`Vf`M6g{j=Dsy3}xG4_v{(1YE0@`o zV^r7+cWzvlS3h^orgiH|6)E!lmvg`T{4;cglEsV9efOQ~m%@|p$GtarNb#aYczKQL z)H-+K7<&8dTTyJ8El8~mzo;}8sWCfGX@tR%59<1pZ%&>ce@zWMa~e0mf!5onZ{MQ_ z4u+N}^5)prap*zMrTNjH%?$ayeU~i$-ej^ozI}7w)@=u)Zv=SzW+o(^IC8jqpI)n0 zEcXxe|8da|KuK`^fpG zFoxUoI2eUuMT(xfd>J#0%!IpZmoGnh_z3phyMEo)uv4dR7A|6_Td7FUryqVm;KvV> z&hFl^cH7o+RjWd+wy#&u%qZUF0J$7ViTf^I^3I=^t>Q24-oEYNq0kZ~+qZ4mqeDAe z+RRfYPm`DZ`}f03^z72EbMuy<6CG{()G1yji_>O@z@Gp8yvd(T7|^fRkwg1Rlq&t! zTW`pk9!Out%zDGub3bp?y5(0tE}<;FyLax=xY@ovyPGv?OvfEIc<`XnV{IN?gOZ`A zj~s2@ts79txEsAs(dW9!pH67mqM6=h3+vf?@6KOwp8{|QFDr)(xfS~azkzNFLoJpe z!cXgTr;newdhF2pUwD&m&_Xio`e)zQP1borq-!Z>xEV=dcF{b!Tg|dHW&pLAA1Px8qhA8wxNvSqR_0oVFHEJ+p+B96lG}^b>YH)q|#T<5b zYq)pUsbfcblrA%Q^5lGoKzp{ul{s(WcjKqds#(2y=OII;PX4rb(PC|T_io<2IWbo# zQ#v{7CIn8OK7D%k>ZhtmxDy!%7fgySe z>W7$=7%jjK#shGjSZR1;yatFF95q_Ber)mv&`Z{9e)oPXw5bDwG&p6CD=j5)%CxDw zwr%lcJ}@i$VM44~XWzbQ)2LU+!WXc4d7Wl0NP>=e{P<~hE-qcQT9(_&MlwxWw8C=H zX7sWdtflMM^UoJ-_n0ED++r(;7aBLy;fgJg_ggWI>Q8H90PSK7Xp0or@3`$SEe~$VSY)m{O%m$s{CA#9*x5sFBlR zK_f3ysb)fQmOdaj-Ds{{uGp_T_vzhex;`k;l1HVum#*C`)od(Op)$hY&i#7<1qwkB zL&7N28j1^HVrHA^T8)}vKEzyfsp_>VrKB7E0-?9dlr6t&`>)93sPku{4nEwp3P~yX zif{a=Q>PA;qCx#;(B4dS)@#vX@9~q&7E|GNwulo16=q*7*0C%ym})j`YBS*>-Bh}I z{W^^sV~>Pvt5m5HQ}Nzz`~LlmJf=R3y6K-M!1*YJ@gJj&citM;qG7$O$B*p~kC4qM zJf4Duiqf3olTv~U`-^@r6D_h3OKL@lB4O=wF^e}26{LL&C-nUIOKLzgj%VrsqbV;s z3;h!f1$qF?053ognSqk6W7vF_5TewW09{mZH-=?q+>C%W6)IFHQ@#4}^A}HqUpjF9 z+>Q$u=r&A0p^LEE95A9qV;U&B5TMHxlPE_XGn)gDuAqccz1&V5nZ`ul+Oc!jxr^t6 zLraNqhF*_zk4MP~Y>c3DW#5dvP8Lyoy}iA48JVne0p$?0S0ch?j#H^Wc>>wJOlAbj zlq*Zb`%fG@d@+3AnX`K?hS#iF%g-<1!GlE1?ualsDIqgHp8P+4oczwnkz+@XK6mrx zv`Lf6BvGMq^39~2On4EIpsHxOKK}l&+tiZLG2~GDjsNA>{%h7`n^k>w7om4uq0y1)q^=i;u^# z40*)+6)DKz-sOIC+#4lJmOOPO{Fe)XDS~&BI5&cP67DAFhvYSNRI&oa0rHx0icds3gtLKuO}WJ;X%?6`kDa> zak42A^2@C&TdC@xQKOEWKY#SX`6HJuv8K0R{`_Q&&6ZeKgGJ4a$m`gLK+0ipWI+@t zpj7KiB|6fiMARwg7N*fTCvxVL)qcv!$&Wu6@#<@%UJZ|m#;-cjQUvT#uSH(rVOG7K zJ$sgy!o>#=cY1e$;2?fTR%OCi%FZEXh0oEMO8L8wXAZc52*83SSU7*s?W_3KNTDCY zZh%;rmC-q6K(Cr8)=#WjVP7vRJr|oTvjqfc+oA2vi{YnFowfLQ-?()%JUpCdu|Gli zqBn_=Ae5slQ3J_9L)tl;sS4gGd<5SmExJs6d|bZ3fP#7RuHUpFDJEKW>SC+%j_rS5 zw=(j=$p^QtFZlXPr912D)Tu+q|9;-QtVc;(*RG1XeBSBE7LyVZps%9i8w|DU*3||0 zFJHA1TZQDrkG^XSa=n>J=+c-GD^WFgN$ zBW5!|8zykjIF!-M3M96_bQu=S{nDP4ynFq|$m1v5wr+*cYTUT#?%jLtL|%%$d1c+| zB_s^&3Y8!nh!ZmvGKl#}y@vHLFZpiXm-f_$Th=a%x)yp6edgWz*VqXAd69iceTJ?^{aOxJg5By=bvw5#g7e*-+e0+|L#(S&Zy0_;$XC zL*6Ww;vAwnighy(icMr_nKCK2Vz9iY0ieUmOW{P8)4;GJD}>MG0hDnuJQO8UqzE)Q zH^DgilEP~;>qN>f`0A@$k&)u&g?52l<3XTXq!5ZLiDG=j2t$m!B_i~3kG@N{ZaaS6 z5qsyZ#moEHiQ`Ob=Lzty(x}0Y-~W(!_ujb^C)cfAO_8yONl!|cJ$14!GedGisxq@G zVqT@Wc1tegB>J z-g!GQ;hvXScm2k-pwcDH7JY1N)ZYF3$Tn5QH_bwmI)686Jx?FS@)mNf$sPWcYZ-^hL{-H>_5xcmF}xB5$ApR;f{|WzXIpj(DY6jVjq$89Hxo zQZHVrRJ%U?`o7q!dHu#kN|r^H#K#~qr&H(7XHT83SGmgF_yikj7}n)F{ixSpZ`88& z=pjR!RI9Og+43(}uMR3$h?oI7mL1XYsU`=NN!ek0l_E)9nXPaL#s>+>O{&&-f9zYm zhYc%VxeD7N*J;wMZj%yU4rZ(6?P)W9{CZx~ zs?}GlSXHk>N4&Lgi$p<5vn%ewWKFIoQe z$~6Ux6vtM+O~(%BFND`EU+HdK!q{=|e)F%d>sG18>M9^o77B?`vdLv&8Ig!Tr`^SV z(0TJ0&=n|z;z^&7{n5ZcG2vwAPgGgfxrp?|mXIO~RbXOys!0}AVL=WotKpR7Q+8_h zptLTTJsVFoGMWLpWn_U7pFwXC0tXRKi{Yh9%rr2GR;sjJ=gvh-m+jZBN8O557A#(p ziCbZV@r}3NIeGjn#@!L5fv>R+PZJQf!%txY}h!YOgV@?mUm6~bW%vk(4PGU^y)vPK#?LdXU=4u z@X531aNWbZ*tBWm$M3(d-m}02S2(0ZGO~Jp{GBx*TEL&3{y%ETI{*#;Y(-$vz1$9* zl=|h#iz%CI9pGTcU9r+d-2#PBoat+9v(CA!9CQ%IOFAGcP4HKpkrtf4Fqk4EVX>TL z@uZl-F}u&88rl=ClLZghu`$9$vlLC6ng+tr+Jqv-7fCk8VZQl->AM2_7>Rq_m`Tv* z^ZNU4SpMVZpH4b?J0AF8mdS3*OwG<@-mGQia>K`r8#Lx!wt%3w(vRd>3*rC&R+jdl}TEp99!BaMSDq!l1vP{m!9$e>P3*C&& z>;OOULu>aqs1Tfj!~17u*x1)CIDa4*XUDKkmea8CTfmfbT{=vgMd%3!Bou>=HT)=e zc8@DPHO(JsVKLhoAHc5Lupl<3K1vSo^}&TV^~^$RV2cdU0vXaHpFi zooxrMO(@_}VnCYF2A2M~PqfQwDY8ZmDK$PG*^UG0wP(-1<;$0&*fM@WZey_{e3Dqg z$RvaaImNUvbI68WMyTMy068HOYBum%lC%$qU^i| zl~~;U`@LGVS-)#f;bKJ@dEh(K2ZK~4v{2-ER)!sFeo}CQCGQ4ZtnJB?chO$t`k#D| zlV^O%`?EJWNy`F}n9&3G3`V~Ie|igGWOq7DRtv*ox|x+$%;s5`7Sjk>uS~0l61B2% z0T-m3kM<)~DKqD829-?$APo#8Zo&w#>02V7gFmo1vCo(J-&5V(;>t^Q_+}>&%r96WL>H5~cxQFUk`gfjAVoDKbKj@sG#i%s`_wnq(UzKT|$$E76Mc0SxMK z&&9w={FcjDiqc^{tz)^V&@xD6hC%Axk$J{eq#~*>r4h1X_3{<#Hf-$Kudlor1z1!m z%V>oPlW9!GQc^|a%|^yd7S@YO=49Z->tbXEGs40b)8Ln1My3kPi`XYPum@=&tO;wm zkmd|@RD+@5l!tiKvKoWV8i!N=Q>OKO zZA?h}2sD%2Y~FhdNi4h1@-W~pjmR}@VT zD_1BoYd{IW`=u;^4005(1lHt1h1~+MpdO%t5`-jIs=E}FJdiL$Hk@HKZ{0@fSi@EANpx;rv*es8 zMC7UwhQyV%OK z;+z*Fs48ujpS%G|hj_+smvHqIm}bmNAQ5@XQV;=T>RRHEiv8x>c|60IO#+EVfZ)^w zwiFNbf*PZp%N^~={i}Zc?nh35=YRf$8hGY_n}$x~qMeQz``YPEzvz6u>2vRXI;mT~ zL0sRq@#kq1KbB#&$@u1kk9zg%hqZ`wE!sAnn9eD_>*P0$S{gdtN<3=NN97|e-Ni4t(Oe#o zPR}34`z~D8ef{;?BFhuAVu#bLE z(>Zzk=<6>H(q*MHH#_*@E0ibj|>k4VW~XrMXRiW+Crho2tWoO zlnBE~%0NhYKr#c1BwkeH7U$%Fc7ueFhC!-Y4hbjul2BxkKe^xP^!eAHUjtgD|M}_s zV^W|=(o29B9Fy+3e)GiD>u-JV{;bIpv8Kq%OrP}8yKlZR_Tsh3cRm{bugR13YRj>l z?u3Sd-K)nX9#F+YDehM}r)LShB28V{|I#x+8M+thIK6@XNxz~KSUjFDX3pq2Xh3MG z66=>Ozj^)Iu851vWQvK4TQ7#svbj9^BmEW)6yFte zNijZy#I&=@k^D%BfC;*e37auYAwUjSLP8uq?pV9bWU08KlJ!AgIuOYZ>1t3dGL+B} z3ETr-Q=GW1!nv&|ztSi~X+!HY;YJmx0G3FFZmgt{e1nj-thZhp(S7KEleccpT)bfE z+?kP)5mE|)5-|=3<4uxfn?>c58+3m7R7wC@fRaxNWLBot)Eq#Uu{LZA`{IpL!Y8Qsfdqe3{Y@Kbbj z@m8vpPqG4%a!42kNR&e26mZgFQ@NGeqe6K<>LcMLGlJa;EO}fqC^e=~S6+|cI`5M# zQwsT+lQASaxs&SR6*4j)hmt(0@(WuU8jwgMK0K~op4dL2wYx>huBUT-GHEjG0xDRmR;|96@Nrxmi!y?AsVT)n3c74r!R(i)$8D1r z&Xx#Hf2L<1JG2+B!(_A`I&`RFl?tpY+xhFRW5w9hv%kKJVH`(lsHTQ(1WbDRv~J$gpj z%ze(E_htJIZT9Tgnf^F+;J|@KZ?83L)}VLw>(e(+iPGqEk>M8)?%flY5Z|~-^VaRU zNPh!_G&9M@B|=?$DuL zy?Q?_S#kI7{jM)|V`Ca7WaHuC+)%9I&CazswQ1+W+_qaNx!cWy_yk{(bA zh>G@Y+s!Ch;=rE0 zd*b)p+icpue=pSq%60A0vqsfQAAImWM~CjcEHa*89)fIZ(FycOtnwu#VDKSmri73n zouJxpsG>s27Y}QX_%F_X5G{U>cP36Cd=gCuAgsQG`ujQt$WrnLljpbf^x_CLy=05lQ z@84bnIlba<&&OYu0aRtWJRC98O`tZK8vdkw3Kc0?tfZfh|HyaWefy2E^S_++;X7}< z^YJHkqpXgjEyEx(!asN9(5EAZEnB+C-_QExdvD{fW9g!W`TTq*jCg6&>ScZg{lxd* z{pO2V70ZX-zI8qMQ6eBoUlpIOdi~X_my6e_>CJBICUexyo8`(?FvBgo45iD4-iy7Z zP(oB2{P5}R8LQ{a9M-qLJ|iRI*iL*w{O18wiNeS`HHy|l-VL}5XPLylWuN6$I zeads%Eo?oXCs@O&2A5iRKuST90Gz;IE+pmr6r6hUtYEdGdgT$Mz?1M=gcO-8B!oet zRH*!7R3rltB*6!q<)fXekm}-7$o?Cq^Q`)aC=xMKid0hCtJ&6Nd+uzs>uO{eqq z`RS*hSeEwi&aDHRw;YVT;p^j{^6=itV@F@?Gk`6I8`Q2bWI*pBFAa7XEaHHho`5Fp z)J%XfcDgUkV;unay)MFdJZ25R(!fS6nqjGu>Ve z7~gC>H>+t$RC|wI9Dg@#Sa%wJEj*t3ySB0Nkc~0%Ki*=`))Pk$weR#o+U;BI$Bf0V z;mbo`QWJDS9ng~j6w%t51NHW8)qmidqhB95c;JO|r|b_ODhIX#>JS>pL>7Het8-?~ zDqFo;aDn`DKm2IvnzfZ1H!V`A@QkTbjh}xOeeGJ|!iDGg{6%y6LJbmZskU23tbz82G`cqdp<&h^MLGJu6^S=eneM6&p8c-@N$;AANZDZY&mu^kK}U*^ZLO7V1G!@Gj z=I6w?&0wrmx1IqLMWeB1t$K|cw_@54qq&Mzs>UZKdj}QhIcDs%cg77F*yr+vGsFh? zz@IXw2+e{x!-~1t8Qa|+Hx*yf(A zTRHK`Bqmgo$`9x86;C4JI3RKB#_ z%5`;zn{r=;(F!YJ)JINK#<)(jd^k}LNMm zYfqg@TEEZvQ#YTV{twr{|I5qxA5QHb5FMz9M$24W%+1I?UAw$8Xu#;#MuE=EFz!9D zp9cKen9<(;0ps3z2d(zpspG78r0KJ>2}{Gx7B7?8$`l(7o(0QBlNUAzxH(1?*c^6@ z!CSX&clg}-?%ldge&fykyZ18oM~#%8rq`D$S^VDZJKzqcAVFoy+>E@&)S}7bxE>W< z7El9L6w1TiFzL@TaLE`Jh}j$Ca|8)Q6)uUxe{=_!1t z%to77kCs+or{F^#(g5YDi#b=Q4@(h;Wh&KNXvV<&s?aLuPUx(*9~$`a{gxt*7ho_YXb)^;g%xf8AvN>RkMV>9KkZ zQ-`=)H-`4^-L6&3UOmDRV{gaZi3U&MMs(?q4(!?K%uL_Ab31UHziD4WMXc(8aK zm0LAl&Xux)UWs}sJfpA#9`~MYzp_+v*sx*Nyg@Nh(HNY`@)J6<+f%k=Xw;2JAj@ho z_v+bW`Hw$5jJf^zPRx(XSNC9kPjolct6-@;tbxHo1crs4Mi;~qgP4p~+0F|(Ss!`% z+QqZy>esE)zjv?L%jfmZ?9CfiA3u1&X}5#zWDp+OZO=+cxR)A#7dzyX2ML^OBLHLx zeNx;HslC(q0e0)HMu*I4id_n@jU8}mQu4lCyF)@k#=bEI0Tv%~`|9Zvs}}!2Zm0?@ zV3?3Z@LaigDU;nC9>#w@b6V-zwQ4q~i!BK8*gRiG|v- zSn$x{eRpD`!+LaY(yT?q)oTn#GE-7?i3y>_OK1?p8bcWzX+V>UN;Fyd$VKsCqJ1cJ zF4k}(=aeu|&@y!Zjwwi$Z)t2QH|h%0adIK&P;nDN0%_@J=eaS;RWZxS(VYpUGN(mB z9<&efOMcXclbl2NaS5OuoNEEJZ+>c5h?x%u*U7h(QVz*K4NOflU}~c7N>;SHl2)JK)G7Q!Yr_U5EUhIwW6F#2sA-f-xtXX^9q)9bv*Lp}V{eRdy5Adq0 zZQt*{cX}nEca)yc1*D2}R1icE#X=Q96i`70Q9!Do^xg@f_uhN&5FqqGAS9%>-QRD_ z9Xy_M-@E6$_rBxtyJUTvu=d(>%`(=S|1suh6K7eG(}hffb`EI#!cHhk~hu3W^;no@umJILHPd5Z0IP~(31!3S5*o^DmIHfo2rdiQGg z##_qY3b7v#T#2<4yknLj-g+Q}IZqBJNfoOjfd0YlyF+__jMcP2(SqZr{$|$eW=@?t z&YJH8tvOo5=!P}cy5}cRmtA{ZV&VlEe1Y!%O zh^H{_a%jMFv$BSL^Nl7cUgPWY{%3t7Dpwlu<4*+LA*~}DvfO|*fT?q4O*_3I8Uj_T zUw6*@IUd}=kSj4hFd&7P{Dr*%$_xcnzLNFDa=;*u9zV8{tWR!j>FRYyj+qFF*_X!@ z79L(8Op>>#kW}@8g3ba>0$%d86aeMPp3(;1IE+&~>eLfaMN~-vFn)OsZ|W=$rGCGz zI<3g;xsShpyhXI$SV*Y}u=}GSRF(cid+Ix7fOUn=fUIiwABw22)P}MJ8b}A}wE|+# zjei>iKk-E!o?@JOI%B>N$U2ho@BDUs_bZJ}1WWA#)>#*tVs0 z8Hc*?clYLh`72N$&q?6l+Lgalt`gjb5hHlT#cZcuUzdUKXAnsQ9VQ;KU~Y11U_cN- zgalKNi9)$!OCAZ4XBAq5M8u+jik=C=-1hv65id70mFQ}d+1E2CP?ezL0IvYXVP6dj z@%JA3?f0b8CNo8PS|&zLZ;Od)5kEbiSWG9}0WlS-mXKb>>lVjqI!Cv`CdnKP>v1@^ zJv2BFNAi?}2Q6#Y+IRD&-r_4kHZ&|jAWF8Pb)|a&_9a$Bwz-mlU|tlsmgy@xaYiBo zM_2|KJ@-l@&S%Mquo;y5)Y+;@qY0BHmakTmyc?)qBxqC|i?z=(GK>VA`+BkE7@>XQ z{*A^)krCmZ$WIRiAR9rYqGTffwBQsJ6zH+$I?*!u`+AX$RBO#{*ZQ@71NygY{|>1? zR6sn-9@UWedMiLgeJ_;&<)W&GBf#hKFpT~XlPy@If~16Th*P_0TUhs?-51UzunIHa z8YQU{daBzgr8u$h1}LaZl*DcFXwajHE45TBHIQCJ&Sei})-^&&va5<9Pt`grb@3l{ z%3sJT#$wJDN_qfY&J{jNvmSU=U*+9X&q7H57dkx1t=Oa}4WIX!6S~cFB>Uw(o95Eh zgvc`CB=Thxf5Ji2FLW;!!K7q*c%qgO9v6i@78dk2Kfd0 z@YK}sMVEjLmi}{?SiXqEr?4@x$-=i7jOskW^jw!HM9%>qp_9-LXHg>ZMPDR8;DE$4 zmQsT6JS(BCEZRIgo}fyliJ^ni0rXukS1Bju=>S2kQk1^SH5!0gDqh0?Uair`~ z=yLAcwsqC=rRY;c-V%k2h@Zfj)t2=v@hKrQ;u}K6a-gtR=aO^E1&;)fO)mTXUEAXB z#1bqDa(Zx(4zMh1-=*8UsAz}LV#kdKk}1$jh~UiJ;wKLB5HYC` zhjFwJCvL2dz`Y31x+jn0H_n`Hkn~F;jVZxU?!nQ7djUR)EkxIhibFI93RkEVVbJ1D zej`q}MSzi+O~WoS1sd&E*3E*^B(zfVTw^CYxT=jtU@Y|0fy#wGz*z(aT%>A4rUk_e zM$AY+IWBnIk3R|5IbnE=NMSXMh;O!>lA2bP+iv`w8LuN*xVa0oShFjN^y) z(-fvokA_`_BXq$@T(IZr#S4e_>=6wMNGJ3|<4Q}m$rHzp8}~C+Q3k4sI#}sa=n-%a zNHRtbubTGjZ#%Z`klf_-(UvQ|h7yp8Tx`&pETAH-QMrd2Fix8d`cSLvD25V==Od$o zZp7$=ydn!rSGeSGkcU7Ru&_3YQK#TD0=j$k@@0?i-XYZ&%$ROKQqWf30hNLZKEPdM zu;M4)4VD-*1lyz`O0u*7x>@D8@%@+NHE&Py^@(m z$cW1*R9p1!9}Y!4ZkNMz{tpH6zvffmzo&q_^DoqaPa7%@rWDYG0FC4Xaw*6{J_ebQ zj9FqMh1r4=C=-gvG@uiOOiE7W;=2!D8OU|5Ls)Jh{It+ zbTql`q>E(>dB96hKq4CjQ-LB1g9rjNZg4@g0Elc#76c8Q-jX|2_F7?z(~OA!T5|W{ z7|1R(R-|81PN0XC6*~KqiR&RdS}E8QyB;t}MmPYG(O~v8>YNX6UmeutEuABmrclwM zHzC^)iM&lEz=|yBM0uqvT#0!aB9KYG60-dsP{idD0`wk$U6u$*C$V;Mg)E|=R zNx`VBdt|pwp)pTN_$kD8nl)}TG6s7tUNrDDqtnK{lKe=xjYDNzfj*E^mLoi6s4B`8 zXD-{=^B^qZ!R}y1!?37UD5;hUByE9+us`VFa@aI3tC1bZ7#lV7hA=nC_lqA6#NjEE ze@Vk8mUu=A(DNGLVd-NwqNi`){!P{#dvP%#0 zb-`;tfA-9nix*xlS0?uQ)!EZ$#>K^zE?ZW%0t7ppjztUSu3WL~ zcFeWPl`5LerXvRqUO9g@J}z#-f;ms(A6BbUY4e8l%a$x=JDSp^OM`|mnDmrq%N8wI z6%}>u%9SF83;X#8A{ee(y4aqRxog)BGN>Fse6&pIlD_`_;Mcv_8>?0jI6BErAn~_*2XofqLwZ_arBs%#e(ZL5VHZ?u{jX#R?a6s5x*BLTCj3O)a@JBs#L6qPd!6UczAE&{JEPptRpGPzMVT7 zHfmT3mw7#D*43&C4k~t9`~ADO*F`N4@ba2BclOmw7unjM46sew`2erKL7rZ-YU}m{pJ5b-(ENt&!k1*wdZ6- z{W9UbcRI#iyAoisOd0acnk9?OF6)Kkhdyc9+>yi9Eyh9by>s)*8D!hZ6UMGyv9dy` zQVXX}nlgGMpetrI_P5cxRxX_V>)_98R@mZ?wYb^o+}fl2d(lf5mMc-@z~)UO2M1@d{d#;f z?)%|k!68rX-5$`f-Tv+CeYDvq6DZBP038J)YLD*SE?+!TBrNcoem#%u*^WvY2twzA zasZ6x?LkJR@TKz?szpSi!p7ZY`J@Sb-hBtT`1FY*J-T%2*{!p~k$L>&f%b1UGiWl% zHn=Kk$>mFDA?)g93ww9#eD?TZwoDk*rNhCU8`=24(*XPxxm_GO(7RZzQIjWr+~sYg z_;bfk_I|H(udeU#1Wz41+NoV@qr-}j+w+6>qZchISF-57t?RxW{5dcd`P!i2I>na`+U6-%encdDu_ioJ}{nK}Y2KpEcOBc-R^YI5?eLf)A z&u8wenG=5c!K85{#>Wll{s98D!7SF8YV`7s ztXwsj zy%#N*ueX>UKY4U)>$a0IF-D8|(c}1&CysV~r{nU)3+&k$2Tq+rlm5l0pVh5fclnYf z-+%ixDjj6u=ymJSb(oyar7M>A=|3PNBW=&hl{1$v0UboF)M`tWD%G`X*A0u83>f&i zmyiFs<3}#;*}Zt%jw&oQbk47b44OJ=!uIXk-|g0QWmI&ZzMqmmMC-Ke-@Vh@;>8wg z*Ds#jyK`I97O$%X139ei29q}nDHa*loSck%_sRLEte(70&d>Vv?%cai>$Y#a_j;4} z1`X={#g}*4JFVd>akp<2FI!QJwgO&k{;P?oR&5H2Vl=u(aj_>3A8yp*P1H|H#2UAg}~z(1wH3kTeY%d(&ovy|?&W-V|8CLX0)tvbtR&A<+M>D>8?hmLGpxmF0LbtTuW z+joHOz`p(C$B#?9b;CQ@FZ)?Cawtk$ToJzM^XZ_UeyUyfl^k(!!$QcOvuyZh!DPs{ z3YDu{wYpLjDkJ7a)~tocZ#fc=WMvTCax>-{nt-;iHc%oM4QVc3xysTnqGlbf*$cIF z=?Ya6k~56H{v=vrr`@gFcYwV&Z{BR&xFI`$GXem^vUi}7XqEr{5o7I2{Sw!zG@r(CjZ?lQS%T=n_i6?Sv>ojP( zBYHimZ>tE2;+xFc9v#KvxpNmTM4^$PbJ?rctU(MU+phqvN=g(rJTIsFVSGH)3NKhN zo3#uOsC8GZQ=1%HCCa_T*sIsAYtkmuqH?yVg<&Wpf+ifE=AO;Wz*|QRjQ2Uyk^}7xb2lJSsJbU<444G-okqP zMuSGOhDW5<>MmZq5Od?kt4*3}NX)A>7AaZQr%-Wmrl(OF0x46n)Z_TXm&FYn#hdU3%qFaK#H`CBfLrCM8RQK_nZp(3a+-Ht z%KXzQn_nQG0=%y;AQ6~P2y+ZPg8*x{18;$tm}~3`#?F3Kszz3=T&2&a{lt3?RM6{F zpCy0WtJkneV>`5OcVy4bfj#>Q=rg}`y0_l#Htn}*jheL#E?$vNB50N|soC2)a2@p*>PGx$s zL5WL=M9uj4`J#aZfW-krNdMQcc0ILQo?y>Kr%s| za*i)2o73CGBZRS(D_0HXJ`-GBM&j_@k12!;`=H4 zc!^Fz*tdYIbeQ%SIBFkSkr@FU#3@v^mt=>H`E%!yL1p`n?L~?dY0HK*Y z_U>xazCHUN1D?1Wv->U~he09Xbl}$Yt6AA;)n9o9w3zYB__T+2Zk#&0b@j?OJG{-t z4xzz84DIfX7*+<)5}sHJgcK`M^hV6hRcqEMj~Gb~A_{)o<6tVe2&oAAX5S*kNPp;w zS7OM>$>ijc;X`azBe=i|kb&2@0OFkv0NZe7;fmU>g%J?(C>v(7o&C;D>ksVR1G|FI zfE_kUasERFmkcK$IaX4UN^>1{y?FnSlhF}TvzErsf92{`cyXj8rSIOg-=;A%YuP6D za`udCduQ#8wWLfl#sj=tJ=^!bcmf;3Uj{jweJN!b!Gp!mTkT}3{Ad`yq#2+Wuma)h@+9+*dSgA5iI=|Ds zUHg^|8W%2625}G71fm~m2b<3H>x}t+K?S6r{Z_3C6)p;c0sQC= zuFwoTtWc4WjYt=e&;slV2Z2;*N z!5Vx<+o8IoN9a~>qMq#)tS;bEgj`~vq~}WBgq@)-XTd^L&To^)dE821bKo`5#!eK3 z0@)F!`56z=gxY$>|&b%2$XfQJ!q*4n6fYd6WhswYq)q@%hVF zYE`JTXvNCnRjQ&ziCVep(t!g*zWcsHWVJWCfAYl_gSh+jsT1bHMf^fS$XGjiz~_7S z?uFgLo!~XBb)W%idzg-iQL42F?LjU20Cv+XZ%4j@`=29LehEc^=g}trPI53OGc}sS zv1wP5?IDFjO~mgpBmW>>2#oSwB(VSI!*jE$=y@GX19C?^d6FJr(v~mbXK~_Q&2Km_ zyPTP+4XadKv~_FA(q&Lh`1l71o+3pO<)P7}XJnxV_V)J&ddy5J7w%_SBvXNoj>EM^Lo$%i z@PeQnLN@gibcZi9iTXt1W7*CvC|JM}38R~; z2ql14fSb0kXA4<`#%j=ioj-XKOK~^#L_GOb=pamq8IYf=>``G?5Gz!xcy;$c_L>-dCDKzW|Db2ZvGyb*i5H7;pdsLGzRm zZ8-H)2{H7rA5W;g^ckIt(JQ=AQRICI@0L5TZi%>!SPGuS-Muq;!uXe7uF$YSV^AHq zP?(1zK&BfQ$EK7aAwg8d7QjgHXovg@6w>+lU`Hk8A--`AESDO6Kwu!%qL{Wjoc;kp zO3;RLAS$w*u!J?QA3Sh-u`B@AG&)+%6<;0u;1(cp_EoHzLZhWNo#y?u@}+T zpqDHK;HgW)4#lu=9uEWz2@T7|nMQBH?uvny=6I^2r#BcF%ZioJVc|uZw{0hHg_Z;? zzCwMcOVW_T5F`{61ryb&Jn0ewsv{1QA}E7kAv_LxBnY2>dW}!$o+v!OYAdA`S`bR!mG7;*!cu;RHXCr#r_E-xcqKe}vTgI)$cV}m$Vg7L zr6y`}vrJ|$WKpn5dWASjeIiH8WX?>_#DT<%Kp>a>(Y@Ol&youG`;*-RNyn^lB|naL ze5fq{CnNpvqtT>@%&YaLJH?j}Bo4+;(t zF%a1pcWHPA;g>X*^2~w{w}F5}Ff}tHDZxu`U{lIlH)2YaC`K$yN>YL&JKb%y`Vij2 z4si67u5*iG5wFxoNz=rLUIZyL*BXEO7V8u*FE;Jen9SPj^ki#Bx-}zBW5-i5&BreQ zu%%A9J2(4I%(a}XoWQ`K;E=G0>b2oipgc%t&Ca+Vd)3d|%fRNr_-3&7LEu+OX0*24 z?2HT+3bbY+%qk@%J1fItwya;dyjamcI%!>Q~od3w2I{+Z?R@+w{*5-R z-+r}0$A%4CG;MO}!bNeD%gt?Bt47nv%FS!lXkM-Af;n>-6hqs4{){6x=YtL%y1)0{ zz<~oj6(Y^CbimuX_a`5=u3NKn(?%WMXqT4qOjaIbQv!g*;OA`hTGO4|wz0qd>JBjb1L>t;1W5D!jaD!y5EkpdC$$x?ZQ*v|&S=7A*yl zUD~rJPc*MmnPAtHXNlv74352d9U2ZF{(Y)b_U!D~8 z{4jEO+gddb?%ylMR3m9kk+Pll?%in_QFXw8PpSFp#fy_hjL6N(L~GXKwWiIhN4AKJ zs8_qz#PMT!j&b+zwy5_Ce(3U+!9@R(p#2}3srk+RcNBQxfE$S#&9g`fdVSx~llI+w zyzkc21uHj@nIbnU@uwjJhxG4r{&LKqA4ZKEIozHN$dW%$Yu4Cfubx{pZuF8ZTTWiS zRJ%#zQ6s)ZvrRREwM<$zlXPLW%w02YhQ(>a8Bk-_`nug47R=Mwa(qY+qH#5B()8%n zSRcP&Sv%`AUre63<=VCHzW6k1%A{-OFCxB*Hdr1O!n|(#+SNW9Ysa=NQ&OM3-R+}8 z58{v9x>YeEa@owuW=FOOW1ic!WY7Ksv2pwE#P{kym?b?5`T>3V1cU_dy>owm?7fkb zrn34+cn6~hA6gT&m`}U8CB|BfjqE%FM(*h*lkq4Q_&39N)k~ zjpkue()c;^j>g{GcJ0QKxY%857I~AtL*uU9w*8UVd%N$Z?z;KJU^ZEt_M3-ykMGy> zmlaX_FI?Mk^G@ON73@a6UFSq*cW37g{^a9thJSw|=63&4W5y2X>&!?5w($bNDaFK! z#F=YxXB(W^iH~CYc7Lbq=Y2cB+chY((4hy9BkR>i=4U7SCS5uoymocY>ZlVNqt~xk zOa>A(7HBUZwdjMuU-7>u*^4Ii{>@*9f2}pSh@KT@05fwyPeXE2N~ibVKk(r3p*!~@ zs=Tyx_9QE?ZW2Wd%+BI=dfRd}*7UXWXBb>IFI-3LZi`m8e!(n_Bg4mT^K-iEz24;T z^=k&dU?+Qxx^3T1{CUgyi=Pi35e94?`HWDc=`ex=0+Kr6#qJ;|L+YnwLNEFHjENntT)TNu&Z!F)i zVWo&jJdQtj|HIhhC(=_>xJ@*w2wIr+bjBKW>!01X@80d(3e8=qDamW+&x@#28)&51 zgZp=9O`A%vj)jJtPCv6HJSeD5>lWDk5)%_yfr{uWYH)DDId06)!-fsR2d;dDN}apD zhmjCpkwy()J$>#xBft*N1FxeMt{ynX#Ubn3Wu`!-R=0+GNJyQ7KLLUib>SFetmGkfv!6$OhFcG?}s z4>Gkp@PQc=j~m%V1d_OdyNz@Svk4fBRdtNzEDO~F<0s|ZQHgZsuQZgkk$hoi;qd~V-b4-1eq~?YNJ-HJsQH*f_%L(@zI7No#x2#BbuztPMy23 zDCpFwGb^97C(m*Vlt#ycAyFi4;5j}0qwXIxYT35`z%Td>X?O0_Uz46Zf+iHep5gCp zsavaNu}T$@YQ=gh^$VFP$y!Q!j-0D;e?9Emk9+nkU$Fwblf+S;;X<}bRVuUY;r+la z-Kb&1^Jg!hv7sSu#5l=co?5k6>z~-a|G~Zc)?7Q3N9tcWe?iq(>SE`lmH0ci=gpeQ z;}?~T+wE;K77Ph^vvn&vlJGQ%EByxy*uHW_T-*az0x#Zdl(#DX_&+V_*fXMPkAeXHr5$Lk8lKJ+u6cXKu z_Z&7ijVmrTwoHZc{SAVKz*>c87QuKmZBeQJex|s^RBkWga_n zq+Z>IG-J@4ub;m-ZT#3PCy$z)_L__T0n=%}$1U3zc&V9m{5I&Mr> z!ef4dK_)%CcYNQju_L~VC*ZzAdu}A`MeLBG*_RaIjGfM`iC+FVA?f43qzr?7k)~0c zCp;5_A~>ZmNEQJH_Z@t#aWf$p?+UyKXH79+y30pKRC%@D%4JKuv@X3pcm0}`B_pdv z)~>@(_(Ue&j$J%@6ck1o!rG18IQuuQ{ic84Ds>yZTE7WeChQGGDpiQNepNW2^eF$x zr@%j_zzYZ5%uOZ~vSIU9Eh8hUA3C_dW8((XCr@G-&E#s_xG4o@Pnlx&_Nr8ge1SqA z@B)KjV!~5#6+o4&H+XsZa5XbC6X6{X29k?Pd>}?Cr~Bg`pRAuhH#aQ>u$w<`PMgeyk|NhXPeWgm4k*aV%knlNtShRQV-}5O}%*)qD5Kv-R z#0zin;>Aa{Ztc;xKah%}^xOUV1%-qzSg-&c{khX8*||OONrF2&d*{xbwQJX2vT)IW zPy6C-MzzR&4!aFor=M>CoB$UM|A0VEZY~H!?F>-{g+rmuzA0*DNSTs#Ua1AUqBo+p zrmqrwD7}Id6DLj_G3@*JTd~2x!FES3)bU&w?E;zyG1rQEn%)vwKXl08ppXJhTeOA+ zRH;_I^&4;8fB2}^`(4Sqf>tOcjz(c{q5rA zQD6NuJZi~Y9in5dwQJ|jy+0dp_}c9^-s$o|o7T@BJwONoCV&&L!=9U|b6A;$+1Z)Q z=-|)-Tt`bva5VNm5^JDsY0#{hcd??|*R9^ab&EaM{(AE^)F88yS*hGRaF7r~ltwpf z)aXMO&h9&T_0W;Cb7#(y9MG8EnB;EV{kTV^%9a8_(3OiWRqHyiW836!zni{lUBTid zN%)Tw$-u9_A2i~pO*?nhXz|*TMRO6W36K4Uyeo^#0=8ks0aFzWHV_GA1t%3k9pv9e`*^smSh2 z)~q{qJ$A{K9WzFcKDuw4$(2n9p%31HxWT|8#OoP$;B-v<`KL1{kIY%T%w_R*8VxmH zt>1IdkWtfSMMp>7JbvtU>{awKVDjLh-+k1t|L`#rs@AQyZPP}PI(a~Dou9W~R(b}* zmmqt+Aw4}cAixiAc*JBb3b)Ye{lW_N8$5W#mtV$Qz9d4vNc|9*Trpt3ff2NRv*Z4w zXAfV!RiJ3;&-)M5yBsE?F4r#JgTf6}gy*+Ihm?D%^ut?scW&DB^kLl1YgaE{xx@+! zDxr^JvE#DY_wCp*Y3Q&it5=7WE$6Wx(f}pso3K8ev0&*0D2>6KYY~a1E(%-J9Yll>0@)IPeJ+z*9#Rb^5D@!P@YEfk9-RJ zD-?L)fLrM~1&dK}*lcWx<*HEeW!uw)lxNQl?Arsb_5QT4m%r~90|zl@&z(Ju4iuML zhs$2HMB(dKE|@h=qsD$W_9mbc92Sn;CAf^10SpvL5Zu!D>htNmIkVS9ul%@2AHCUI zWLg&ZASPlQksUSyt-*1mHu2F84KLE5QR4&q_H#2@8H6-uIyo_J-@Xf6+HJWAh=Kly*SX&E_v8G$m}%9a*)a$B&{&(c(rQ_HoC@*&P}h3I--5B)FZV zW|HxSmoI<+`gIG;!0Egbd#l{b6$Iu0FrWzZ0rgy(f&~jLTDIcw#?6x_Od$GPOst|4 zFwj*(QxG*7gvn%xs9YtWSn;!`P9r$eVb&!e6jcE`WWF}K9g{}?xP9ZQ^{ZEghlk?? zjmr(rb_@&!N12i((A(U(bH_h0040sl#D}CJuE7xLTZ{DH+qC2)m zTtpA=+40pU-KQ;?U;mXlREQNqSc}k%fs3((=cYHBd-Uqb>SyVa#l1fH*vLLUP+Y24 z=#7JlIKyBLmxTcO6YK|=0S+BLzb7R@-?N=b6;`BpZgLW;Y8?AowQY6f z;NFX;jyN+?=FObi41*yq+r)xELdoOr3EpNOFw>iz-br~7clN-cw>ozvF&^NG2QhGg z`!UWVi+->Bhu|8&gYXDmWlNXNc@USKnS(WN|IY10dv+i3 z)9~^o3nxCflNcY5F8J}?TSxb7H(0Z-o;k^G2_Jm$0UCtn&70d3pRQOq&yk+GZ2lY~ zZHkvI7k~He{yn>Kwr$(K-O7b?Qy$z(e)NERuy1$nf)W<}y!cmuQX<5%ymytWP<_;_ z`KxD4TfclM`d@-fP@!P{B-awDP?F>B+&q7J`jlVN?mu|zt#=a=GbfB0WhbmuU=G#= zTe40XGjh$cc~gHG52YVJe2|%(NDA_UJGLg?yOsSkVfwVGUg6Hh|y!HDZFI^t> zGJ^R3>_1}EbZRGe- z<%$(({vI!nuql8Wi!U*jS}Mb3M&F@Rm&CjG&L2P4;hoM1;(DFr%#!Dc;|e-` zlCZ^V>W7m!bOVp88PC!A4l0g9Z}$x~;3{_@L; zH6kOc)$H-rSKT{x#DAoy$Qy&PM`KqOG^(7D@dTB z+w4uwRS|DVlO$$NlSCgHOjCaxGj-&c!jYBx3?AIMTURiN+A!JiJPH&D(HAIi_Uzd@ z^%|{Nzm7Ube)p}&c(h%*!l6TlQd3jE1gr(fT(7ok_hzR~BwGqER_y!9Q-1pLi?Jg| z8N&+Cm^aVp9LbXz8s^ zp9}Dsy@Ah}a~8B|-Lhw|-ZSUUlLZU0Hs~l^+9TU{wcWncT(H1fIHr8~2}+4g8`t|- zd|2!tkJ}}wm4v^dcELFY>jKFXrB1#1@}+ZQKOb1LY89vl*++c+jX<@=;9Io#&XX7D z1}-V$PD5NM`YF8;HMh~nxBaL6vNO^H!ix|r%1cIb^cSXwip*p(5t!=MnJ14PH+jr2 z#i~YrHsqUkx^#nsrz9uFUAlr6o@!<3mVe|^;9sDC(hdF#<^F$){MB-mFc#LlnC#LL zlYG4`?53vE;_u8VmNa%&#GbU&jNrf!@wP_KtJkKbWn#@k|6tZRvU4)6cBhYTAo>x2 z6d(b(nZzXULley{b}dnbXg|rnZD0*8q8Y|6aks#YjLm;h&;roXK6y59YjbFEC*cJ3 zxKcb&7*No-wy#+^Z}yz%tvi(i2?D!dR^IxX^|e-?OIK1;0)zYsCW7n02@nQxfS?aa z&r+}ol8gfd2%9KoXWM*z0%U=&ab={X8qF4q1s@Dp32F&DO9uJ{bfRR(eCN>dKQbia zCg+MMfS>|xgInZV!5I!6JnqWWhQG(ipMDN2;D>UFN^A}isz?$%wq&HMz!v<5(bHFH zOQT7AnvCuxAT)sSVuD51$Mp`0G&A)Xo{l(>2nC#Yhi7{E5SU765Wec}l*F{4uuuX? z+2WWmbFqJ-$DleTee)k}DcX?LRuoV;Vxw|`D*(7285RYKJaJq;+^#lFnv5SizGAJq zc;}21^fVZs<#y3&sDsBN1y!RgDOBC^gSykB8>cY} zOBdz3=RPrpN^kmHelk(F@1onIs^z!cT2*!9wDNz$K;<5i`tWQ4l-*LJDv$T23S~+`R(>@d>~p zW@Ea02Uuj5;{Ty0CbAmA9CpbG>XF^31L0FoHfl5^pdzzV@ZP>K=jkn*HmHtwjdhK!4@^G3K^8os zN#jIZig z1h_HibBQk@`r6G_J|b;H8M1b;+?SOAJ4NuG0*d7EGZHoKC{QrCb*omu9o)-gGTNa7 z%MLO6i6NF~B9NP%=~25AyB`u34ts#H$oB3I(bQ83{xUv+p^|IYWXQ7G%sRc7AGy6m zRt28vTVQw?=#I)*P+RqcRgEV(2_sUEf$kHOBg}!Ju(*L;K`I%q8nsLK(^tpZ`!c)$Z-U70-Wg2@0>s?$UpKa@PDAd z3ny^{$5^hILM)b|WaLK?JWxsII* zOa_A-4`O4YSFalQ{SPQj_zkhq1lTaKrKEfnEZ{^K1alL~8nmDykr6QoO4<+(jB-p{ zRfa?s^n~~qu;xkpqtz=G<>cfH`)MroyGaGb77b)-rH2UEAROHVz`$rw2GkI|7EVAP zsh816unFJA85}8G@W(DVh_@f7l#$Lk?I0(Zqb!oL1OUb)C81bQ)jp6OA8T~CRE~Qb zIiX0FD7_Wt1d6$>Y{V`(?i5}NB?aqYCm?vP-6qL7=qO4Ckc}M}kn16aSZGC|B4V#Z zlM1I1YQtHeo-O_1y}Pid(C{##=_Sn|2@{2z2({!9Kv5wX3kUUG8jw6+uvPVp@^XGL zc%CE8GCJWTZsC!F`3hfUDNJ+hzZQh6w4t~S62AUCZ z0k{}A-J`Q?l8G0GC{O5*(`g||sc;dc91#~1SdS0^*_Lt8rQbOdH%Pb=gy$!SBQ%zt z!z74x1nOvxjptu|rQW-5wr^LvX7~5HVrHaO49f!l!j`zh!$AcZIn;v!F8Tc7m8K$f z2IrZQ?Cbi`IT0sVIHw6$q#acFL@2B z{we+;T~HT!7oe#J@u`8{7)(Yk>dgiy%L)$?<*{&SKmW+5z<)=9KfLAtU9tZ`zlq0Rfl+BAbAr?WHU2}VwfiB@x6!V&U`uOo0e@m*oZb1&8>h1?Qs}T-Z*oCRC$Uh#}PEev(7HPRnjw+ z;?VZ>`GvzHL74q=xr$7J!|i_?S8)A z;YACH)2_>TXxFZVb7!A9b*5<1BEg{{Wc8adZK}VI_sSJfcAKqOk>d0P+=rKdzw1}6 zSRJ+ew7s=fFJCNOx)J;xn&Q%}w`VV%Hi?YYGGR5EV|#ZT-L=K&wx2k-_osfn zRxO$DYcWjxW>EL8??f$H80zcyU5}n?R!2)9kI3EH-k*HT2DgBCzjr#`yc&b+--_8& zhxO}OFf=s&!TonzwPee1w=FyBmkGVQce@pH$!5)F!LB3IoeLS+{^`f>Px$WJAU~hS z4{m+lsl(naTfIpWC?XgM_wjWRxDv%QBC6ZHaQf7*-s_y5mW<#(YCzAiBffIka$>K? ztX{ep$r>!mOv_j?aYA-VlHQerO_2oJ$eY}K>gb`dBZrUpW(cW{4(!<4xotDOEyJ1l zZ29=nw%jz*5}I80h0}k#arKIstXmqbx0lKYh2RXT$mTG>jBDM}gPs z{Wgv3ty>jcs#vj82X?+)Co<{o9cyaRxF5bXlM~Tl{c-5vF+U75X>xV;^a(>gGud-T zelvLPuahdhR5~H6loS1!-j!Ma(^HL*6jz1SJu7^;+33S%}Eu0j-QctSp*T z0cjQzJ(qb|j!~g3lb|~*5LQo8Sh~lDRye2TJSke5L7rFfn7=QrxKEi#DJvkS0A2kr z=P*|>@GA8jYaLmCh=fmL`A0ql{y7DHSM~k#qW?2Wh=`=L)fm4%`k-sSe*GM_jA~_G zI(h7H+qQ3@;U--L7DI`I#U@Yt5EC$3&6Z^_fgPfwma z{_eZ)P?(uHeAv*YO`AfcR_)$6uz!EqQpKl@{Nd**zqV=LnfX~Gvhu~Vrz%&iLc5(h zcKG_6VeACM3LHNRblYc-pSZMV*Ya&!Dpe?lQ25o?Ls<~)-nsLWk3L$nXnvnQy@LuC zvH?TL+d5rVM*8fZM-Be|`>s7c0)ZMhE`X=l$;8h_pwKvTxr$+}}~$ z17GawpfjO6K^jFh<@|ckmjyyXTeN5)_zFW+QZhc*vJJbiG8rgHimZxWif!g zJkqWFF2xYw*$ETgqm0jzJPO?Bx_Amof!}YJb7AnXNC9Z-RpPYA%&Vr1dW$F~CDmVG zUi~i@s8Ea=o*!sIPV!t$0D37wb7B~fYWYYjcos>naQQX=$fv+RrGUJzFVrD|7>iV< zna&j1;I#~$x6>3-x=z!$2dSuSAqD;#Z2#?YBjgf8VaPGMvkclQbsM_PfeyFj<(DHW z)N5$=^UE~oOIE4y@cu0eIEinB+fd{6Rw*W5G|lzv*FST7fBfAWOwgg9_pe%_NR86P z*rVn2nUmsPr`6SM+$7sz%+;H%SV^5`qstg~9wL+0Xwod#;G3=Ss!_KQqFwymM{0jb9N6qQJtJZ=6n3Jt zyGRP{%Rpo-I)l|@eyLFt?~sByI$!UgB8AIGoIP>IfF}crdM7|HGCpd5E2?sn!KyWq zMVcUf!AqeGodIo}R=uuX^A=WvC0B2{em%Bv-g4ckk>af5cYz=``{7xVNK{0SI zh0qcJ<8-srDJ4DVD$+M%u&AeyveD_w$f%9WYMAn$SR5oZqj=Bi{{S4=||MA$>b;|QWE)TT1)=lk^OQ?oc}IpXf$4=P_?q;F-S~5s{&!JBg-|+0UdUV+AK# zcTQ@mcTf=X<>JMQwd>a{UZM=M=hwOOs@1AP15#m;>V^C(~RRC+v3ngnJrp`eUv>3aSzIreTnRSF%ROHXnsL~ zUOwLO@$p~;H6+A8khMEd&lbQgHYaeIQ54_=;&x=9U=f!RH=B{i$HYXA9HBK@5Tw(d zJ@fVTVJ9ue(*y*0QLPB_iJP+n)X+pqylfofAq%#7at0T(V`VN*AkikEqC z_imwLB~XSFe}C}sq3rDJ{-1sEio5EmeVSf~jAMLdVM@UTQi#syT~ z0wFJrsWfCnT?w_QM%6=4rB!hbyTbn=f^^3VUR6!1LVe`^zet;$700K4TSmc-b&7vtC<0eM8;8QhIuYs&6r znNJf=?%jLy)Cr_JBic=bDdl{PWG0zJBr__0Y2<4{?lTbof))V>m07<`bR<99wsq_E zQ)e1AX=e5bc(wJL(b22ewu*4)EyO^o{1u=RaLn|EQy>Gf^Dat(n)T`-G0psCTyk>RB&Ry9(9I1#o=5*vBJcFO&72(zvEnK8T#qtZL|9b!W<%hAc^JdL16;X{$ zNhJPurl#)Oy(={#e#yMK%yGB^8Ab{gDRle#WqW#(J2%6qu|2+cZu|PEoRmj1ejTsz z52#e5c42IrLBVTQE_G+8E}Am~)CBOr#n6zDySHv^icbw1J;6ZlE#Y^G!P5I3hq!9m=5nk1STOkj1%G4BU(!TJ@5rQiE}ux z!@%T~peprl2uvfGOO^uAS*x)kdKxs=JP5Gc6)XS%KmbWZK~w~?3ss7fjX$m z$;fS7z2?XtM^%2cp^`jUag)Yzq}fIuX)3!O`9~^d-w>k?QHeSyHD&4jUwqYPZ6L1h~Th-lb1zp z*tF?&+?G4M8BwEV|AB)DamiuUA^jQ*BL8B*5|EII_uKRtIk~n@EnBy1)Hpdk>z7}r z5g8v=xJZYOdwtQf*XyIt`YBB})~kT(w-? zhCV^z=oaQKUev5k?OJu~RjFR<-7fE4yB;(4i-Gs=-(!FbQ@reT`ltm*^j)2WkCx_)VKf5pWoZ!H(?~I7wTLFOLHt7WL+U@ zFnGNT5bBwXBr8_6Jtv*|B7gqaNpAOa_<3)CFO`4tZ=?XP-9LNm|KF|urE--Z;Ue%D z#S<8vf^2pI49MhD6IGU(PFHGLG8+Ye3moiH$SPPgry7mTEy;=4OP3gK-gaJVuopa$ za76S7r1sC0UE57&1az)A*{6ymu1}sm!x7M6B0dF~4;a>y-AHB&cY)ucjZkPTJ_R;5 zz9R9r+2rMLo1_v}>S(O(c0%iHVlq`;4|cm3Ql-r@PglEJiAYi_PJJJ82Z zL~8W=vYl@g-{GltJ?AgxP)^tqMYo^^mJc;f0M-vmkkNxr&35H z!vqbeN}z$I4)#;HR$5vb#!O!?w&x^MfFUzG%gfIXP9_mX03u5qC?#Pj=mkg#&XWLr zMbhUvC|_{erKBiWI2gE{ZQIF|0>n6o3GyWUkmn~sF8~WBTfr;AJE#rRaLy@z`8^MD zc*3G*kYehf0hlXVB~emIM+7kQFi+kMSU2b{C{;6k?T;LH(_6&t0AiOrIQ`=7CLf>1skm*>D1i z{+xp$*mRmCrzRur!IzDS2mfQMf9vf({_;QN`wOSzMm5b-B?TNXKnw;-JXwCqCWH<) zDr14HH|c!?!9VoKci$2$Y*06|)ue0&pmTc#hR_9KK^X{&cAL$K8eMN;O9bj>5sW^A&<<@bIaZ8jlg@-L zg2k`REm;7)g9>nqT@(%i030R@SZV_H7zjX*0l*U)ZXtPy0s}Rt1$~@gs1i9veuR^; zS)#=PW`p=hy1+CR0~YI$0!1|TMzL**XbRWDdxZ|WeS(7ENs`?KpE!-q+uMhHE-*IY zk{}V9ZXkn{go64wTY%CK0=lVL$KwEzsZkyU7)keK3k#jyL4q$M8EA>X>oqpsQV!6Tia_H3 z-GM{_q1552BkwONm*0zfc!uXm;_S42V`C=I)Ac4BV`FY?+qP}nwr$(q*ya`6$;M7L zw(a}+{h!b9yqY&NJ>AvaQ#I9f&Ve>g_$neb7P0vDFf2LQneeuIc!=5cu>06|Cbqq0`bkwdEX*0Cp6LPSaWhKb7mx=H!BU`+GF;SkOeE8U7OmlTK%bnH z%2=s-)$RUyR-Er=Q1^5Ev>zhgjtOc)?x1)AB@iBLuN$%;z|)Ud2=`A0iC?7)Q6Ga$4AQ^u%02h$%c5BT>(2)>={W-q+h0|jF)(Tf8csfjtoZHFFbWqHS{6>oXZvJx5oALJASP0yiR32`D< z#$q5r8f2P|ckmpbL;ct%AYpfM%6)`>f4e4i%nxVinP`U>f;p-8@SK*LUc}E!<;_Fw zN6v)=&{aS-B~A#P#Z+<*7EweDv8BbRf@n2ogH6YTx!LU_yM2)Mh%xt?`+Q~C3uGPS zOR3l>FpPNcPwQ`B`!!)EjRfnJWU4{?WrPxN>Lf8=phmHXrZL6MCN~wh9rzS zIX=eWen!F~O5|=Y=KCw?v2l+uT#)JV`21UB zi9|jPA#{nq<;eIanI2U2Qs(If`c#W~v9vhFA2CwhVhU*y%UoqT*zX%0mQm5l)TM+n zQB#;h)?{}yQ$qTVDOk_lKie7j;xd^m{Y5|ST&9#$kLeGWKEOty=DlVXaVPnfp3Ccl6+iN+L^TD)nXPUbj+67B6)wh%M_6h1L)kbt?m z-zNUNRW+nW8I=*UW4t(Mi~=_oxfjAiX1y&>_92*pDs~k})me!+)&YAFtu$(Y#rfw= zt~Jbq-7jh)8!(eK^j6Lk@weYQ-2@+Wo2g%=Q%9t2h7+HBO*NwKp`4yV?C>8to+|@1 zeGf9ltOy!q_J*kxGxO2`dX9M)QK+OuG?a_k16mA{9?`SJw~4xIfrNqB5j|_f`rsAf z1|3DskZ?!33KheTH;K-AE;$Pk0+2v#U(kODA=YlSnvAETyLt(WhN&ZeT4=lwOD%II znuB}4%*LY*L7&{-;+x}OiIsRXb!>q7)16MU)WGRghLcJUf~z5H$GfjwE?29$x1$zo z@)%4z5c6C$B%H{SV(FM5Ic45%46mS9AhK=)+T5rfv}pjhpQaovxbXNld-X9=oBfK~ zDx1kDNaR-}dW5@(9{`h@M6S4G%h8@u6Zo`U=$J+{7xT2la@aGe!di$FrQ^+l5br!H}Z#S`QRg ze9_pMtP)%bk^;JXIGzc)?A0INUn($N9vxB{t_HDk-k<9&HQ6in8l738JiF`Zbb1p}AJT_;Jb+$@tIZlCBHRJMp zrOo1bWJen}H2fDH9Lz6D z9R{DvM$y#C^lof#^Wi(R`q;hUsBtkc83O2#xq8_5sU0#5I0Vf$FV%+$G^~mWRVvle zIc9MLJq~v=zN`Ku(P(UTyFIs6_L|5zEx^lSr13Vo^zOiL=Jj#hm3P_$NteT%CbKy( zD_1QtR~RTrSdQ8E3}i{82eLE2MD8ERLBqg2ShRtlNB4U?^yNJq6c>PoqR(WzpRFP_ z#=Ss7l9KS~arn!bh@b_}O%3I7dwbRA+gms=K>U}lrqORU*@&~H5S&P-YoX!-Y);av zM#sc>JJ`j(XFQLAV;J)#_;Us#$ld@;$Owh6#hH zi_vOuet#O+H3Wx6y%DL0x}h40sgD^sT}>QJpULxkJJ}uZY|1AkQ>oDKn;e0kOsDy0 ziB7L6)hG?nmB;7TWbuC*akWlLPNGt+hAI<8b9=ow4Rtod=KUJ;j~}efSQ*q}OlQ{;~#=;CQFg zYGQitLufi;Zk5Jb)9ZI>G$@fWyxnZ)lE>lJd|L${Ovy0KH`{Ko@qO*4Q1&iY>PmM^ zz#0e1MWINFC}b($lTbDpB;;f7tAvt4D>RVa>lU3~soi!oX><8GZ~qn~yhEOpm`=j-`KBb>IJ} zFKqvQx$1FCbrZi91*m|ap^Avsi=cuR8QGLGVoO&`enrx%UC^>2cZxM2{}vGy5J{<> z_r8YI@wDCfzR&i&b|fhOw*@>jW9E5sGdC~uF+b++{hYQpE8N{nav1+>9RGT-)rP$^ z+xeN+%XizBtxo%;=Qb$1X`I<5F_!kq(J)EZOT^B6UgW;r@p9oTQrNKhe%<|9v!ZXe z()hZMz|Ziu&HX9&(7mGnu~2@G%~~Cf*=sb+l<8*?iVt|MmDKmiyLA7$>r7N0^(Sei z6p8tWuVm{w4~4wOYnh^MJx{(rgXv@7x}{p66Hm;H*ZIjppZ@Yww2>(;j9NvC(Re>jHQ$uk~ZnFfetx z1+40&435v{Hy5iykG<}^casS5jhgd$Z_Kz3W+E1z>>#p_hKIhGb{Ds~JD}(L`lrjS z zr~Nno-e$YUR|dCz3lXW0Q#aS!6|6WM_4>$1nP~5CmI8xM7!mUt{{sJ@mT?~a|FA?o z=ap4?{%@461DL*_Exo__E?O98mnWy+&P6eNUY0e)3Z>a%k79ps`wz0h z9#3ud_+7?gq(2&JAQT^WlGwJCgsh^nKaI_rLiv)a8=Pf55`e5QfNSu)Mqtnv z9?ou3DBj|j8-8Q#a*co1Lfx5gV^OB?y1pOd z#i? zyYunm*_`o#rn&dWYMTe3)&Bd@d|$iG=Ifnb=k)EvKm$3cUvF+R8jk>|fc77f;bOgY zcfIO+Oa_Zh0gq`fX5;%y{opVB`LbLlM}u{E92`?K@RM-JzEhCj{vN0Oo7KwW>V}!w z**eU&0R#CIe@Tc1V0Ac=Q!2Z*3$Kb43`5a&)qCB;mdmwNoBVz9{rQ@?^6jj#(<7VB z+LJzZnwlynOv5BCrNE{0(CJJMxQj9%@$slqV=nbwdR|0VVVOV|tNT1To%4vK3VZbb zNy9Kcw9;a&Ci`Zvx#$Vpi{`+}M8^q?{9Im}la0FGl>Ih+&s}h;5<$6<&i!rt*G|}c zkAvOqQQYrMX89s%`?aG4?%Gc5LY3Tn8VH(v*bK$_>;_7)#Y<0*YwxQMNFtv$=zUG9 z&v9}5Xb5-NHICd#xeHF!L<-?e>O!!ysrCf4=XNy)$TLGT-}0`M!MfhbB$UR=MZo zg;=cR7M_OiDWfH`gvrSpm&-N!t(R7tHNt7FPD@xA*Xe!a&p1B7EMZNa|5L^4h^vG! zFpQgb{SXK0=`9XE3xCG(DKwfRs`iGh)g@GFNzzRo`h}4 zu-bx%U3dF%du#InA=tVj40u8ljaAYV2ja2h?r}Xn95qgav+>XmjMNNCUase!fkY8! zf5`J%KQ#kx2D>^!tJGbs*BZU6mY+8r?)NGdgfJT;yU0L)gD~J9+~YIzS%v&corxLF z)8GK#sy=_I?K%Ru>v{T~Gz9Z`?AwNcFt`(P!jT&Pn=K}XyI>aiP>yo95Q)I=l)>Uh zElD^I&urrNIc%w=&B9-AUQ2P@?|Cswnpfix$7Nu$KG-`c&xWp)2fiY=0T{3^HY$^(kk71HOkW>~tN%(uwz6h&p4WhO2Ggx?>}mv34co!0K|#q-=X zCec^)9wE)z?YA01*uEB8R=yd|H*SsfJ716Zwew_boUj+G#4qhN+T_Go7gbbMeSI!( z-0|Z^Om&6{6h0%gFQ_-^$>n{XX1MkJ6d%gW^E|ok+y3OEK;)m;X?(walli<|y_>`F zbDxMYGc~c>>CB$ttjYn)zq%9XEpjr`#-p?K@RRA`tboT0O73pY1viuW3SjXL+9u_{Mq|n24a|3w)4L!N~$B;0PHrR1Wlw zgf|32BEoItR=cFr1i@OJUkI-UPX|J+#W0}*JrDGP_&4s7PbmCjvSNZ_=e-fVv~;~UP0qTp-S0$Pa@tsU^^EO z*d+5OXy@>77hMsFFeiFmd<3Q9XSgr-X^`*bhO5O!_h|826Zy=C+u6mMve~pz`s`7S zw(Vv+$RZ}p>(zUQ3q%+95EcK_I2IB@GcnI}*py4_a}sN?gZi$@O=d`-@|7zHGQh+S^cw>BqLz>^u zZQ5GTTwRjcZfXR_<$}WKKD?H9FW=}j`p;9s2y=HSj_hx6CR8@Oi$c~hiT${Z@|`|n z^R@0*n~|j@zLV(XvU%TaH{i0S>qGJ@^!iLLCx5?2aXKA#g|H*ajTBp}M(DZ!7;u~j z^Hb7db0HFQ#vh-S<*n9xo#t33kM=z6CG6xVg{0U2u&@q1o>{u`N==Vx1=cyhh=EoNu`K2e_WIDBk-DI7J}FZ%Wq zp%za{V;6vy!pxw5_qz(e?on)a9o$E_S1It0$Aiaa9kWvS&Slr5TW@?CKE@MA`(e9i zv&i%Pmu{B#aVW`i2i<;+#djV9H8F7@?UF0T^R+}B-|t^Z**A0bQ*9cpivq^Yofmz9KlxyL`;Vet33=7iSd~r-tgf(|LWw@@ferT z*E?z~fd$Fj?dfPT6OWRr|0HAS5Z|Rmp)`$`n%G^OnbZb&Xc{U?3L4Hqm*w2;lH=ED zv}~T=%VvK)o;{ww#DKVwVshI_zD&}S)o8akomiGV;&V9k2Z0B1QyDQd)SypmUjqNX z>+2b|?~1iC5|h2aK>RQhA^^e{r>13G6~dm-#YYJs26k^QiHHnI(P4PLCp{(7=!FZ0 z8G=|Wiq}4Eo`2LevKcV|;MG}wa~%g8e^xL0J~RYl%w$QoGBf|=r^(?r-o{B&!Dl#m zc^>U)FxkFbVdWlfnP~pgey*bdjVBp04jT{^;NF3EuPmF!3S$ni;y4{9xS76)4>NmTIa|i*x_gU^E6DoXu`G-|PMCJj-X^ zHzyt#05`$iV>lRgsnz*(;VdS!-=DG3VLpo@hvV`{it&8yMX~Nc&MkgGip1mLKIc#M zlV{t@a;Q@`x^x;C8tvyW%i;g6QlCtt70rqjEtAV*c>n@awA%E#-KMA6cd9Lfh;|tb z!AwoU2*jj=btDAW1z6S#1iO>p@EpG^$?@ka4^IbYP_1bkU69LJiyqI|eg3_se{o;s z8hbwisZ^7_UCzCxmsX4q!XmaCosUQHJpQfXZH@9g{o@A)Zo`heUy8log9F3T@%(pV zK%ym|TW}ldUrckEf|XWl6@$hITmCX_GxU77C;V4EgXu^F{_Hz0`>X7F_HWh~o?85(&O_ZFxU+eUFplbzCm9f8AsMzJkMdolC{g z-)ypd@BKKW2k&$vT7SIUVDHuI{>tT!`Tj?&v{Fbeg;GtKI`O+h8E!if9fVpR7xL$w?m!nAWw4LEg!)R|^_-`tr?|`2$L$Mn zd2=;eHkp~{ds*en<>q-FWg^}iN1#=!>VwbuKOQa`{*2>nR;5l;>DmqZ9b$|mPiQe& ztKFNIo6c^2L{q7@7p{u+dQf*}&-v#X)t@n4g;oMGnOGlWkx-xVti}PTw71gxbQ5gT zK*HM+7U?bpI9{?6i#-g(!2bDqbjj^*ILdDG5~0QF zVyg)fU=w0_0X&b%&e}D`t{$5ff%i&D+Ps|;_Bf(3i4|?PdI5YdGS;_{Qii*u|2kQ zvt0|KXlitB$TXObObBv8-)# zIf?(W*#fkP#bWc}dueC!-b#18#0ZQFG;=}w4kQsbQo!&?w-42r_;4wL`Sf%~Dzte9 zA<<%0DTixbD>N!_44e@hP*_b$Li9<=%&fqdDOG(FNynxl*XMS- zATf!>=B$j437^*_{Q)LrBXjjkc^EUIUpdma?R8$mXgpGXfgwtqEg~9w4k=ys*mRul ze|LM|j-gM0XV#^JgUZT(i#jX=s;Z--GzG)lVs(0hnvW#+R8&w)net%A2LbN1}t zuEUVRs#;H$`tlr(;QRB|pa-nAtC_~BPaWZ(#P5_dg$3Au&lxRVhj$B*2%fLW^4V;* zF=jx9I^8v_wHSydD+2rolIi{CWaRiJof_5o8dqO_&}n(JEAk0ngEk33R-I}g)_$Kt z)%+h%aZwvI-4fL*F?cP?i}7sfDx)-ZlhUK)ds{eBuQVFHrweBJ8qE%?v&D+GZ7=If zws4m#Dl(%8=}y6j-ikulLr5%?01qp(H)m#MqDGy_02%$zT*=t1`c?)E;$jxGmZ-3y zVtseqpZE_i5`)TcL1|dxO~um`@x1FK)w0|s$WdmtNogk$h?jvYf3&6844nS7&j$Gu zn&MuJiU*NuYV%8fIYxQ5$Z6sQO}mbLHnhfNGH7aIl}oW;C4w-Cg+2vl%%b(9zjWQR0}H!6(xjiV@m(J68w){!0;=LwMFMM0?+GIB1&GCz-jntje0$7`sn#)Z zV8lZ8jD2NTAd~ev3WR)txO81qbl>)+r+1;y0YaLeAf~e@XCm877V|4vd?cfg7`fTQ zq=l-MtL?S{@Lc_Sg@qE5NHLKoWNc7IO*ANXLQFE^+1tONOvuA7tFb#8uZ!v#Xig0) z5G)g@7;0456_>^o<0fGQIkM8|2t12i9fuWP$Drl=e`8@;%sQqR-_cuGQ@O|k~#_{|w<_ybnJoXHEk7jdYvl1|{E5IuNrjyIF zpYN~sN51^Pi#*@s;zfruUB^Mvm+eN^>qu419R;-Z#E|s#Bt5@pjV{@juK%T~_|u8k zev7(c{1oKtdW*Fj$6dtq`AS0sZcWr*2c%L|t&usRJamBQgR=ewtJP8-+x2_vGLBli~4Ckm9(^k{7!43}uH8c0y>D9+%* z6qhwA!Kd`igCa}Zmq5bDTQMZ@r*h(XJzaF|4G6=<7*$lIJ%)PWk!Sc@z)X?*m!6+o zD-~rN<~D&bcieZRph49TjEgS0rNW7BcCTok1VSzXNuVUBvd87m6IREQ%z-%33gb<= ziS#x;I<=W8=qUg+GxI@tx1ONl`fSuy=xq~6IW)klQ#sXxYF^aFh@980NVmfT1ZBSV zbwQPelhOSv$5}kCuK>*%b~$SP6?spf=Lu|J`$u#oQ~UEL!ac(ELX~B@+|+rnE+dHY z9;6nc=o%u(h(;P@Q6dC9HXJq+Cbj_`{%woh+F6QW8UNkCpA<%j4T@n!#m3A!avmg< zj>ZS%7xYB%%DW5@3YXBHXdQ~D)tYsx+|wd}-CNTFotz6Ofyu>C2{4qffoSwGMMlN0 znA;PnPr}xDMnpYF&>D3-na!lp4wbySoA`YdJ}&YktS@Jo5rG^n(W=4_j6EdeANaU> zkVoZ>v=tU*pD60y--mRJh#h7h-S$=1jfKR7#V~b~!ztt=z}=ys&+5`JMJygIhC$ao zY*)K!p7Z`_q7lTKqGCrv6~5$tn3}*svgXTibv7dzNY+*TC%LTHDOAjpm_yyv^q8B_ zp!6qou#+Z_(!eFTA08pZWAX{9ohto0ei8QGku>COwrG=tq?k3D6c=Y)<@|H9tK@2X zlt95c5_`c!CpMMmS%dg&$Hg7s@%Nt&kG*m7Sxi2hfrt9N6sW&M7xhN?vRI(BiKG`M zC8zvb9t)xv`t$XIysYh1B(UFi3kOxN>F9h$LlNh8?OzG4m#F*m7l*Xp`}p54<5hJv z8m%d2LQNvjlXtZ~JlCq@^B9Pv!p@ld9wq%n?Ztvtr3;su_pU(~7Txo|REavZT?<*I zm{DHUSYJdiEvxu7oPexUn zp_!D*Pi!1W-U0u(is@eXP*mqeCZ#KqahK!|l3!AjUtH7e52 zN`%M3MwqB2{)hyDz%7wWF>(r0PRDT%F7j0wDTX963nC0-t^`rg>MbsMyG`^^=^na1 zaSjc5`0L`~)0HT1;%m!P_mc#+w&`KtB32%pSm824Th=|^l;aCax%YIf+kSP}M(tq+ zi4DWh3~T_Yc-)J)(T~MCP3TB5A73n;^e6+<_7&8JJL)844INdiQ|Y`4I=Tvf(vbNm z0tA-(rm(*c;VmrUd~v#NqX*aeZLHD>OmxqzN~jP~8s-iuZB)B#X~T=8|1CPMPkgIlA18x$hJrjDz^Oa{WTdnUb= z}{qwY4GF?+rN89EfL&3_4LkI){oQ(YDTVwzgkKBQK z;b4+$P81vsFFY0x$|cxRYU}*O-?K2nz=hk!@?!%U9ph4`qf#_mI3+EUt=7??6H8@9 zrgH?3{)?7=B#?+GS`E9@^qXPZaz<(ik-hkIq5O4x(E;z@r#VsBIe2LWh*HpfDuJb8JRY!Q)jPEJLN7CQrX?Yo=D7*9ZRWg zgBN*Y1>Di9L!+!}VP4#50f~%PS{%qL}rHNUnH_^3Oh z3eb_PkcLF5Q;PF&F$HDJry`kdIxCk4YbaJ0O}MySkQ<#ceknJaKx$nAV2J9Em-){@ zBPD_09fx4-n(xo4mNE-~7T4y^5W1%(7jyP2n`aOy>(N>(oAM{h$wfqoXJ`+S zlP%UAxt+a;s!Dvo{hd(H_fqO@d!vLl(TPGVcT*!(%)H#IU26_&5jkfWOSeR`U^Qfq z$b16lpO`ZHV_-r|Hr$U4P`v*o+(E&YS}!ZDC|^&P4R-`JCrmE1+{{9Q^g@nYJ5?x+*QKsCK56p9x9~F=OM)z%ABk+x;`=~X?uV5A8K#tY zHe3!|fxJ|>WFAVqTtsp)Ir+G`w6_yo;E7`@y`(e)Q_)!#3JxC=Eeac$4TT;7|BteT z*fKeHI^6o93h3uiBvM0l!>3w$T^3(y{wm`U{U0jB1)Kax-+1yBjR8M01^3ceD{YQ` z!V)-z%T)f|EZIRt7;n95b!9oqU;=yc=ul3JhLOS4C9X5xg1GpJwbAJLbBab~Qb%Ol z>nYRp6b>`P7EFUZ!olbFWLSOFPmABxPX$jf4n{KDznBLzme@Xw8p-XM8*a>q35~f5 zmnD;2-%Qa2G(jcGeFp)LK&?D6!?>s5wLy)oae(u?BO(~T?8!T{Q5QTYM4(laKnt;5 zjMLf|&%fDmi%p~HB++dLXg59Wl{)&>=6)yPeq8Og8qL!bR6E_Ky)SONShQ;;E=dui zRxI~mZnir8hSlLFhtVh3s$lR9ri(G+qHE1o2foA`-f=q4Oh)eJZiCVFCDM&nfN1Cj zyWMJwXEwfZl7dJ7W{2&1vmo0jQ8rqs3FU$>61yrVQ*7#5cNJHy zI5cmC+jKF3Uhg~E%okM!)(n$hh=FI~n|BT2FXyLA$2WDQGKO^APPYkqzHw(u$!kgM zx_++-e7-%Ie7;XHnfh{>2kOopXvIt>{+BH)d7E|DD=6eWcNfmV?+;!VbKz~aYj+dE z2<531Hmwi|Snz5OODp=EJ@VJ)czy=M1FxfxSFBcx6f8iVtG1fYuW(>M)xgmbag0n! z?!7ZZtzKf`y6$6tjXamZ>A;iI^|xP( zS?&sap;%nKcPD+H!6I(kmfzj@U{{X~1h#1);s#}uNS@CZI0U|8umNTmz1?Q}^Ey6r z0B?H`(`9I?Y%X%$akV1G=T#D{X)BzSxY)6&>dGomsMBk{|Fw;;J2GDq$KRoEy1mVqeI6xZ zU>{jWv>+#`Rj4*1*Vn4mZ0UUTim}Q3=9k9*tJ5y>FqUCu*oQ6|H86dA>VHD+Oi(;txAgDmKXoE|EiHb{#dUe&?@& z;r(sTV+4PIp8Y<|J~LPRVt4PHrV-zv^T>O@fX!2gdTJ9n-XoF=ECPrh zB>*{E_&=hFG~>xIVxcDEV>r6bV_t}mEh{;4qSi|9 z^ODlr6eI;Kwao)U<%C~c=Jv<}T)Z|6}ocgUqx?~UVF`FlPX*7An z!tcZ%f8Dq<^LcCm~*j1|b?XbzJ=M@V4T@rI!tn?GTKh9P%*h@^kJn@NrI_lJ_u zEss)gC9LFo`5%Jbd!8dyleN>$RM#9`7t8K zJOvw$n=%gH%6*uNkMb=~jCgE&ChX=mDA{t$D1sr4yEi;~(K@Z|)RfWJ7sP*C$>Klp zJFrqiodzL7K1bFhp2wVFdp8v!1NUci%>lliD>R?InTq0Gfpc=kVuW zoJm}ens)^s+9hys1xg{XJ9v^T|Cr+n$mk_qOpN+92n6;nf5!8f(aD%Iwbpqhp&juGwL|kc`!z^^83Rht+hC` zoAp33c>dfb68c`fyo(j`Bb@O%t}!8&#j`(mA?rgmhzk0m0kW~DQBl^Ix5co_?NJcijafKh*)60_ll%!?bMEWu|O zzY-b@V{V#WXsD2Bv?vS52sovnp^BC^`URzLeVL}Fq!d4r$_}cU*Iv>=ttsFu7d5Y95s(hcWrjrI z7GT2LTb>FcD6-nVY<{?`rh-fEE_;guozJtf*U6a%30esdVI}pY{S3zT`WI0g<_M~l zv`4Fr?{TwJmhb2F3}QQ%EU;0krxYDf%Tz(^8W!{$M!a)cywazva4sp)Kl83Huo_z( z1WLto?+AB(ua&{IVkPWZ9Ig?UJ{vSzFC?AD#?)(AMmq=3f)d=}16ID*?MTB8iUL~B zJxK_Rp`CmL`&f-*uYQY**OmZ%JPMq73z=Kv0M=x5lx~uiA8^kwnlDJOJuVeISBQ*8 z?piqYy~-@kXJFh+=-Z^y$y9b2ERafW=jGuUj!Bz19t)aew@>tIOo-4UVmcEFicc{c zTb&ZIV8G#QFe0jO22)mzZG&i#lkPo;scEUmyhSYvPJKmYh+9k5XB`P%LR}XbLD6O0 ziccwAY*TJPX`4Jj55T!i_&OX;<$8J$gh6K~on%K^D|I-RFCM9t*sJm5wrHfB^>Wot zzzydGNu1;=yB94kz)CKmy|$?RqbMuG&e|k`pi`EcnkRUPwx5dLQFv4fF(FHMB<<8~ zes8IVnW-LJw}?*^Pas$-;!LeSwWdo+zi?6MRS7vqD_Jy)S8mxZuyNQ*BSPD|vfwBI z8?3Oh^2G#gkamQ?LVUio*eb)L;#|?JcaNCWOb47$;V|-dj)ym@I;@F=-J9uvjfgXQ zUQRlZ%`_JBYP$-7LM7GS_^5ArJN{15lTxB&vKdaGNN#ynU=0e4MdC+?RDyT&YnmO?xG4 z&u*!c%+A7pi!68Pj%pIkc*6L9ysifg``GdO0k6!| zkEK}GZt12dqUGeC__}f*1nIhexJ$ZAtIIBu(QcbeO)WNkA;Lbj$ZX$Lg^IZnZ-0q+ zNxY=u&!m#zlqC#e0aeOhS|OxOTzl(CHxb<3WCEm`ODRFJD#Ii+5=Y{fM#=rJncjer_UL?f?IZL+HE-uAS`35E16%$k~sh@#x`?o(Px&5p2`ry zfkJ=;3t$r|2qHhz3tp{^ulJIGql^$3ATmG|{{#j0NnL^IRDrk@4tQUH;6HQ3auMX` z*@MMRHYiLmIM80qK$j&u>mO8dsB!V)pGKRq0>In)fX}azgQAIVdXpl=lUICgqW=$}s+0veLS984!Y)gK9PsG)>B zH9cLE#};_ls}_m-pq0m!)}{+@9Yn5%)7fiuM?q%44i+Q`3@RMCO7HVT{#6;v7*CS8$gk)lIY#^WU%5Wb}W?ly>l4FFr!Mvc6H`vsaYT~=wJ6kdMh3@p9XLkkF44NjK-K# zD^j=}iG@1m%J**xX#yFk%GT4E>k|)IDob8;01}rhH{+8bZLisCP&WgSJR~yR;YClk zyKPnFN084pXB)L&x8f}DTIfHCMzFw=plL|_(2^lDI%!SffrV~r#}i~0FqUkRLLOA zYP((QizcGITPap*H;VDBDjVw>GZj=?f=eT)>q;s6&+A*dbs z11F))f9UTit=qMo0Dt_NU}yxVUpuC3Ddr@xL#NLY!_h$yA8tvE7FB>{<3u>iLwX!% zDJ3Y=QQZ)cW<2lA`XXSFk6to`o@KP*6Hh)sz~@Kyb3cIsHb{zfD^yT@I8-o)A$qY8 zqg{eM+k^cZx%@aiuWkoQo55rsvBUz@t7WNL7}?91Jcu>1W@Mq1YN*ergJqm$M1UXc zgW}Q$9^^WSFXxVKHR*U~|0K~&7c@j57-$!#T~OSdnJ^}1A{mEPOZw0-9=+PSZc1;Z z2zDNU>~BZe1awDWlY-?wfm@gtP?H_9o2|?YFI@`cgaVQ zi2{W!mD~X|__HIb7+?pu6sl@lxz-4z8WrlI+faX8_u)YKBm3;mBzjaR9Re`poG^_8@P z86bs3SoA?-kUz2n_oE&YApJW`NEh!#yH!N2#ac*%N~VvF*qspX)^wtt)(J&q&lnb@ zaZL)~`$aL2K8-PJGR7jjj#|b7>PTtXc2n$11bDrrDsTJq@bhSsfB*`Lm!zCnazTMEJHS&VDLpzT@w4@Tokf#fB6@O zXmsnVa~65Fcyb5KxK!FxhV%bL1M9>F5!{zmtEhj0-baER49>AkJ2va?KX-2#A3%83 zTxX>GpO+`tLSz`qwPUrKF#a!E-~|n9U+9UF1^Lfyy9oCox&xNVDdT}2G2sacb5HJ_ ziVd6v==SE}L4-F?)rv}a{&UFxKVEjBx5;)HZ3W}e2bH21{xvH7yQAoh`6Tdp~ zk!*->iAf(UfJIzyZcVd4ChKAu{==unA7GMWy+^%Z?9R7f-^xV6=w?dl5rK?sDoHsi8b8kdMe6=jOR>wCc7NLf(b$c6;jpO38gec zLvx*=&N)j=^Z%v}41^n<`I4S=Og&R1`Fb?Zsr?{dd%Jt+KA3hn(>#yU{OLl1}zGD8NOmHdBM%PBi$xWicdSY_sv!UEw5!Bgw}LBpChDZ3Ai#SmN(-TI;2;cX0a?e-BfDl<~=y#KhK(_ zj+bLKlPG*csai^H?y<6bz^5b+MSv^!usU5Tn_3U^ai+jfx=~ltgpHp5I)XUPV!l|? zvPLu-oi+^qx7Y(-Jb9+XTGU{x@VZegGWs^ z`Q`0MiJ}^xOQRYONzwFN&CcQvgjOxcZO&!Cj`P`(hb7<=GhVhdrefO@EwGAJsoF}y zJt)Fm#7wJ&>t3vwDNFf5#YS_#Tzv{iH>KYexxm_)Yh?%kyh0yZq>@68yA`^}0`E6Ss1r?zE8Ji%GB#HKBH#XHIw& zR%Bg*1)xX_L%M+m>Y>GZ&TFP!ijCAl6ximj1DxcCz*CtV>Dreab>^Ozn?W;R*;@Ne zqtP2wD*WA#!6cB{2Tx5?VqwuC25ZVIHOUba0@j?;=l1khEgm2QcL^F?LI@fl5S#>e3l?00yGwxFg*@ML-}l^m?*H58%uIK6 zO?7oowN=-2POx~6MIF*l=>0R(ETI4`3 z>2EVGsnluP0D@UNb@Fe+O|14|kd{AEeU}5%kWIvA{*^I5UxZSm5LwmC<5+|ShLhEm zh407wq@0T|?+jC!W#FDzZ0gw{WnvUj|wHW0L zimxM!#V0sCyMCRnFSd&IQ;hQoZFK4W1-aHDC9RXNqKLV3zB$l64pHhQy9Yxn@di6E zW38Fi(vTW^il$=e*qPr7jBT+Nrzo7LX>m8p8$&uF`blnoBPE$8Bo!MY(%MjO)GR1% zE3Z&Z<26C4ATgpw1iYQq$~R^D5$mR@R&y~yurlB5qwHqJ2C_tUKYpR1yMrizabqDd za{h^CFe(x$Kj|m8WFTb@!3s)^+%Lkoo!Sy|F!-0}({DjWBQ$8}BHod5@=ZFQuuDKl|7HYlQ)lQixNF&j5l4_OQi5%~EYX{w zX3fcPEH`;2?wx?&lUHmTvz-d&&ZWrVPzlswA@A|YMklacas`Yy+AuVv$wlZL&X-IU z+|(vT?Ge;_A-L_E<_p2=o83x~&<9V7fqq=xF-r1sa}5$)Dbl}8His}(yirUQDGYsW zr*H3JVEA^W&e85BZ!KKMv*Ru0aPWCjh2dEGiA1iQ2!5<(y~Te!#yj|&aWJ4T>58hP6^nw<9K8bLq31F~-ZGGJe!Wh$$P z4IA5Ti_KcT!qWt$;UX(&)ct`T-d&_tcX+*bC97T0m4ALBe81`Wp zdRxEU{8l`*GLbSTGjb2YPr-O5Qk2{EFav}jd*Kb40gUsg_V*d^?H0~`WyW>l3{7A^ z6t_ag^Y)xOU6<9SWn5l56b7Pz+G{c1;ATLOM9jO((=u7sL5X65%b7VqtuKr5r3C__ zedS!9Cc?zz{q$>j#jq!doZFQ7TYD{x>yG+JAr)|Q!4}tr8~G??OW<*F_$>^8$bh=d zi679j#v1^a6G^J;l#h&l?;Pl)o_tCtf2l1T6o=j@cSFLA=0%~H%M?snc zW}`!*+Fv3<^`9u7Idua24?d&Y+lR`u?J>NHR^S1CD=>R`MaN8SszsB(5c{&WIpMZ9 zq;7YaJ;8pz4n`uabcXxfKD~GCh{+g9{RohIl8vY(vS!Xr7n#WYY|g)tyjKYKn7T~~ zKP`ULCA=^@neGlV&UPvVSlpXrfZ`M}Di9^q>EAuV2GC}fZAz`kTCg;0Nszs17KvLWZ$xh09SNY#M-e}5Tn#B zAvR6a?o%mgecCI=DXRAC{5u>Xnrq^ILPV3HF(JIG-dsASyiEy%R77dQ4GlL>BUw2O zm87uVFNlY-<|fa)o44%bLvizIduD=v<5T0ZYLyMS<=;}PG|Nm)sr01Oa|K{N`y>gk zh9&Wp&D-s|l1V8eF~K-qfDbD=E4=G*fY2qEeKniO6748L*5TrWA5|8cq_sO#fixhw zWe1xiYv-9GnSznKQ&h=p|D91OVfWV4*nlL&sEUSzZUg1_)Y2W%z!Db!iJ~8hNU_qG z&sid^tXdcw{G{2%5_&Jn=TP(+#7BGprM-HWhz9iB)C$F<;rJ z_~2XhMgafeT;)f8rynJzZT==_$7u-to`^5d%n77m@%7Bb=KBlSrQCfn1D9|9s^G2|pg@1YL^z<4`An zWyZ>FD0R*|=l{fzeTA&;K+E5II4#|D5dDF4y-hXp^1tg=ZU=0=$k#&1nnA_F1-Ax5qo>FzK8oRhb>Db%DeY|aWn89Z8Zdam#28TRVvjvsoH?J zFf3vH-5%@PR#!`mr}NGpXK?R<{2@=y57EFFoG!`=fBcI6!TlUBf@CE>c;&0}%`i0DXu zCeaujlE+x)AfV_CHVi1T%N3QdUiexJDt&lAXG+9o!e`U_$r#IamqstzMH7ErzT0L- z#;R?*QGCJmlsAvjVCOTUZj{JJ4H1)_OT* zNCZj?IE!Bs_S~Y~!Q(lTpT{uPDY@ZD=S0~tejmYi1(v-#Id zJufsr+chL7&*j<MZPjk~|e#}3b^rBFn6E>m-0C!eHlu*2co6*#Pzn|SKucDaV-xNj7W z^nmm^+76fVrWc&uM%P#*C#4jsECE+Ma!N1(QI1lTA-F<{3I6qsnM(3bEG6IN499Ju zfAKJMP|5{0L0gck+RfBgHq1D-28Jb5V=B@EyCzw5BZcPOPb=bl8Cy5rSJLet!$=aFBge_FJ@RI6Cb=*KaJ|fBwOb%nQrhsl<0TW>&W78v(V=Q3Lq=a;ss zWu<=0`(q9AW)PN)kd`tBvTRqpI{A(BdXipw(VWD}4dwfjg}z<24I3PJz_X1ekjyvT zDNB;`pe7kc_s?!th-yW?ECt(ojJUQ7d8H6*iP}GYhmK)_tW|mvvIqr&g8Ia}>P7CQ zDTWE7jkTp8URM}@^q>qFlkPPDxuz4Y z@eAn+Db-;;>6A>Xp_1;n=ei!R?O6GI)cP9r+v>?Uvm3t1XRT;K6Kv#V6Y(#si?P#0 zc(EZtSrSy5qFo5kP!NI%y@<-5s^tm8DOB2cc&ixyrv;L+6_NtEd3An)nz_g((7y-C zrGaahDqa#Xx-(1ih{b&zrSKPa(GN=`n*0Ii&yxLZqFA5Zh8}3cg=#6^;oj74Z)Mwi za@3Sefvx(KETKxHx}(6~a;ED8_nYE5C=B4u(~$X?+#(ifMnkr$nK&*S`; z6I}k>9bHjNO*uGg1V9yDen0)=U!kz8lIN@d|Dj=b+Fro^;O+J4 zCJ_NaSA+0|N(NtxRe^FH|7eLf4Hqglp0{+_P)L13E+mB(YZpJAjtQBwcc}N?NRf{` z17flJYaNMZ1qq!HA0L0Fp`lJS1$^u?X;*2`PtV+l3U=U7OIf&rs|AV?;k{A?VzOF# zb#9s>tzntOt$L0)bLq-=T{Rj>Cc6XsKORcuw~ZE*K1U5r2l#*q)L*r>etT6tiQq0`@j7Mc*13?oNq+4Vi_|DkP zd|^#^ZNGQ25iFt~QgImkL$>vFgGTk5P+VA$r&m(7IUubZM zKKbMTVq|BpNO+f#l@Wv-6yG}$zY*%bn-I0jTkJHjqgAGaJ7@BF@^GNHZ>U`(!+V>O zer0ZJ%I{T+Wq4w^ex~Bti&u;I#v*GeGhF@SOnUylYE8X9n)#aFbGpg}Hucka`xNI5 zr+)?xN|ZNC_B62%mdq>t5%{dFte!H^uKm7#n>jH?`ouBwCDkMf@l)U=uiEu^SYhzT zv{u}=N(tGJm4srGQG$>nbp9FwG|S%snw7Wj!4Gs>S)^U z=0z%{B#Hd|Mzx8(^8)kG_cc4etouvoJ>q+0-o$#}w#rWoIgLlV(Q}^24;~BpGq6;O z6|K^#+oUE)TYY;6BzcMVHh2igh{N+9I*1v8q%E9IXisE*cJ-PL5J}DKl&)b5W~6D4 zd=&>9jqZ7Nt&GY#&T}fc;!&m&T-((*L9q>6Eh`^VmUcJ)o1aw`cTBDRvyf&gT}17l z(Xu*Cgb@U9@hGbIkNdG=@Px+q6vcs818ab@IKQ-ed(2$pIROQt(?toIn`YgYVX zJLC)YFX35GTw^`q2eR#YhC#4I6?~KV$N`iI!~_jgw9F1s=L)&5z?bi+Uz0|8`-yG5 zYTt+*L6hlhor_3Rurj{ku=<{AslOwA^8(NT*byAJ?k%$EqyjdO?pRW@YDrdiVY8di za0xK+D1Tds&FByK8Yjq&QuyTQDPhr*v;IulOcu$a=;7;@G<(Jk^UrZv05u-7fF8CP zmd@EJyJi4i=kj}8CRhBB2(d7d6nYD%tiN_I}$2n|BU7wq2qm^ zDcVLEx0G1|V$bmOV~N?d@#GA#J18cm4pN6HA>_T%U@d&@4_{Vde){ahgwz^mCgu$g zioC_O{OaPjc5`SvCgf$)5wnmbar$W)E69pe1O;SYLO_y{i&Irh9K3sk9hMNVqDJk) z*(zI^ZzUob6^u`S7VAf1rlHRfk0tS%Cqv1LE-s3yk~qg~e$^-&_Y=uv_!*+z97R{f zYDl7VxBAL!Mq{orm0K~@;D&6wb0&w@<0OKHzA)Am=HF{(0eVe3%Ih?A5yBPl4W}`l zC8Go`99qLqc<)n01E@tuan)OeYU_RB4rfJ2h`i=z#vytZ#uhSK04=8NFG4J94=all zz`~Z=#mNuGQN@)N?{D{TW1$5ZU?KuwnPXB?@-@ErCkMar9GMRZ4#K?;gpIz zzpk#PVt|o$QjsMmuvCA<1@B=6PrmQrk zmS*um;#+fUay}`uD*ZBV99LWMJKN3ak_Oj4!t^S}#^>P%otD=Cc}DW5q+o4WCk9%( zE2EAjM9W5qQpLDE5a^%;kTD>V)R{#kkTvZ$&Q8|SdWLdq{v@QJTez;#HLw;<4o&rY zKL;VR18-FIOtsn3enpoAT}QN6wL4DmCIMv7gt262E;}R)omOi zocFGR21;8IMz^=78(q_v`Y5xEr`(pj@BN!X!Qet$x?`NH2v5yl(B3 znaFRDw@qb9Y#1MVjS{Rf1qu8Pvn|r%jZx*BXWvL)nY!bUS1m5v1+3K0v*ZQj{YbwG z2}HFTDC&S|Bt9RsI2z-NBSx{)En$e~a4|jYp^8^T#C5V}J~bP!kVY{(lY^$=cLcj) zG_Gs>vfqH8m!K^xkHN#mE&Hyn$Z5!WN1Z83I>Aifj(@wO@4)OsN0JzbK#J>gry=`r ziZVf=S46h-6u&ytHsuE+T;2pyMEHu8Y|mZkGB-9Ed~FXoFj9^xudDPbNjdT#Lmg)| z_!v-(ab{5J5kTesps1j@pocu#`22qwOFkis)~75xp?C@%gy{QuD5?7nmMkDTUNBiI z0k#Qd5F*)R>e&X4OObYd3d!t42@;emjV9f;vSiM7&Z~+czLvTK8MW<@F~2z=UuFlz zG|kG|amhmcXa6+e6v>ZezA=rf8S|Q09ooLE(*G|_nwMh5$PiwKu_9pU$@SxkVQdwz1kA_&q z)hOhhSOP%hnl|P$#~LuJJGG%G$P7EXl&SBAh?G+8PP6-$mq@cn)j)k(O$FGU<~ z@OpN>tGA>$ow`fW3!Mzy>K}P~VT{!t7!gLtZmj0sQg$Ql81BYpvHB_N43uw|sbF$g z;Y-iYBl3Wp&mW~jO%O*7-UGizM<|f0#whVb166PVFu8JTWW{vB?tk% z2Pc{F0Yjb!Q>RV=jpjcpj8k}5nj8ZH&dM=`lV^i75~a@ZNuunHX#6aWM2>7FZ z+uFonufYD+;wE`QPO1FlN9h1SrK3$|x$QPWp>?=Vu9y9SN2vwd0Et#M`4wusp-*?syGtIUW#Q@W?c_GK4*nPT1WWf|w*edMVN!n4J%VVX=-Jx;**6C=t zC8wq*(Z*39!+6(!r7snGs54MqE&WC2l2w|iEJg*y{fVIo&Hv(%gnd6yRk$^ zvZ+dg#+`Lf@$9Sg!vt~cN>CZvDSFKg^qYluuA>d9`psp*(qe=CwEfp;m_wZE%`^3C zMy6TGmRKLe9B9@q7KQ)R*uwiHJspKy*}#1{?{@L=56=^@pKd=l^Vv3x=6 zpoc1wO-rML=A5^Fo!;_ER-ybFvl*n3N{Ng}Za3s1Pe`2ajYW_9^mD2t z9XtuDYFxal1kr;Qyf$YJ4=JYremt2`2NpGSb~Zc{9IU_E7OHjh1!e9BV-#JxU#xgP zpD#yp9k2@xb`pF#Ae5*2j(k8;8#f}j*@RMNVfE$G5Vm*DOuZ5ZE__!0V4|pHi-?t0 zzoHzcXUCW)WQWu;rDJ-~M*ljQV%OWh4Y>jM%t8jJ@ma$@Hz{`a0|g82=Yj@xmL&@a z3(0LadTh4Obnu8Z`#f(M&9$D9!@}j(u00=7?oa6a=V8q0%w#A1)VuXhn1NQ{Ik%r| zAa|$u>&t;q<*Sn>?6%CN*FR#@rsv?thzpPGtz!x()#(+y@cK1~hfL`Lv#y*j#Wc;` zbDGk(=x+V$vrj6lHP7DGZOm_M^ud0>a`BP&bhFAxDlIqt`|CNm>Y>fR7EP)~pooiJ_=7F%n3sy8cK`SqvPFYh+zfJ-YB z!gaPkyf<`4+~e-!teZTxetC*KojaJybNq@$f%8S2_)q#Yp0NLD)iv9!ivH)5+L6|u z-D)509Z?|;i!WD!hukodTA3>UA8A!f4v~~Uoy?Zl7q}cuxlDyT;rg8id&Lsqv*)ZX zO4QU)h%DQMFUka^YSn%WWWr}4tx}E-*RGH0Y^fDUG;Hg#WvDB) zsQpop%M99hkPJAx9M7K1%yaEA2;ofh;?xkkTOh4+F?C1CHLgvOX;9mDPdVGp_{?AM%r-UKn#mC zMVcUEv%w1)b0DvUkcOFe6>++#Q^jNSLY7lt46Pu-^oL`bQx^?Sib(_R3eU(zGtu(IhPKxi^@miAXZn?`As z%!^?EUX>6oGa4(?5v%zNpp2r3Q6_ibYiq!x$Mac^Xis!@qLS0%^hV0}*Pj;0Rx6KR z>)rhmv}QdXGBm}?Y3yV}B{B)VCiR?|9!9`tM7(2hhT!S`M2B2XN${X%yXe%jty$JVfj@W}WRL45hCcoZ8w1;6Efm@7^CyL0Pbn=B6v%Pe)aX zUxs3Baf1(7a@sg6cJth^_ZKa?n;Q&uKrI#%B=P>Gw|m}7I{sJZ{mAaS6ZR9ME>fCe z6o5mcAFAERYMOgC1YZeHYQJ=rug zgeMI$rYhI0G=zOeJ|}Ljs^XB*2vIU~!=W0{lYJJItv{>2YdtFZe=Z8s1xYUY{Cwjv zq+hH2Ik*J9|J^_1Nj6>X``g{xPn*GQr9I>{wXAozd`w_@h>dmiI5Rw%WFY~-@-llE zooE`3aJ{oVc3r>Ld-JU-eW=(~na(#759zN^8`b~m7svw9in@0M3RQatBGA zNS7@ZDk!!AT7_sV<9CYZee-w(H;e0f#QU5X$aJV*RON{Yxv{la1iALRQT4I<@yf1S znW|5!%Y0J0v<8Kkk93qaRM}@ByaRw2^RBOrO7mX*NQxJU^FUJRZpD%FS{>26-(|H& zx*_ke+t4Hafb%4U|3ABeQkbps2HXG*UZ|v0xuHknx#UGCfBV_l2h*Hcty-qy2a`1R zhoNlhd!F^jrULW9j!IcO<1HzCAiVuBA%cBU(8Pk*)7<}0UOW1;JZl3+m|eel^5kg= zOhyl;0rsSFc6G9{b%0R8yqzIbFV)mIpP*Sopb%GEi~os2qk3?h0(@vZy!--uf_!`& zaK~fGW68n61%B|uodEpe!XG>jez<#t!5Mnw^e^b){($o6{)zwJ z;RiyG6yQ((zc?H`+#K9o+}zw8f}D>8|AoUtTn{|_llOn&aC}ZQ`0XL$A%%b0w1T?_ zqHy?smk%f8A)62I|H>8zj{w&{W%Llu!GU&|Hl6z9$+IL|Ej^f?@+%<=Cr5U$rIR^? z-OJJW$%7LS_i_eX*h64c<`8RJs3^l>QyT-7t)(b~F0V3&va=M###Y|P6{6{*qGjP@ zZy{*OATEX`;w1zp;0S?%sk|H=pl(85q7098h2Z`NFo=Qb5d~&1%Anu~wuT4^$#~gX zX*oGLP;qjwb8s-giMU!?328{n{zC#D5@oP~!JLIaAWu(Ec2910Cs%6_r=Xx9h=U8n z#l;54U~}_^!oXf^P&XPl1vIKh8qyFq3s+lbn5`3(>VYQM+{qm#%D@1Zz`xwVY_0w` zZK&HnMF1}h&_fJ}lbr(ur$$93@(>h~w1v5;L0mP#j?NB{2k_riG$AgJfxjUKTQLC< z&_il)-y_ar;2$KBN5cOnrN0S07RW0I)EZ_Z#>vUc^AGlag8yaa5vXfx2}k7-fT#G# z;eP=CMbwuvmvYr~H-BW?7Ha+fV4Ul}82>jv{|)|&^WXXTzohXO?LWQ%{JTDu1pLE3 jc>fg?c;H`AgFxZ( spark.sql("select name,sum(count) from global_temp.df group by name").sho

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

-We can see that details information of each stage. The first block 'WholeStageCodegen' -compile multiple operator ('LocalTableScan' and 'HashAggregate') together into a single Java -function to improve performance, and metrics like number of rows and spill size are listed in -the block. The second block 'Exchange' shows the metrics on the shuffle exchange, including +The query details page displays information about the query execution time, its duration, +the list of associated jobs, and the query execution DAG. +The first block 'WholeStageCodegen (1)' compiles multiple operators ('LocalTableScan' and 'HashAggregate') together into a single Java +function to improve performance, and metrics like number of rows and spill size are listed in the block. +The annotation '(1)' in the block name is the code generation id. +The second block 'Exchange' shows the metrics on the shuffle exchange, including number of written shuffle records, total data size, etc. @@ -362,6 +364,8 @@ number of written shuffle records, total data size, etc.

Clicking the 'Details' link on the bottom displays the logical plans and the physical plan, which illustrate how Spark parses, analyzes, optimizes and performs the query. +Steps in the physical plan subject to whole stage code generation optimization, are prefixed by a star followed by +the code generation id, for example: '*(1) LocalTableScan' ### SQL metrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 9351b074c6590..ac66a71fe7ec0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -62,15 +62,22 @@ private[execution] object SparkPlanInfo { new SQLMetricInfo(metric.name.getOrElse(key), metric.id, metric.metricType) } + val nodeName = plan match { + case physicalOperator: WholeStageCodegenExec => + s"${plan.nodeName} (${physicalOperator.codegenStageId})" + case _ => plan.nodeName + } + // dump the file scan metadata (e.g file path) to event log val metadata = plan match { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } new SparkPlanInfo( - plan.nodeName, + nodeName, plan.simpleString(SQLConf.get.maxToStringFields), children.map(fromSparkPlan), - metadata, metrics) + metadata, + metrics) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index f898236c537a8..bd14be702a407 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -78,7 +78,7 @@ object SparkPlanGraph { subgraph: SparkPlanGraphCluster, exchanges: mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]): Unit = { planInfo.nodeName match { - case "WholeStageCodegen" => + case name if name.startsWith("WholeStageCodegen") => val metrics = planInfo.metrics.map { metric => SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) } 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 ccefb53fc4b39..fbf97e9a3240c 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 @@ -83,7 +83,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { // TODO: update metrics in generated operators val ds = spark.range(10).filter('id < 5) testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( - 0L -> (("WholeStageCodegen", Map( + 0L -> (("WholeStageCodegen (1)", Map( "duration total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}))) ), true) } 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 b8c0935b33a87..a8b77b8202123 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation 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.functions.count import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.status.ElementTrackingStore @@ -616,6 +617,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils assert(statusStore.executionsCount === 2) assert(statusStore.execution(2) === None) } + + test("SPARK-29894 test Codegen Stage Id in SparkPlanInfo") { + val df = createTestDataFrame.select(count("*")) + val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) + assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") + } } From 6eeb131941e3a4afb6f8c55de06e9c26f942e155 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 20 Nov 2019 08:32:13 -0800 Subject: [PATCH 54/83] [SPARK-28885][SQL][FOLLOW-UP] Re-enable the ported PgSQL regression tests of SQLQueryTestSuite ### What changes were proposed in this pull request? SPARK-28885(#26107) has supported the ANSI store assignment rules and stopped running some ported PgSQL regression tests that violate the rules. To re-activate these tests, this pr is to modify them for passing tests with the rules. ### Why are the changes needed? To make the test coverage better. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26492 from maropu/SPARK-28885-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql-tests/inputs/postgreSQL/date.sql | 34 +- .../sql-tests/inputs/postgreSQL/float4.sql | 12 +- .../sql-tests/inputs/postgreSQL/float8.sql | 24 +- .../sql-tests/inputs/postgreSQL/int2.sql | 14 +- .../sql-tests/inputs/postgreSQL/int4.sql | 14 +- .../sql-tests/inputs/postgreSQL/int8.sql | 12 +- .../sql-tests/inputs/postgreSQL/numeric.sql | 990 +++++++++--------- .../sql-tests/inputs/postgreSQL/timestamp.sql | 32 +- .../sql-tests/results/postgreSQL/date.sql.out | 38 +- .../results/postgreSQL/float4.sql.out | 10 +- .../results/postgreSQL/float8.sql.out | 20 +- .../sql-tests/results/postgreSQL/int2.sql.out | 10 +- .../sql-tests/results/postgreSQL/int4.sql.out | 10 +- .../sql-tests/results/postgreSQL/int8.sql.out | 10 +- .../results/postgreSQL/numeric.sql.out | 941 ++++++++--------- .../results/postgreSQL/timestamp.sql.out | 44 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 14 +- 17 files changed, 1150 insertions(+), 1079 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql index d3cd46e4e6b89..0bab2f884d976 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql @@ -7,23 +7,25 @@ CREATE TABLE DATE_TBL (f1 date) USING parquet; -INSERT INTO DATE_TBL VALUES ('1957-04-09'); -INSERT INTO DATE_TBL VALUES ('1957-06-13'); -INSERT INTO DATE_TBL VALUES ('1996-02-28'); -INSERT INTO DATE_TBL VALUES ('1996-02-29'); -INSERT INTO DATE_TBL VALUES ('1996-03-01'); -INSERT INTO DATE_TBL VALUES ('1996-03-02'); -INSERT INTO DATE_TBL VALUES ('1997-02-28'); +-- PostgreSQL implicitly casts string literals to data with date types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO DATE_TBL VALUES (date('1957-04-09')); +INSERT INTO DATE_TBL VALUES (date('1957-06-13')); +INSERT INTO DATE_TBL VALUES (date('1996-02-28')); +INSERT INTO DATE_TBL VALUES (date('1996-02-29')); +INSERT INTO DATE_TBL VALUES (date('1996-03-01')); +INSERT INTO DATE_TBL VALUES (date('1996-03-02')); +INSERT INTO DATE_TBL VALUES (date('1997-02-28')); -- [SPARK-27923] Skip invalid date: 1997-02-29 --- INSERT INTO DATE_TBL VALUES ('1997-02-29'); -INSERT INTO DATE_TBL VALUES ('1997-03-01'); -INSERT INTO DATE_TBL VALUES ('1997-03-02'); -INSERT INTO DATE_TBL VALUES ('2000-04-01'); -INSERT INTO DATE_TBL VALUES ('2000-04-02'); -INSERT INTO DATE_TBL VALUES ('2000-04-03'); -INSERT INTO DATE_TBL VALUES ('2038-04-08'); -INSERT INTO DATE_TBL VALUES ('2039-04-09'); -INSERT INTO DATE_TBL VALUES ('2040-04-10'); +-- INSERT INTO DATE_TBL VALUES ('1997-02-29')); +INSERT INTO DATE_TBL VALUES (date('1997-03-01')); +INSERT INTO DATE_TBL VALUES (date('1997-03-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-01')); +INSERT INTO DATE_TBL VALUES (date('2000-04-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-03')); +INSERT INTO DATE_TBL VALUES (date('2038-04-08')); +INSERT INTO DATE_TBL VALUES (date('2039-04-09')); +INSERT INTO DATE_TBL VALUES (date('2040-04-10')); SELECT f1 AS `Fifteen` FROM DATE_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql index 058467695a608..2989569e219ff 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql @@ -7,11 +7,13 @@ CREATE TABLE FLOAT4_TBL (f1 float) USING parquet; -INSERT INTO FLOAT4_TBL VALUES (' 0.0'); -INSERT INTO FLOAT4_TBL VALUES ('1004.30 '); -INSERT INTO FLOAT4_TBL VALUES (' -34.84 '); -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20'); -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')); +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')); +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')); -- [SPARK-28024] Incorrect numeric values when out of range -- test for over and under flow diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql index 957dabdebab4e..932cdb95fcf3a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql @@ -7,11 +7,13 @@ CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; -INSERT INTO FLOAT8_TBL VALUES (' 0.0 '); -INSERT INTO FLOAT8_TBL VALUES ('1004.30 '); -INSERT INTO FLOAT8_TBL VALUES (' -34.84'); -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200'); -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')); +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')); +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')); -- [SPARK-28024] Incorrect numeric values when out of range -- test for underflow and overflow handling @@ -227,15 +229,17 @@ SELECT atanh(double('NaN')); TRUNCATE TABLE FLOAT8_TBL; -INSERT INTO FLOAT8_TBL VALUES ('0.0'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double('0.0')); -INSERT INTO FLOAT8_TBL VALUES ('-34.84'); +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')); -INSERT INTO FLOAT8_TBL VALUES ('-1004.30'); +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')); -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200'); +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')); -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200'); +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')); SELECT '' AS five, * FROM FLOAT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql index f64ec5d75afcf..07f5976ca6d2f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql @@ -8,19 +8,23 @@ CREATE TABLE INT2_TBL(f1 smallint) USING parquet; -- [SPARK-28023] Trim the string when cast string type to other types -INSERT INTO INT2_TBL VALUES (trim('0 ')); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))); -INSERT INTO INT2_TBL VALUES (trim(' 1234 ')); +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))); -INSERT INTO INT2_TBL VALUES (trim(' -1234')); +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))); -- [SPARK-27923] Invalid input syntax for type short throws exception at PostgreSQL -- INSERT INTO INT2_TBL VALUES ('34.5'); -- largest and smallest values -INSERT INTO INT2_TBL VALUES ('32767'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint('32767')); -INSERT INTO INT2_TBL VALUES ('-32767'); +INSERT INTO INT2_TBL VALUES (smallint('-32767')); -- bad input values -- should give errors -- INSERT INTO INT2_TBL VALUES ('100000'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql index 1c2320ff7fad6..3a409eea34837 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql @@ -9,19 +9,23 @@ CREATE TABLE INT4_TBL(f1 int) USING parquet; -- [SPARK-28023] Trim the string when cast string type to other types -INSERT INTO INT4_TBL VALUES (trim(' 0 ')); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))); -INSERT INTO INT4_TBL VALUES (trim('123456 ')); +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))); -INSERT INTO INT4_TBL VALUES (trim(' -123456')); +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))); -- [SPARK-27923] Invalid input syntax for integer: "34.5" at PostgreSQL -- INSERT INTO INT4_TBL(f1) VALUES ('34.5'); -- largest and smallest values -INSERT INTO INT4_TBL VALUES ('2147483647'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int('2147483647')); -INSERT INTO INT4_TBL VALUES ('-2147483647'); +INSERT INTO INT4_TBL VALUES (int('-2147483647')); -- [SPARK-27923] Spark SQL insert these bad inputs to NULL -- bad input values diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql index d29bf3bfad4ca..5fea758e73084 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql @@ -8,11 +8,13 @@ -- CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; -INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')); -INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789'); -INSERT INTO INT8_TBL VALUES('4567890123456789','123'); -INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789'); -INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))); +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')); +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')); -- [SPARK-27923] Spark SQL insert there bad inputs to NULL -- bad inputs diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql index c447a0dc2c7f2..dbdb2cace0e0c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql @@ -26,466 +26,508 @@ CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet; -- ****************************** -- BEGIN TRANSACTION; -INSERT INTO num_exp_add VALUES (0,0,'0'); -INSERT INTO num_exp_sub VALUES (0,0,'0'); -INSERT INTO num_exp_mul VALUES (0,0,'0'); -INSERT INTO num_exp_div VALUES (0,0,'NaN'); -INSERT INTO num_exp_add VALUES (0,1,'0'); -INSERT INTO num_exp_sub VALUES (0,1,'0'); -INSERT INTO num_exp_mul VALUES (0,1,'0'); -INSERT INTO num_exp_div VALUES (0,1,'NaN'); -INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047'); -INSERT INTO num_exp_mul VALUES (0,2,'0'); -INSERT INTO num_exp_div VALUES (0,2,'0'); -INSERT INTO num_exp_add VALUES (0,3,'4.31'); -INSERT INTO num_exp_sub VALUES (0,3,'-4.31'); -INSERT INTO num_exp_mul VALUES (0,3,'0'); -INSERT INTO num_exp_div VALUES (0,3,'0'); -INSERT INTO num_exp_add VALUES (0,4,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119'); -INSERT INTO num_exp_mul VALUES (0,4,'0'); -INSERT INTO num_exp_div VALUES (0,4,'0'); -INSERT INTO num_exp_add VALUES (0,5,'16397.038491'); -INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491'); -INSERT INTO num_exp_mul VALUES (0,5,'0'); -INSERT INTO num_exp_div VALUES (0,5,'0'); -INSERT INTO num_exp_add VALUES (0,6,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026'); -INSERT INTO num_exp_mul VALUES (0,6,'0'); -INSERT INTO num_exp_div VALUES (0,6,'0'); -INSERT INTO num_exp_add VALUES (0,7,'-83028485'); -INSERT INTO num_exp_sub VALUES (0,7,'83028485'); -INSERT INTO num_exp_mul VALUES (0,7,'0'); -INSERT INTO num_exp_div VALUES (0,7,'0'); -INSERT INTO num_exp_add VALUES (0,8,'74881'); -INSERT INTO num_exp_sub VALUES (0,8,'-74881'); -INSERT INTO num_exp_mul VALUES (0,8,'0'); -INSERT INTO num_exp_div VALUES (0,8,'0'); -INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420'); -INSERT INTO num_exp_mul VALUES (0,9,'0'); -INSERT INTO num_exp_div VALUES (0,9,'0'); -INSERT INTO num_exp_add VALUES (1,0,'0'); -INSERT INTO num_exp_sub VALUES (1,0,'0'); -INSERT INTO num_exp_mul VALUES (1,0,'0'); -INSERT INTO num_exp_div VALUES (1,0,'NaN'); -INSERT INTO num_exp_add VALUES (1,1,'0'); -INSERT INTO num_exp_sub VALUES (1,1,'0'); -INSERT INTO num_exp_mul VALUES (1,1,'0'); -INSERT INTO num_exp_div VALUES (1,1,'NaN'); -INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047'); -INSERT INTO num_exp_mul VALUES (1,2,'0'); -INSERT INTO num_exp_div VALUES (1,2,'0'); -INSERT INTO num_exp_add VALUES (1,3,'4.31'); -INSERT INTO num_exp_sub VALUES (1,3,'-4.31'); -INSERT INTO num_exp_mul VALUES (1,3,'0'); -INSERT INTO num_exp_div VALUES (1,3,'0'); -INSERT INTO num_exp_add VALUES (1,4,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119'); -INSERT INTO num_exp_mul VALUES (1,4,'0'); -INSERT INTO num_exp_div VALUES (1,4,'0'); -INSERT INTO num_exp_add VALUES (1,5,'16397.038491'); -INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491'); -INSERT INTO num_exp_mul VALUES (1,5,'0'); -INSERT INTO num_exp_div VALUES (1,5,'0'); -INSERT INTO num_exp_add VALUES (1,6,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026'); -INSERT INTO num_exp_mul VALUES (1,6,'0'); -INSERT INTO num_exp_div VALUES (1,6,'0'); -INSERT INTO num_exp_add VALUES (1,7,'-83028485'); -INSERT INTO num_exp_sub VALUES (1,7,'83028485'); -INSERT INTO num_exp_mul VALUES (1,7,'0'); -INSERT INTO num_exp_div VALUES (1,7,'0'); -INSERT INTO num_exp_add VALUES (1,8,'74881'); -INSERT INTO num_exp_sub VALUES (1,8,'-74881'); -INSERT INTO num_exp_mul VALUES (1,8,'0'); -INSERT INTO num_exp_div VALUES (1,8,'0'); -INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420'); -INSERT INTO num_exp_mul VALUES (1,9,'0'); -INSERT INTO num_exp_div VALUES (1,9,'0'); -INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047'); -INSERT INTO num_exp_mul VALUES (2,0,'0'); -INSERT INTO num_exp_div VALUES (2,0,'NaN'); -INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047'); -INSERT INTO num_exp_mul VALUES (2,1,'0'); -INSERT INTO num_exp_div VALUES (2,1,'NaN'); -INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094'); -INSERT INTO num_exp_sub VALUES (2,2,'0'); -INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209'); -INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047'); -INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047'); -INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257'); -INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266'); -INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047'); -INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047'); -INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593'); -INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685'); -INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047'); -INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047'); -INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077'); -INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429'); -INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787'); -INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307'); -INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222'); -INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940'); -INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047'); -INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953'); -INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795'); -INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518'); -INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047'); -INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047'); -INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407'); -INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476'); -INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467'); -INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627'); -INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740'); -INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811'); -INSERT INTO num_exp_add VALUES (3,0,'4.31'); -INSERT INTO num_exp_sub VALUES (3,0,'4.31'); -INSERT INTO num_exp_mul VALUES (3,0,'0'); -INSERT INTO num_exp_div VALUES (3,0,'NaN'); -INSERT INTO num_exp_add VALUES (3,1,'4.31'); -INSERT INTO num_exp_sub VALUES (3,1,'4.31'); -INSERT INTO num_exp_mul VALUES (3,1,'0'); -INSERT INTO num_exp_div VALUES (3,1,'NaN'); -INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047'); -INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047'); -INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257'); -INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352'); -INSERT INTO num_exp_add VALUES (3,3,'8.62'); -INSERT INTO num_exp_sub VALUES (3,3,'0'); -INSERT INTO num_exp_mul VALUES (3,3,'18.5761'); -INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (3,4,'7799465.7219'); -INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019'); -INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289'); -INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552'); -INSERT INTO num_exp_add VALUES (3,5,'16401.348491'); -INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491'); -INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621'); -INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504'); -INSERT INTO num_exp_add VALUES (3,6,'93905.88763026'); -INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026'); -INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206'); -INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595'); -INSERT INTO num_exp_add VALUES (3,7,'-83028480.69'); -INSERT INTO num_exp_sub VALUES (3,7,'83028489.31'); -INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35'); -INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240'); -INSERT INTO num_exp_add VALUES (3,8,'74885.31'); -INSERT INTO num_exp_sub VALUES (3,8,'-74876.69'); -INSERT INTO num_exp_mul VALUES (3,8,'322737.11'); -INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553'); -INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420'); -INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420'); -INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020'); -INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854'); -INSERT INTO num_exp_add VALUES (4,0,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119'); -INSERT INTO num_exp_mul VALUES (4,0,'0'); -INSERT INTO num_exp_div VALUES (4,0,'NaN'); -INSERT INTO num_exp_add VALUES (4,1,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119'); -INSERT INTO num_exp_mul VALUES (4,1,'0'); -INSERT INTO num_exp_div VALUES (4,1,'NaN'); -INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047'); -INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047'); -INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593'); -INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385'); -INSERT INTO num_exp_add VALUES (4,3,'7799465.7219'); -INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019'); -INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289'); -INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883'); -INSERT INTO num_exp_add VALUES (4,4,'15598922.8238'); -INSERT INTO num_exp_sub VALUES (4,4,'0'); -INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161'); -INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (4,5,'7815858.450391'); -INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409'); -INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429'); -INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061'); -INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026'); -INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974'); -INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094'); -INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606'); -INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881'); -INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119'); -INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715'); -INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637'); -INSERT INTO num_exp_add VALUES (4,8,'7874342.4119'); -INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119'); -INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839'); -INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143'); -INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420'); -INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420'); -INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980'); -INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409'); -INSERT INTO num_exp_add VALUES (5,0,'16397.038491'); -INSERT INTO num_exp_sub VALUES (5,0,'16397.038491'); -INSERT INTO num_exp_mul VALUES (5,0,'0'); -INSERT INTO num_exp_div VALUES (5,0,'NaN'); -INSERT INTO num_exp_add VALUES (5,1,'16397.038491'); -INSERT INTO num_exp_sub VALUES (5,1,'16397.038491'); -INSERT INTO num_exp_mul VALUES (5,1,'0'); -INSERT INTO num_exp_div VALUES (5,1,'NaN'); -INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047'); -INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047'); -INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077'); -INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446'); -INSERT INTO num_exp_add VALUES (5,3,'16401.348491'); -INSERT INTO num_exp_sub VALUES (5,3,'16392.728491'); -INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621'); -INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584'); -INSERT INTO num_exp_add VALUES (5,4,'7815858.450391'); -INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409'); -INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429'); -INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192'); -INSERT INTO num_exp_add VALUES (5,5,'32794.076982'); -INSERT INTO num_exp_sub VALUES (5,5,'0'); -INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081'); -INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (5,6,'110298.61612126'); -INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926'); -INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766'); -INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689'); -INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509'); -INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491'); -INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135'); -INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710'); -INSERT INTO num_exp_add VALUES (5,8,'91278.038491'); -INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509'); -INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571'); -INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228'); -INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420'); -INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420'); -INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220'); -INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427'); -INSERT INTO num_exp_add VALUES (6,0,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026'); -INSERT INTO num_exp_mul VALUES (6,0,'0'); -INSERT INTO num_exp_div VALUES (6,0,'NaN'); -INSERT INTO num_exp_add VALUES (6,1,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026'); -INSERT INTO num_exp_mul VALUES (6,1,'0'); -INSERT INTO num_exp_div VALUES (6,1,'NaN'); -INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787'); -INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307'); -INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222'); -INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823'); -INSERT INTO num_exp_add VALUES (6,3,'93905.88763026'); -INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026'); -INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206'); -INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907'); -INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026'); -INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974'); -INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094'); -INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469'); -INSERT INTO num_exp_add VALUES (6,5,'110298.61612126'); -INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926'); -INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766'); -INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679'); -INSERT INTO num_exp_add VALUES (6,6,'187803.15526052'); -INSERT INTO num_exp_sub VALUES (6,6,'0'); -INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676'); -INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974'); -INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026'); -INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610'); -INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980'); -INSERT INTO num_exp_add VALUES (6,8,'168782.57763026'); -INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026'); -INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906'); -INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184'); -INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160'); -INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680'); -INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920'); -INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789'); -INSERT INTO num_exp_add VALUES (7,0,'-83028485'); -INSERT INTO num_exp_sub VALUES (7,0,'-83028485'); -INSERT INTO num_exp_mul VALUES (7,0,'0'); -INSERT INTO num_exp_div VALUES (7,0,'NaN'); -INSERT INTO num_exp_add VALUES (7,1,'-83028485'); -INSERT INTO num_exp_sub VALUES (7,1,'-83028485'); -INSERT INTO num_exp_mul VALUES (7,1,'0'); -INSERT INTO num_exp_div VALUES (7,1,'NaN'); -INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047'); -INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953'); -INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795'); -INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700'); -INSERT INTO num_exp_add VALUES (7,3,'-83028480.69'); -INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31'); -INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35'); -INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974'); -INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881'); -INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119'); -INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715'); -INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686'); -INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509'); -INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491'); -INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135'); -INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574'); -INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974'); -INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026'); -INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610'); -INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294'); -INSERT INTO num_exp_add VALUES (7,7,'-166056970'); -INSERT INTO num_exp_sub VALUES (7,7,'0'); -INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225'); -INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (7,8,'-82953604'); -INSERT INTO num_exp_sub VALUES (7,8,'-83103366'); -INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285'); -INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118'); -INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420'); -INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580'); -INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700'); -INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382'); -INSERT INTO num_exp_add VALUES (8,0,'74881'); -INSERT INTO num_exp_sub VALUES (8,0,'74881'); -INSERT INTO num_exp_mul VALUES (8,0,'0'); -INSERT INTO num_exp_div VALUES (8,0,'NaN'); -INSERT INTO num_exp_add VALUES (8,1,'74881'); -INSERT INTO num_exp_sub VALUES (8,1,'74881'); -INSERT INTO num_exp_mul VALUES (8,1,'0'); -INSERT INTO num_exp_div VALUES (8,1,'NaN'); -INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047'); -INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047'); -INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407'); -INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615'); -INSERT INTO num_exp_add VALUES (8,3,'74885.31'); -INSERT INTO num_exp_sub VALUES (8,3,'74876.69'); -INSERT INTO num_exp_mul VALUES (8,3,'322737.11'); -INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410'); -INSERT INTO num_exp_add VALUES (8,4,'7874342.4119'); -INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119'); -INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839'); -INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956'); -INSERT INTO num_exp_add VALUES (8,5,'91278.038491'); -INSERT INTO num_exp_sub VALUES (8,5,'58483.961509'); -INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571'); -INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456'); -INSERT INTO num_exp_add VALUES (8,6,'168782.57763026'); -INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026'); -INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906'); -INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424'); -INSERT INTO num_exp_add VALUES (8,7,'-82953604'); -INSERT INTO num_exp_sub VALUES (8,7,'83103366'); -INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285'); -INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172'); -INSERT INTO num_exp_add VALUES (8,8,'149762'); -INSERT INTO num_exp_sub VALUES (8,8,'0'); -INSERT INTO num_exp_mul VALUES (8,8,'5607164161'); -INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420'); -INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420'); -INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020'); -INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735'); -INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420'); -INSERT INTO num_exp_mul VALUES (9,0,'0'); -INSERT INTO num_exp_div VALUES (9,0,'NaN'); -INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420'); -INSERT INTO num_exp_mul VALUES (9,1,'0'); -INSERT INTO num_exp_div VALUES (9,1,'NaN'); -INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467'); -INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627'); -INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740'); -INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526'); -INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420'); -INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420'); -INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020'); -INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677'); -INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420'); -INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420'); -INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980'); -INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484'); -INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420'); -INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420'); -INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220'); -INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807'); -INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160'); -INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680'); -INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920'); -INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280'); -INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420'); -INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580'); -INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700'); -INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689'); -INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420'); -INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420'); -INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020'); -INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748'); -INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840'); -INSERT INTO num_exp_sub VALUES (9,9,'0'); -INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400'); -INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_add VALUES (0,0,0); +INSERT INTO num_exp_sub VALUES (0,0,0); +INSERT INTO num_exp_mul VALUES (0,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,0,double('NaN')); +INSERT INTO num_exp_add VALUES (0,1,0); +INSERT INTO num_exp_sub VALUES (0,1,0); +INSERT INTO num_exp_mul VALUES (0,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,1,double('NaN')); +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (0,2,0); +INSERT INTO num_exp_div VALUES (0,2,0); +INSERT INTO num_exp_add VALUES (0,3,4.31); +INSERT INTO num_exp_sub VALUES (0,3,-4.31); +INSERT INTO num_exp_mul VALUES (0,3,0); +INSERT INTO num_exp_div VALUES (0,3,0); +INSERT INTO num_exp_add VALUES (0,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (0,4,0); +INSERT INTO num_exp_div VALUES (0,4,0); +INSERT INTO num_exp_add VALUES (0,5,16397.038491); +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (0,5,0); +INSERT INTO num_exp_div VALUES (0,5,0); +INSERT INTO num_exp_add VALUES (0,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (0,6,0); +INSERT INTO num_exp_div VALUES (0,6,0); +INSERT INTO num_exp_add VALUES (0,7,-83028485); +INSERT INTO num_exp_sub VALUES (0,7,83028485); +INSERT INTO num_exp_mul VALUES (0,7,0); +INSERT INTO num_exp_div VALUES (0,7,0); +INSERT INTO num_exp_add VALUES (0,8,74881); +INSERT INTO num_exp_sub VALUES (0,8,-74881); +INSERT INTO num_exp_mul VALUES (0,8,0); +INSERT INTO num_exp_div VALUES (0,8,0); +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (0,9,0); +INSERT INTO num_exp_div VALUES (0,9,0); +INSERT INTO num_exp_add VALUES (1,0,0); +INSERT INTO num_exp_sub VALUES (1,0,0); +INSERT INTO num_exp_mul VALUES (1,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,0,double('NaN')); +INSERT INTO num_exp_add VALUES (1,1,0); +INSERT INTO num_exp_sub VALUES (1,1,0); +INSERT INTO num_exp_mul VALUES (1,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,1,double('NaN')); +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (1,2,0); +INSERT INTO num_exp_div VALUES (1,2,0); +INSERT INTO num_exp_add VALUES (1,3,4.31); +INSERT INTO num_exp_sub VALUES (1,3,-4.31); +INSERT INTO num_exp_mul VALUES (1,3,0); +INSERT INTO num_exp_div VALUES (1,3,0); +INSERT INTO num_exp_add VALUES (1,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (1,4,0); +INSERT INTO num_exp_div VALUES (1,4,0); +INSERT INTO num_exp_add VALUES (1,5,16397.038491); +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (1,5,0); +INSERT INTO num_exp_div VALUES (1,5,0); +INSERT INTO num_exp_add VALUES (1,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (1,6,0); +INSERT INTO num_exp_div VALUES (1,6,0); +INSERT INTO num_exp_add VALUES (1,7,-83028485); +INSERT INTO num_exp_sub VALUES (1,7,83028485); +INSERT INTO num_exp_mul VALUES (1,7,0); +INSERT INTO num_exp_div VALUES (1,7,0); +INSERT INTO num_exp_add VALUES (1,8,74881); +INSERT INTO num_exp_sub VALUES (1,8,-74881); +INSERT INTO num_exp_mul VALUES (1,8,0); +INSERT INTO num_exp_div VALUES (1,8,0); +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (1,9,0); +INSERT INTO num_exp_div VALUES (1,9,0); +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,0,double('NaN')); +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,1,double('NaN')); +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094); +INSERT INTO num_exp_sub VALUES (2,2,0); +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209); +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047); +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266); +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047); +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685); +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047); +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429); +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307); +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940); +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953); +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518); +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047); +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476); +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627); +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811); +INSERT INTO num_exp_add VALUES (3,0,4.31); +INSERT INTO num_exp_sub VALUES (3,0,4.31); +INSERT INTO num_exp_mul VALUES (3,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,0,double('NaN')); +INSERT INTO num_exp_add VALUES (3,1,4.31); +INSERT INTO num_exp_sub VALUES (3,1,4.31); +INSERT INTO num_exp_mul VALUES (3,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,1,double('NaN')); +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047); +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352); +INSERT INTO num_exp_add VALUES (3,3,8.62); +INSERT INTO num_exp_sub VALUES (3,3,0); +INSERT INTO num_exp_mul VALUES (3,3,18.5761); +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (3,4,7799465.7219); +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019); +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289); +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552); +INSERT INTO num_exp_add VALUES (3,5,16401.348491); +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491); +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621); +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504); +INSERT INTO num_exp_add VALUES (3,6,93905.88763026); +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026); +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206); +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595); +INSERT INTO num_exp_add VALUES (3,7,-83028480.69); +INSERT INTO num_exp_sub VALUES (3,7,83028489.31); +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35); +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240); +INSERT INTO num_exp_add VALUES (3,8,74885.31); +INSERT INTO num_exp_sub VALUES (3,8,-74876.69); +INSERT INTO num_exp_mul VALUES (3,8,322737.11); +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553); +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420); +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854); +INSERT INTO num_exp_add VALUES (4,0,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,0,double('NaN')); +INSERT INTO num_exp_add VALUES (4,1,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,1,double('NaN')); +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047); +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385); +INSERT INTO num_exp_add VALUES (4,3,7799465.7219); +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019); +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289); +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883); +INSERT INTO num_exp_add VALUES (4,4,15598922.8238); +INSERT INTO num_exp_sub VALUES (4,4,0); +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161); +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (4,5,7815858.450391); +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409); +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061); +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026); +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974); +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606); +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881); +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119); +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637); +INSERT INTO num_exp_add VALUES (4,8,7874342.4119); +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119); +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839); +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143); +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420); +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409); +INSERT INTO num_exp_add VALUES (5,0,16397.038491); +INSERT INTO num_exp_sub VALUES (5,0,16397.038491); +INSERT INTO num_exp_mul VALUES (5,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,0,double('NaN')); +INSERT INTO num_exp_add VALUES (5,1,16397.038491); +INSERT INTO num_exp_sub VALUES (5,1,16397.038491); +INSERT INTO num_exp_mul VALUES (5,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,1,double('NaN')); +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047); +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446); +INSERT INTO num_exp_add VALUES (5,3,16401.348491); +INSERT INTO num_exp_sub VALUES (5,3,16392.728491); +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621); +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584); +INSERT INTO num_exp_add VALUES (5,4,7815858.450391); +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409); +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192); +INSERT INTO num_exp_add VALUES (5,5,32794.076982); +INSERT INTO num_exp_sub VALUES (5,5,0); +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081); +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (5,6,110298.61612126); +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926); +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689); +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509); +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491); +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710); +INSERT INTO num_exp_add VALUES (5,8,91278.038491); +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509); +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571); +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228); +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420); +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427); +INSERT INTO num_exp_add VALUES (6,0,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,0,double('NaN')); +INSERT INTO num_exp_add VALUES (6,1,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,1,double('NaN')); +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307); +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823); +INSERT INTO num_exp_add VALUES (6,3,93905.88763026); +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026); +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206); +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907); +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026); +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974); +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469); +INSERT INTO num_exp_add VALUES (6,5,110298.61612126); +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926); +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679); +INSERT INTO num_exp_add VALUES (6,6,187803.15526052); +INSERT INTO num_exp_sub VALUES (6,6,0); +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676); +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026); +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980); +INSERT INTO num_exp_add VALUES (6,8,168782.57763026); +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026); +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906); +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184); +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680); +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789); +INSERT INTO num_exp_add VALUES (7,0,-83028485); +INSERT INTO num_exp_sub VALUES (7,0,-83028485); +INSERT INTO num_exp_mul VALUES (7,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,0,double('NaN')); +INSERT INTO num_exp_add VALUES (7,1,-83028485); +INSERT INTO num_exp_sub VALUES (7,1,-83028485); +INSERT INTO num_exp_mul VALUES (7,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,1,double('NaN')); +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953); +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700); +INSERT INTO num_exp_add VALUES (7,3,-83028480.69); +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31); +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35); +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974); +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881); +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119); +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686); +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509); +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491); +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574); +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026); +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294); +INSERT INTO num_exp_add VALUES (7,7,-166056970); +INSERT INTO num_exp_sub VALUES (7,7,0); +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225); +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (7,8,-82953604); +INSERT INTO num_exp_sub VALUES (7,8,-83103366); +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285); +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118); +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580); +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382); +INSERT INTO num_exp_add VALUES (8,0,74881); +INSERT INTO num_exp_sub VALUES (8,0,74881); +INSERT INTO num_exp_mul VALUES (8,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,0,double('NaN')); +INSERT INTO num_exp_add VALUES (8,1,74881); +INSERT INTO num_exp_sub VALUES (8,1,74881); +INSERT INTO num_exp_mul VALUES (8,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,1,double('NaN')); +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047); +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615); +INSERT INTO num_exp_add VALUES (8,3,74885.31); +INSERT INTO num_exp_sub VALUES (8,3,74876.69); +INSERT INTO num_exp_mul VALUES (8,3,322737.11); +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410); +INSERT INTO num_exp_add VALUES (8,4,7874342.4119); +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119); +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839); +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956); +INSERT INTO num_exp_add VALUES (8,5,91278.038491); +INSERT INTO num_exp_sub VALUES (8,5,58483.961509); +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571); +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456); +INSERT INTO num_exp_add VALUES (8,6,168782.57763026); +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026); +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906); +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424); +INSERT INTO num_exp_add VALUES (8,7,-82953604); +INSERT INTO num_exp_sub VALUES (8,7,83103366); +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285); +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172); +INSERT INTO num_exp_add VALUES (8,8,149762); +INSERT INTO num_exp_sub VALUES (8,8,0); +INSERT INTO num_exp_mul VALUES (8,8,5607164161); +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420); +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735); +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,0,double('NaN')); +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,1,double('NaN')); +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627); +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526); +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420); +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677); +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420); +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484); +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420); +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807); +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680); +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280); +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580); +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689); +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420); +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748); +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840); +INSERT INTO num_exp_sub VALUES (9,9,0); +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400); +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_sqrt VALUES (0,'0'); -INSERT INTO num_exp_sqrt VALUES (1,'0'); -INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505'); -INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396'); -INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923'); -INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473'); -INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406'); -INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230'); -INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542'); -INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_sqrt VALUES (0,0); +INSERT INTO num_exp_sqrt VALUES (1,0); +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505); +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396); +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923); +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473); +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406); +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230); +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542); +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_ln VALUES (0,'NaN'); -INSERT INTO num_exp_ln VALUES (1,'NaN'); -INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514'); -INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971'); -INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464'); -INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038'); -INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127'); -INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991'); -INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668'); -INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514); +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971); +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464); +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038); +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127); +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991); +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668); +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_log10 VALUES (0,'NaN'); -INSERT INTO num_exp_log10 VALUES (1,'NaN'); -INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459'); -INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075'); -INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345'); -INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626'); -INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671'); -INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914'); -INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138'); -INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_log10 VALUES (0,double('NaN')); +INSERT INTO num_exp_log10 VALUES (1,double('NaN')); +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459); +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075); +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345); +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626); +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671); +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914); +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138); +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_power_10_ln VALUES (0,'NaN'); -INSERT INTO num_exp_power_10_ln VALUES (1,'NaN'); -INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184'); -INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393'); -INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636'); -INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098'); -INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067'); -INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627'); -INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952'); -INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184); +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393); +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636); +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098); +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067); +-- In Spark, decimal can only support precision up to 38 +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627); +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952); +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_data VALUES (0, '0'); -INSERT INTO num_data VALUES (1, '0'); -INSERT INTO num_data VALUES (2, '-34338492.215397047'); -INSERT INTO num_data VALUES (3, '4.31'); -INSERT INTO num_data VALUES (4, '7799461.4119'); -INSERT INTO num_data VALUES (5, '16397.038491'); -INSERT INTO num_data VALUES (6, '93901.57763026'); -INSERT INTO num_data VALUES (7, '-83028485'); -INSERT INTO num_data VALUES (8, '74881'); -INSERT INTO num_data VALUES (9, '-24926804.045047420'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_data VALUES (0, 0); +INSERT INTO num_data VALUES (1, 0); +INSERT INTO num_data VALUES (2, -34338492.215397047); +INSERT INTO num_data VALUES (3, 4.31); +INSERT INTO num_data VALUES (4, 7799461.4119); +INSERT INTO num_data VALUES (5, 16397.038491); +INSERT INTO num_data VALUES (6, 93901.57763026); +INSERT INTO num_data VALUES (7, -83028485); +INSERT INTO num_data VALUES (8, 74881); +INSERT INTO num_data VALUES (9, -24926804.045047420); -- COMMIT TRANSACTION; SELECT * FROM num_data; @@ -657,16 +699,22 @@ SELECT AVG(val) FROM num_data; -- Check for appropriate rounding and overflow CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet; -INSERT INTO fract_only VALUES (1, '0.0'); -INSERT INTO fract_only VALUES (2, '0.1'); +INSERT INTO fract_only VALUES (1, 0.0); +INSERT INTO fract_only VALUES (2, 0.1); -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL -- INSERT INTO fract_only VALUES (3, '1.0'); -- should fail -INSERT INTO fract_only VALUES (4, '-0.9999'); -INSERT INTO fract_only VALUES (5, '0.99994'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (4, -0.9999); +INSERT INTO fract_only VALUES (5, 0.99994); -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL -- INSERT INTO fract_only VALUES (6, '0.99995'); -- should fail -INSERT INTO fract_only VALUES (7, '0.00001'); -INSERT INTO fract_only VALUES (8, '0.00017'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (7, 0.00001); +INSERT INTO fract_only VALUES (8, 0.00017); SELECT * FROM fract_only; DROP TABLE fract_only; @@ -682,13 +730,16 @@ SELECT decimal(float('-Infinity')); -- Simple check that ceil(), floor(), and round() work correctly CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet; -INSERT INTO ceil_floor_round VALUES ('-5.5'); -INSERT INTO ceil_floor_round VALUES ('-5.499999'); -INSERT INTO ceil_floor_round VALUES ('9.5'); -INSERT INTO ceil_floor_round VALUES ('9.4999999'); -INSERT INTO ceil_floor_round VALUES ('0.0'); -INSERT INTO ceil_floor_round VALUES ('0.0000001'); -INSERT INTO ceil_floor_round VALUES ('-0.000001'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO ceil_floor_round VALUES (-5.5); +INSERT INTO ceil_floor_round VALUES (-5.499999); +INSERT INTO ceil_floor_round VALUES (9.5); +INSERT INTO ceil_floor_round VALUES (9.4999999); +INSERT INTO ceil_floor_round VALUES (0.0); +INSERT INTO ceil_floor_round VALUES (0.0000001); +INSERT INTO ceil_floor_round VALUES (-0.000001); SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round; DROP TABLE ceil_floor_round; @@ -853,11 +904,14 @@ DROP TABLE ceil_floor_round; CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet; -- good inputs -INSERT INTO num_input_test VALUES (trim(' 123')); -INSERT INTO num_input_test VALUES (trim(' 3245874 ')); -INSERT INTO num_input_test VALUES (trim(' -93853')); -INSERT INTO num_input_test VALUES ('555.50'); -INSERT INTO num_input_test VALUES ('-555.50'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_input_test VALUES (double(trim(' 123'))); +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))); +INSERT INTO num_input_test VALUES (double(trim(' -93853'))); +INSERT INTO num_input_test VALUES (555.50); +INSERT INTO num_input_test VALUES (-555.50); -- [SPARK-28315] Decimal can not accept NaN as input -- INSERT INTO num_input_test VALUES (trim('NaN ')); -- INSERT INTO num_input_test VALUES (trim(' nan')); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql index 260e8ea93d22d..bf69da295a960 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql @@ -16,19 +16,23 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- block is entered exactly at local midnight; then 'now' and 'today' have -- the same values and the counts will come out different. -INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')); -- SELECT pg_sleep(0.1); -- BEGIN; -INSERT INTO TIMESTAMP_TBL VALUES ('now'); -INSERT INTO TIMESTAMP_TBL VALUES ('today'); -INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('today')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('yesterday')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow')); -- time zone should be ignored by this data type -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow EST')); -- [SPARK-29024] Ignore case while resolving time zones -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow Zulu')); SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; @@ -54,7 +58,9 @@ TRUNCATE TABLE TIMESTAMP_TBL; -- Special values -- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('epoch')); -- [SPARK-27923] Spark SQL insert there obsolete special values to NULL -- Obsolete special values -- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); @@ -73,14 +79,16 @@ INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); -- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); -- ISO 8601 format -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); -- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')); -- POSIX format (note that the timezone abbrev is just decoration here) -- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 29fcf61bd5b78..68ff2a9f68899 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO DATE_TBL VALUES ('1957-04-09') +INSERT INTO DATE_TBL VALUES (date('1957-04-09')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO DATE_TBL VALUES ('1957-06-13') +INSERT INTO DATE_TBL VALUES (date('1957-06-13')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO DATE_TBL VALUES ('1996-02-28') +INSERT INTO DATE_TBL VALUES (date('1996-02-28')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO DATE_TBL VALUES ('1996-02-29') +INSERT INTO DATE_TBL VALUES (date('1996-02-29')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO DATE_TBL VALUES ('1996-03-01') +INSERT INTO DATE_TBL VALUES (date('1996-03-01')) -- !query 5 schema struct<> -- !query 5 output @@ -51,7 +51,7 @@ struct<> -- !query 6 -INSERT INTO DATE_TBL VALUES ('1996-03-02') +INSERT INTO DATE_TBL VALUES (date('1996-03-02')) -- !query 6 schema struct<> -- !query 6 output @@ -59,7 +59,7 @@ struct<> -- !query 7 -INSERT INTO DATE_TBL VALUES ('1997-02-28') +INSERT INTO DATE_TBL VALUES (date('1997-02-28')) -- !query 7 schema struct<> -- !query 7 output @@ -67,7 +67,7 @@ struct<> -- !query 8 -INSERT INTO DATE_TBL VALUES ('1997-03-01') +INSERT INTO DATE_TBL VALUES (date('1997-03-01')) -- !query 8 schema struct<> -- !query 8 output @@ -75,7 +75,7 @@ struct<> -- !query 9 -INSERT INTO DATE_TBL VALUES ('1997-03-02') +INSERT INTO DATE_TBL VALUES (date('1997-03-02')) -- !query 9 schema struct<> -- !query 9 output @@ -83,7 +83,7 @@ struct<> -- !query 10 -INSERT INTO DATE_TBL VALUES ('2000-04-01') +INSERT INTO DATE_TBL VALUES (date('2000-04-01')) -- !query 10 schema struct<> -- !query 10 output @@ -91,7 +91,7 @@ struct<> -- !query 11 -INSERT INTO DATE_TBL VALUES ('2000-04-02') +INSERT INTO DATE_TBL VALUES (date('2000-04-02')) -- !query 11 schema struct<> -- !query 11 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO DATE_TBL VALUES ('2000-04-03') +INSERT INTO DATE_TBL VALUES (date('2000-04-03')) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO DATE_TBL VALUES ('2038-04-08') +INSERT INTO DATE_TBL VALUES (date('2038-04-08')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO DATE_TBL VALUES ('2039-04-09') +INSERT INTO DATE_TBL VALUES (date('2039-04-09')) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO DATE_TBL VALUES ('2040-04-10') +INSERT INTO DATE_TBL VALUES (date('2040-04-10')) -- !query 15 schema struct<> -- !query 15 output @@ -582,7 +582,7 @@ struct -- !query 55 SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema -struct +struct -- !query 55 output 0 @@ -790,7 +790,7 @@ true -- !query 81 SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 81 schema -struct +struct -- !query 81 output 20 @@ -798,7 +798,7 @@ struct -- !query 82 SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 82 schema -struct +struct -- !query 82 output 1001-01-01 00:07:02 @@ -814,7 +814,7 @@ struct -- !query 84 SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 84 schema -struct +struct -- !query 84 output 1901-01-01 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index c205f70a65f2d..64608a349b610 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO FLOAT4_TBL VALUES (' 0.0') +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO FLOAT4_TBL VALUES ('1004.30 ') +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO FLOAT4_TBL VALUES (' -34.84 ') +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20') +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20') +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index 9a785670fa950..d38e36e956985 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO FLOAT8_TBL VALUES (' 0.0 ') +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO FLOAT8_TBL VALUES ('1004.30 ') +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO FLOAT8_TBL VALUES (' -34.84') +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200') +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200') +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')) -- !query 5 schema struct<> -- !query 5 output @@ -684,7 +684,7 @@ struct<> -- !query 76 -INSERT INTO FLOAT8_TBL VALUES ('0.0') +INSERT INTO FLOAT8_TBL VALUES (double('0.0')) -- !query 76 schema struct<> -- !query 76 output @@ -692,7 +692,7 @@ struct<> -- !query 77 -INSERT INTO FLOAT8_TBL VALUES ('-34.84') +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')) -- !query 77 schema struct<> -- !query 77 output @@ -700,7 +700,7 @@ struct<> -- !query 78 -INSERT INTO FLOAT8_TBL VALUES ('-1004.30') +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')) -- !query 78 schema struct<> -- !query 78 output @@ -708,7 +708,7 @@ struct<> -- !query 79 -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200') +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')) -- !query 79 schema struct<> -- !query 79 output @@ -716,7 +716,7 @@ struct<> -- !query 80 -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200') +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')) -- !query 80 schema struct<> -- !query 80 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index 569d137891dd3..8f41accfc8c49 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT2_TBL VALUES (trim('0 ')) +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT2_TBL VALUES (trim(' 1234 ')) +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT2_TBL VALUES (trim(' -1234')) +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT2_TBL VALUES ('32767') +INSERT INTO INT2_TBL VALUES (smallint('32767')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT2_TBL VALUES ('-32767') +INSERT INTO INT2_TBL VALUES (smallint('-32767')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 52221d58392f0..6b1a8eb4a5883 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT4_TBL VALUES (trim(' 0 ')) +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT4_TBL VALUES (trim('123456 ')) +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT4_TBL VALUES (trim(' -123456')) +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT4_TBL VALUES ('2147483647') +INSERT INTO INT4_TBL VALUES (int('2147483647')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT4_TBL VALUES ('-2147483647') +INSERT INTO INT4_TBL VALUES (int('-2147483647')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 20bb5dbda645f..c4d8affe75267 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')) +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789') +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT8_TBL VALUES('4567890123456789','123') +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789') +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789') +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index ed649feaaebb2..5ffa29a93d5de 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -83,7 +83,7 @@ struct<> -- !query 10 -INSERT INTO num_exp_add VALUES (0,0,'0') +INSERT INTO num_exp_add VALUES (0,0,0) -- !query 10 schema struct<> -- !query 10 output @@ -91,7 +91,7 @@ struct<> -- !query 11 -INSERT INTO num_exp_sub VALUES (0,0,'0') +INSERT INTO num_exp_sub VALUES (0,0,0) -- !query 11 schema struct<> -- !query 11 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO num_exp_mul VALUES (0,0,'0') +INSERT INTO num_exp_mul VALUES (0,0,0) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO num_exp_div VALUES (0,0,'NaN') +INSERT INTO num_exp_div VALUES (0,0,double('NaN')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO num_exp_add VALUES (0,1,'0') +INSERT INTO num_exp_add VALUES (0,1,0) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO num_exp_sub VALUES (0,1,'0') +INSERT INTO num_exp_sub VALUES (0,1,0) -- !query 15 schema struct<> -- !query 15 output @@ -131,7 +131,7 @@ struct<> -- !query 16 -INSERT INTO num_exp_mul VALUES (0,1,'0') +INSERT INTO num_exp_mul VALUES (0,1,0) -- !query 16 schema struct<> -- !query 16 output @@ -139,7 +139,7 @@ struct<> -- !query 17 -INSERT INTO num_exp_div VALUES (0,1,'NaN') +INSERT INTO num_exp_div VALUES (0,1,double('NaN')) -- !query 17 schema struct<> -- !query 17 output @@ -147,7 +147,7 @@ struct<> -- !query 18 -INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) -- !query 18 schema struct<> -- !query 18 output @@ -155,7 +155,7 @@ struct<> -- !query 19 -INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047') +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) -- !query 19 schema struct<> -- !query 19 output @@ -163,7 +163,7 @@ struct<> -- !query 20 -INSERT INTO num_exp_mul VALUES (0,2,'0') +INSERT INTO num_exp_mul VALUES (0,2,0) -- !query 20 schema struct<> -- !query 20 output @@ -171,7 +171,7 @@ struct<> -- !query 21 -INSERT INTO num_exp_div VALUES (0,2,'0') +INSERT INTO num_exp_div VALUES (0,2,0) -- !query 21 schema struct<> -- !query 21 output @@ -179,7 +179,7 @@ struct<> -- !query 22 -INSERT INTO num_exp_add VALUES (0,3,'4.31') +INSERT INTO num_exp_add VALUES (0,3,4.31) -- !query 22 schema struct<> -- !query 22 output @@ -187,7 +187,7 @@ struct<> -- !query 23 -INSERT INTO num_exp_sub VALUES (0,3,'-4.31') +INSERT INTO num_exp_sub VALUES (0,3,-4.31) -- !query 23 schema struct<> -- !query 23 output @@ -195,7 +195,7 @@ struct<> -- !query 24 -INSERT INTO num_exp_mul VALUES (0,3,'0') +INSERT INTO num_exp_mul VALUES (0,3,0) -- !query 24 schema struct<> -- !query 24 output @@ -203,7 +203,7 @@ struct<> -- !query 25 -INSERT INTO num_exp_div VALUES (0,3,'0') +INSERT INTO num_exp_div VALUES (0,3,0) -- !query 25 schema struct<> -- !query 25 output @@ -211,7 +211,7 @@ struct<> -- !query 26 -INSERT INTO num_exp_add VALUES (0,4,'7799461.4119') +INSERT INTO num_exp_add VALUES (0,4,7799461.4119) -- !query 26 schema struct<> -- !query 26 output @@ -219,7 +219,7 @@ struct<> -- !query 27 -INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119') +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) -- !query 27 schema struct<> -- !query 27 output @@ -227,7 +227,7 @@ struct<> -- !query 28 -INSERT INTO num_exp_mul VALUES (0,4,'0') +INSERT INTO num_exp_mul VALUES (0,4,0) -- !query 28 schema struct<> -- !query 28 output @@ -235,7 +235,7 @@ struct<> -- !query 29 -INSERT INTO num_exp_div VALUES (0,4,'0') +INSERT INTO num_exp_div VALUES (0,4,0) -- !query 29 schema struct<> -- !query 29 output @@ -243,7 +243,7 @@ struct<> -- !query 30 -INSERT INTO num_exp_add VALUES (0,5,'16397.038491') +INSERT INTO num_exp_add VALUES (0,5,16397.038491) -- !query 30 schema struct<> -- !query 30 output @@ -251,7 +251,7 @@ struct<> -- !query 31 -INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491') +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) -- !query 31 schema struct<> -- !query 31 output @@ -259,7 +259,7 @@ struct<> -- !query 32 -INSERT INTO num_exp_mul VALUES (0,5,'0') +INSERT INTO num_exp_mul VALUES (0,5,0) -- !query 32 schema struct<> -- !query 32 output @@ -267,7 +267,7 @@ struct<> -- !query 33 -INSERT INTO num_exp_div VALUES (0,5,'0') +INSERT INTO num_exp_div VALUES (0,5,0) -- !query 33 schema struct<> -- !query 33 output @@ -275,7 +275,7 @@ struct<> -- !query 34 -INSERT INTO num_exp_add VALUES (0,6,'93901.57763026') +INSERT INTO num_exp_add VALUES (0,6,93901.57763026) -- !query 34 schema struct<> -- !query 34 output @@ -283,7 +283,7 @@ struct<> -- !query 35 -INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026') +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) -- !query 35 schema struct<> -- !query 35 output @@ -291,7 +291,7 @@ struct<> -- !query 36 -INSERT INTO num_exp_mul VALUES (0,6,'0') +INSERT INTO num_exp_mul VALUES (0,6,0) -- !query 36 schema struct<> -- !query 36 output @@ -299,7 +299,7 @@ struct<> -- !query 37 -INSERT INTO num_exp_div VALUES (0,6,'0') +INSERT INTO num_exp_div VALUES (0,6,0) -- !query 37 schema struct<> -- !query 37 output @@ -307,7 +307,7 @@ struct<> -- !query 38 -INSERT INTO num_exp_add VALUES (0,7,'-83028485') +INSERT INTO num_exp_add VALUES (0,7,-83028485) -- !query 38 schema struct<> -- !query 38 output @@ -315,7 +315,7 @@ struct<> -- !query 39 -INSERT INTO num_exp_sub VALUES (0,7,'83028485') +INSERT INTO num_exp_sub VALUES (0,7,83028485) -- !query 39 schema struct<> -- !query 39 output @@ -323,7 +323,7 @@ struct<> -- !query 40 -INSERT INTO num_exp_mul VALUES (0,7,'0') +INSERT INTO num_exp_mul VALUES (0,7,0) -- !query 40 schema struct<> -- !query 40 output @@ -331,7 +331,7 @@ struct<> -- !query 41 -INSERT INTO num_exp_div VALUES (0,7,'0') +INSERT INTO num_exp_div VALUES (0,7,0) -- !query 41 schema struct<> -- !query 41 output @@ -339,7 +339,7 @@ struct<> -- !query 42 -INSERT INTO num_exp_add VALUES (0,8,'74881') +INSERT INTO num_exp_add VALUES (0,8,74881) -- !query 42 schema struct<> -- !query 42 output @@ -347,7 +347,7 @@ struct<> -- !query 43 -INSERT INTO num_exp_sub VALUES (0,8,'-74881') +INSERT INTO num_exp_sub VALUES (0,8,-74881) -- !query 43 schema struct<> -- !query 43 output @@ -355,7 +355,7 @@ struct<> -- !query 44 -INSERT INTO num_exp_mul VALUES (0,8,'0') +INSERT INTO num_exp_mul VALUES (0,8,0) -- !query 44 schema struct<> -- !query 44 output @@ -363,7 +363,7 @@ struct<> -- !query 45 -INSERT INTO num_exp_div VALUES (0,8,'0') +INSERT INTO num_exp_div VALUES (0,8,0) -- !query 45 schema struct<> -- !query 45 output @@ -371,7 +371,7 @@ struct<> -- !query 46 -INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) -- !query 46 schema struct<> -- !query 46 output @@ -379,7 +379,7 @@ struct<> -- !query 47 -INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420') +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) -- !query 47 schema struct<> -- !query 47 output @@ -387,7 +387,7 @@ struct<> -- !query 48 -INSERT INTO num_exp_mul VALUES (0,9,'0') +INSERT INTO num_exp_mul VALUES (0,9,0) -- !query 48 schema struct<> -- !query 48 output @@ -395,7 +395,7 @@ struct<> -- !query 49 -INSERT INTO num_exp_div VALUES (0,9,'0') +INSERT INTO num_exp_div VALUES (0,9,0) -- !query 49 schema struct<> -- !query 49 output @@ -403,7 +403,7 @@ struct<> -- !query 50 -INSERT INTO num_exp_add VALUES (1,0,'0') +INSERT INTO num_exp_add VALUES (1,0,0) -- !query 50 schema struct<> -- !query 50 output @@ -411,7 +411,7 @@ struct<> -- !query 51 -INSERT INTO num_exp_sub VALUES (1,0,'0') +INSERT INTO num_exp_sub VALUES (1,0,0) -- !query 51 schema struct<> -- !query 51 output @@ -419,7 +419,7 @@ struct<> -- !query 52 -INSERT INTO num_exp_mul VALUES (1,0,'0') +INSERT INTO num_exp_mul VALUES (1,0,0) -- !query 52 schema struct<> -- !query 52 output @@ -427,7 +427,7 @@ struct<> -- !query 53 -INSERT INTO num_exp_div VALUES (1,0,'NaN') +INSERT INTO num_exp_div VALUES (1,0,double('NaN')) -- !query 53 schema struct<> -- !query 53 output @@ -435,7 +435,7 @@ struct<> -- !query 54 -INSERT INTO num_exp_add VALUES (1,1,'0') +INSERT INTO num_exp_add VALUES (1,1,0) -- !query 54 schema struct<> -- !query 54 output @@ -443,7 +443,7 @@ struct<> -- !query 55 -INSERT INTO num_exp_sub VALUES (1,1,'0') +INSERT INTO num_exp_sub VALUES (1,1,0) -- !query 55 schema struct<> -- !query 55 output @@ -451,7 +451,7 @@ struct<> -- !query 56 -INSERT INTO num_exp_mul VALUES (1,1,'0') +INSERT INTO num_exp_mul VALUES (1,1,0) -- !query 56 schema struct<> -- !query 56 output @@ -459,7 +459,7 @@ struct<> -- !query 57 -INSERT INTO num_exp_div VALUES (1,1,'NaN') +INSERT INTO num_exp_div VALUES (1,1,double('NaN')) -- !query 57 schema struct<> -- !query 57 output @@ -467,7 +467,7 @@ struct<> -- !query 58 -INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) -- !query 58 schema struct<> -- !query 58 output @@ -475,7 +475,7 @@ struct<> -- !query 59 -INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047') +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) -- !query 59 schema struct<> -- !query 59 output @@ -483,7 +483,7 @@ struct<> -- !query 60 -INSERT INTO num_exp_mul VALUES (1,2,'0') +INSERT INTO num_exp_mul VALUES (1,2,0) -- !query 60 schema struct<> -- !query 60 output @@ -491,7 +491,7 @@ struct<> -- !query 61 -INSERT INTO num_exp_div VALUES (1,2,'0') +INSERT INTO num_exp_div VALUES (1,2,0) -- !query 61 schema struct<> -- !query 61 output @@ -499,7 +499,7 @@ struct<> -- !query 62 -INSERT INTO num_exp_add VALUES (1,3,'4.31') +INSERT INTO num_exp_add VALUES (1,3,4.31) -- !query 62 schema struct<> -- !query 62 output @@ -507,7 +507,7 @@ struct<> -- !query 63 -INSERT INTO num_exp_sub VALUES (1,3,'-4.31') +INSERT INTO num_exp_sub VALUES (1,3,-4.31) -- !query 63 schema struct<> -- !query 63 output @@ -515,7 +515,7 @@ struct<> -- !query 64 -INSERT INTO num_exp_mul VALUES (1,3,'0') +INSERT INTO num_exp_mul VALUES (1,3,0) -- !query 64 schema struct<> -- !query 64 output @@ -523,7 +523,7 @@ struct<> -- !query 65 -INSERT INTO num_exp_div VALUES (1,3,'0') +INSERT INTO num_exp_div VALUES (1,3,0) -- !query 65 schema struct<> -- !query 65 output @@ -531,7 +531,7 @@ struct<> -- !query 66 -INSERT INTO num_exp_add VALUES (1,4,'7799461.4119') +INSERT INTO num_exp_add VALUES (1,4,7799461.4119) -- !query 66 schema struct<> -- !query 66 output @@ -539,7 +539,7 @@ struct<> -- !query 67 -INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119') +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) -- !query 67 schema struct<> -- !query 67 output @@ -547,7 +547,7 @@ struct<> -- !query 68 -INSERT INTO num_exp_mul VALUES (1,4,'0') +INSERT INTO num_exp_mul VALUES (1,4,0) -- !query 68 schema struct<> -- !query 68 output @@ -555,7 +555,7 @@ struct<> -- !query 69 -INSERT INTO num_exp_div VALUES (1,4,'0') +INSERT INTO num_exp_div VALUES (1,4,0) -- !query 69 schema struct<> -- !query 69 output @@ -563,7 +563,7 @@ struct<> -- !query 70 -INSERT INTO num_exp_add VALUES (1,5,'16397.038491') +INSERT INTO num_exp_add VALUES (1,5,16397.038491) -- !query 70 schema struct<> -- !query 70 output @@ -571,7 +571,7 @@ struct<> -- !query 71 -INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491') +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) -- !query 71 schema struct<> -- !query 71 output @@ -579,7 +579,7 @@ struct<> -- !query 72 -INSERT INTO num_exp_mul VALUES (1,5,'0') +INSERT INTO num_exp_mul VALUES (1,5,0) -- !query 72 schema struct<> -- !query 72 output @@ -587,7 +587,7 @@ struct<> -- !query 73 -INSERT INTO num_exp_div VALUES (1,5,'0') +INSERT INTO num_exp_div VALUES (1,5,0) -- !query 73 schema struct<> -- !query 73 output @@ -595,7 +595,7 @@ struct<> -- !query 74 -INSERT INTO num_exp_add VALUES (1,6,'93901.57763026') +INSERT INTO num_exp_add VALUES (1,6,93901.57763026) -- !query 74 schema struct<> -- !query 74 output @@ -603,7 +603,7 @@ struct<> -- !query 75 -INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026') +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) -- !query 75 schema struct<> -- !query 75 output @@ -611,7 +611,7 @@ struct<> -- !query 76 -INSERT INTO num_exp_mul VALUES (1,6,'0') +INSERT INTO num_exp_mul VALUES (1,6,0) -- !query 76 schema struct<> -- !query 76 output @@ -619,7 +619,7 @@ struct<> -- !query 77 -INSERT INTO num_exp_div VALUES (1,6,'0') +INSERT INTO num_exp_div VALUES (1,6,0) -- !query 77 schema struct<> -- !query 77 output @@ -627,7 +627,7 @@ struct<> -- !query 78 -INSERT INTO num_exp_add VALUES (1,7,'-83028485') +INSERT INTO num_exp_add VALUES (1,7,-83028485) -- !query 78 schema struct<> -- !query 78 output @@ -635,7 +635,7 @@ struct<> -- !query 79 -INSERT INTO num_exp_sub VALUES (1,7,'83028485') +INSERT INTO num_exp_sub VALUES (1,7,83028485) -- !query 79 schema struct<> -- !query 79 output @@ -643,7 +643,7 @@ struct<> -- !query 80 -INSERT INTO num_exp_mul VALUES (1,7,'0') +INSERT INTO num_exp_mul VALUES (1,7,0) -- !query 80 schema struct<> -- !query 80 output @@ -651,7 +651,7 @@ struct<> -- !query 81 -INSERT INTO num_exp_div VALUES (1,7,'0') +INSERT INTO num_exp_div VALUES (1,7,0) -- !query 81 schema struct<> -- !query 81 output @@ -659,7 +659,7 @@ struct<> -- !query 82 -INSERT INTO num_exp_add VALUES (1,8,'74881') +INSERT INTO num_exp_add VALUES (1,8,74881) -- !query 82 schema struct<> -- !query 82 output @@ -667,7 +667,7 @@ struct<> -- !query 83 -INSERT INTO num_exp_sub VALUES (1,8,'-74881') +INSERT INTO num_exp_sub VALUES (1,8,-74881) -- !query 83 schema struct<> -- !query 83 output @@ -675,7 +675,7 @@ struct<> -- !query 84 -INSERT INTO num_exp_mul VALUES (1,8,'0') +INSERT INTO num_exp_mul VALUES (1,8,0) -- !query 84 schema struct<> -- !query 84 output @@ -683,7 +683,7 @@ struct<> -- !query 85 -INSERT INTO num_exp_div VALUES (1,8,'0') +INSERT INTO num_exp_div VALUES (1,8,0) -- !query 85 schema struct<> -- !query 85 output @@ -691,7 +691,7 @@ struct<> -- !query 86 -INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) -- !query 86 schema struct<> -- !query 86 output @@ -699,7 +699,7 @@ struct<> -- !query 87 -INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420') +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) -- !query 87 schema struct<> -- !query 87 output @@ -707,7 +707,7 @@ struct<> -- !query 88 -INSERT INTO num_exp_mul VALUES (1,9,'0') +INSERT INTO num_exp_mul VALUES (1,9,0) -- !query 88 schema struct<> -- !query 88 output @@ -715,7 +715,7 @@ struct<> -- !query 89 -INSERT INTO num_exp_div VALUES (1,9,'0') +INSERT INTO num_exp_div VALUES (1,9,0) -- !query 89 schema struct<> -- !query 89 output @@ -723,7 +723,7 @@ struct<> -- !query 90 -INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) -- !query 90 schema struct<> -- !query 90 output @@ -731,7 +731,7 @@ struct<> -- !query 91 -INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047') +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) -- !query 91 schema struct<> -- !query 91 output @@ -739,7 +739,7 @@ struct<> -- !query 92 -INSERT INTO num_exp_mul VALUES (2,0,'0') +INSERT INTO num_exp_mul VALUES (2,0,0) -- !query 92 schema struct<> -- !query 92 output @@ -747,7 +747,7 @@ struct<> -- !query 93 -INSERT INTO num_exp_div VALUES (2,0,'NaN') +INSERT INTO num_exp_div VALUES (2,0,double('NaN')) -- !query 93 schema struct<> -- !query 93 output @@ -755,7 +755,7 @@ struct<> -- !query 94 -INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) -- !query 94 schema struct<> -- !query 94 output @@ -763,7 +763,7 @@ struct<> -- !query 95 -INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047') +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) -- !query 95 schema struct<> -- !query 95 output @@ -771,7 +771,7 @@ struct<> -- !query 96 -INSERT INTO num_exp_mul VALUES (2,1,'0') +INSERT INTO num_exp_mul VALUES (2,1,0) -- !query 96 schema struct<> -- !query 96 output @@ -779,7 +779,7 @@ struct<> -- !query 97 -INSERT INTO num_exp_div VALUES (2,1,'NaN') +INSERT INTO num_exp_div VALUES (2,1,double('NaN')) -- !query 97 schema struct<> -- !query 97 output @@ -787,7 +787,7 @@ struct<> -- !query 98 -INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094') +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) -- !query 98 schema struct<> -- !query 98 output @@ -795,7 +795,7 @@ struct<> -- !query 99 -INSERT INTO num_exp_sub VALUES (2,2,'0') +INSERT INTO num_exp_sub VALUES (2,2,0) -- !query 99 schema struct<> -- !query 99 output @@ -803,7 +803,7 @@ struct<> -- !query 100 -INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209') +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) -- !query 100 schema struct<> -- !query 100 output @@ -811,7 +811,7 @@ struct<> -- !query 101 -INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) -- !query 101 schema struct<> -- !query 101 output @@ -819,7 +819,7 @@ struct<> -- !query 102 -INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047') +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) -- !query 102 schema struct<> -- !query 102 output @@ -827,7 +827,7 @@ struct<> -- !query 103 -INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047') +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) -- !query 103 schema struct<> -- !query 103 output @@ -835,7 +835,7 @@ struct<> -- !query 104 -INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257') +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) -- !query 104 schema struct<> -- !query 104 output @@ -843,7 +843,7 @@ struct<> -- !query 105 -INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266') +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) -- !query 105 schema struct<> -- !query 105 output @@ -851,7 +851,7 @@ struct<> -- !query 106 -INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047') +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) -- !query 106 schema struct<> -- !query 106 output @@ -859,7 +859,7 @@ struct<> -- !query 107 -INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047') +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) -- !query 107 schema struct<> -- !query 107 output @@ -867,7 +867,7 @@ struct<> -- !query 108 -INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593') +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) -- !query 108 schema struct<> -- !query 108 output @@ -875,7 +875,7 @@ struct<> -- !query 109 -INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685') +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) -- !query 109 schema struct<> -- !query 109 output @@ -883,7 +883,7 @@ struct<> -- !query 110 -INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047') +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) -- !query 110 schema struct<> -- !query 110 output @@ -891,7 +891,7 @@ struct<> -- !query 111 -INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047') +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) -- !query 111 schema struct<> -- !query 111 output @@ -899,7 +899,7 @@ struct<> -- !query 112 -INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077') +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) -- !query 112 schema struct<> -- !query 112 output @@ -907,7 +907,7 @@ struct<> -- !query 113 -INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429') +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) -- !query 113 schema struct<> -- !query 113 output @@ -915,7 +915,7 @@ struct<> -- !query 114 -INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787') +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) -- !query 114 schema struct<> -- !query 114 output @@ -923,7 +923,7 @@ struct<> -- !query 115 -INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307') +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) -- !query 115 schema struct<> -- !query 115 output @@ -931,7 +931,7 @@ struct<> -- !query 116 -INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222') +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) -- !query 116 schema struct<> -- !query 116 output @@ -939,7 +939,7 @@ struct<> -- !query 117 -INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940') +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) -- !query 117 schema struct<> -- !query 117 output @@ -947,7 +947,7 @@ struct<> -- !query 118 -INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047') +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) -- !query 118 schema struct<> -- !query 118 output @@ -955,7 +955,7 @@ struct<> -- !query 119 -INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953') +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) -- !query 119 schema struct<> -- !query 119 output @@ -963,7 +963,7 @@ struct<> -- !query 120 -INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795') +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) -- !query 120 schema struct<> -- !query 120 output @@ -971,7 +971,7 @@ struct<> -- !query 121 -INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518') +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) -- !query 121 schema struct<> -- !query 121 output @@ -979,7 +979,7 @@ struct<> -- !query 122 -INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047') +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) -- !query 122 schema struct<> -- !query 122 output @@ -987,7 +987,7 @@ struct<> -- !query 123 -INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047') +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) -- !query 123 schema struct<> -- !query 123 output @@ -995,7 +995,7 @@ struct<> -- !query 124 -INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407') +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) -- !query 124 schema struct<> -- !query 124 output @@ -1003,7 +1003,7 @@ struct<> -- !query 125 -INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476') +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) -- !query 125 schema struct<> -- !query 125 output @@ -1011,7 +1011,7 @@ struct<> -- !query 126 -INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467') +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) -- !query 126 schema struct<> -- !query 126 output @@ -1019,7 +1019,7 @@ struct<> -- !query 127 -INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627') +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) -- !query 127 schema struct<> -- !query 127 output @@ -1027,7 +1027,7 @@ struct<> -- !query 128 -INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740') +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) -- !query 128 schema struct<> -- !query 128 output @@ -1035,7 +1035,7 @@ struct<> -- !query 129 -INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811') +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) -- !query 129 schema struct<> -- !query 129 output @@ -1043,7 +1043,7 @@ struct<> -- !query 130 -INSERT INTO num_exp_add VALUES (3,0,'4.31') +INSERT INTO num_exp_add VALUES (3,0,4.31) -- !query 130 schema struct<> -- !query 130 output @@ -1051,7 +1051,7 @@ struct<> -- !query 131 -INSERT INTO num_exp_sub VALUES (3,0,'4.31') +INSERT INTO num_exp_sub VALUES (3,0,4.31) -- !query 131 schema struct<> -- !query 131 output @@ -1059,7 +1059,7 @@ struct<> -- !query 132 -INSERT INTO num_exp_mul VALUES (3,0,'0') +INSERT INTO num_exp_mul VALUES (3,0,0) -- !query 132 schema struct<> -- !query 132 output @@ -1067,7 +1067,7 @@ struct<> -- !query 133 -INSERT INTO num_exp_div VALUES (3,0,'NaN') +INSERT INTO num_exp_div VALUES (3,0,double('NaN')) -- !query 133 schema struct<> -- !query 133 output @@ -1075,7 +1075,7 @@ struct<> -- !query 134 -INSERT INTO num_exp_add VALUES (3,1,'4.31') +INSERT INTO num_exp_add VALUES (3,1,4.31) -- !query 134 schema struct<> -- !query 134 output @@ -1083,7 +1083,7 @@ struct<> -- !query 135 -INSERT INTO num_exp_sub VALUES (3,1,'4.31') +INSERT INTO num_exp_sub VALUES (3,1,4.31) -- !query 135 schema struct<> -- !query 135 output @@ -1091,7 +1091,7 @@ struct<> -- !query 136 -INSERT INTO num_exp_mul VALUES (3,1,'0') +INSERT INTO num_exp_mul VALUES (3,1,0) -- !query 136 schema struct<> -- !query 136 output @@ -1099,7 +1099,7 @@ struct<> -- !query 137 -INSERT INTO num_exp_div VALUES (3,1,'NaN') +INSERT INTO num_exp_div VALUES (3,1,double('NaN')) -- !query 137 schema struct<> -- !query 137 output @@ -1107,7 +1107,7 @@ struct<> -- !query 138 -INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047') +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) -- !query 138 schema struct<> -- !query 138 output @@ -1115,7 +1115,7 @@ struct<> -- !query 139 -INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047') +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) -- !query 139 schema struct<> -- !query 139 output @@ -1123,7 +1123,7 @@ struct<> -- !query 140 -INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257') +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) -- !query 140 schema struct<> -- !query 140 output @@ -1131,7 +1131,7 @@ struct<> -- !query 141 -INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352') +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) -- !query 141 schema struct<> -- !query 141 output @@ -1139,7 +1139,7 @@ struct<> -- !query 142 -INSERT INTO num_exp_add VALUES (3,3,'8.62') +INSERT INTO num_exp_add VALUES (3,3,8.62) -- !query 142 schema struct<> -- !query 142 output @@ -1147,7 +1147,7 @@ struct<> -- !query 143 -INSERT INTO num_exp_sub VALUES (3,3,'0') +INSERT INTO num_exp_sub VALUES (3,3,0) -- !query 143 schema struct<> -- !query 143 output @@ -1155,7 +1155,7 @@ struct<> -- !query 144 -INSERT INTO num_exp_mul VALUES (3,3,'18.5761') +INSERT INTO num_exp_mul VALUES (3,3,18.5761) -- !query 144 schema struct<> -- !query 144 output @@ -1163,7 +1163,7 @@ struct<> -- !query 145 -INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) -- !query 145 schema struct<> -- !query 145 output @@ -1171,7 +1171,7 @@ struct<> -- !query 146 -INSERT INTO num_exp_add VALUES (3,4,'7799465.7219') +INSERT INTO num_exp_add VALUES (3,4,7799465.7219) -- !query 146 schema struct<> -- !query 146 output @@ -1179,7 +1179,7 @@ struct<> -- !query 147 -INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019') +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) -- !query 147 schema struct<> -- !query 147 output @@ -1187,7 +1187,7 @@ struct<> -- !query 148 -INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289') +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) -- !query 148 schema struct<> -- !query 148 output @@ -1195,7 +1195,7 @@ struct<> -- !query 149 -INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552') +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) -- !query 149 schema struct<> -- !query 149 output @@ -1203,7 +1203,7 @@ struct<> -- !query 150 -INSERT INTO num_exp_add VALUES (3,5,'16401.348491') +INSERT INTO num_exp_add VALUES (3,5,16401.348491) -- !query 150 schema struct<> -- !query 150 output @@ -1211,7 +1211,7 @@ struct<> -- !query 151 -INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491') +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) -- !query 151 schema struct<> -- !query 151 output @@ -1219,7 +1219,7 @@ struct<> -- !query 152 -INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621') +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) -- !query 152 schema struct<> -- !query 152 output @@ -1227,7 +1227,7 @@ struct<> -- !query 153 -INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504') +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) -- !query 153 schema struct<> -- !query 153 output @@ -1235,7 +1235,7 @@ struct<> -- !query 154 -INSERT INTO num_exp_add VALUES (3,6,'93905.88763026') +INSERT INTO num_exp_add VALUES (3,6,93905.88763026) -- !query 154 schema struct<> -- !query 154 output @@ -1243,7 +1243,7 @@ struct<> -- !query 155 -INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026') +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) -- !query 155 schema struct<> -- !query 155 output @@ -1251,7 +1251,7 @@ struct<> -- !query 156 -INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206') +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) -- !query 156 schema struct<> -- !query 156 output @@ -1259,7 +1259,7 @@ struct<> -- !query 157 -INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595') +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) -- !query 157 schema struct<> -- !query 157 output @@ -1267,7 +1267,7 @@ struct<> -- !query 158 -INSERT INTO num_exp_add VALUES (3,7,'-83028480.69') +INSERT INTO num_exp_add VALUES (3,7,-83028480.69) -- !query 158 schema struct<> -- !query 158 output @@ -1275,7 +1275,7 @@ struct<> -- !query 159 -INSERT INTO num_exp_sub VALUES (3,7,'83028489.31') +INSERT INTO num_exp_sub VALUES (3,7,83028489.31) -- !query 159 schema struct<> -- !query 159 output @@ -1283,7 +1283,7 @@ struct<> -- !query 160 -INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35') +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) -- !query 160 schema struct<> -- !query 160 output @@ -1291,7 +1291,7 @@ struct<> -- !query 161 -INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240') +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) -- !query 161 schema struct<> -- !query 161 output @@ -1299,7 +1299,7 @@ struct<> -- !query 162 -INSERT INTO num_exp_add VALUES (3,8,'74885.31') +INSERT INTO num_exp_add VALUES (3,8,74885.31) -- !query 162 schema struct<> -- !query 162 output @@ -1307,7 +1307,7 @@ struct<> -- !query 163 -INSERT INTO num_exp_sub VALUES (3,8,'-74876.69') +INSERT INTO num_exp_sub VALUES (3,8,-74876.69) -- !query 163 schema struct<> -- !query 163 output @@ -1315,7 +1315,7 @@ struct<> -- !query 164 -INSERT INTO num_exp_mul VALUES (3,8,'322737.11') +INSERT INTO num_exp_mul VALUES (3,8,322737.11) -- !query 164 schema struct<> -- !query 164 output @@ -1323,7 +1323,7 @@ struct<> -- !query 165 -INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553') +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) -- !query 165 schema struct<> -- !query 165 output @@ -1331,7 +1331,7 @@ struct<> -- !query 166 -INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420') +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) -- !query 166 schema struct<> -- !query 166 output @@ -1339,7 +1339,7 @@ struct<> -- !query 167 -INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420') +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) -- !query 167 schema struct<> -- !query 167 output @@ -1347,7 +1347,7 @@ struct<> -- !query 168 -INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020') +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) -- !query 168 schema struct<> -- !query 168 output @@ -1355,7 +1355,7 @@ struct<> -- !query 169 -INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854') +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) -- !query 169 schema struct<> -- !query 169 output @@ -1363,7 +1363,7 @@ struct<> -- !query 170 -INSERT INTO num_exp_add VALUES (4,0,'7799461.4119') +INSERT INTO num_exp_add VALUES (4,0,7799461.4119) -- !query 170 schema struct<> -- !query 170 output @@ -1371,7 +1371,7 @@ struct<> -- !query 171 -INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119') +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) -- !query 171 schema struct<> -- !query 171 output @@ -1379,7 +1379,7 @@ struct<> -- !query 172 -INSERT INTO num_exp_mul VALUES (4,0,'0') +INSERT INTO num_exp_mul VALUES (4,0,0) -- !query 172 schema struct<> -- !query 172 output @@ -1387,7 +1387,7 @@ struct<> -- !query 173 -INSERT INTO num_exp_div VALUES (4,0,'NaN') +INSERT INTO num_exp_div VALUES (4,0,double('NaN')) -- !query 173 schema struct<> -- !query 173 output @@ -1395,7 +1395,7 @@ struct<> -- !query 174 -INSERT INTO num_exp_add VALUES (4,1,'7799461.4119') +INSERT INTO num_exp_add VALUES (4,1,7799461.4119) -- !query 174 schema struct<> -- !query 174 output @@ -1403,7 +1403,7 @@ struct<> -- !query 175 -INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119') +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) -- !query 175 schema struct<> -- !query 175 output @@ -1411,7 +1411,7 @@ struct<> -- !query 176 -INSERT INTO num_exp_mul VALUES (4,1,'0') +INSERT INTO num_exp_mul VALUES (4,1,0) -- !query 176 schema struct<> -- !query 176 output @@ -1419,7 +1419,7 @@ struct<> -- !query 177 -INSERT INTO num_exp_div VALUES (4,1,'NaN') +INSERT INTO num_exp_div VALUES (4,1,double('NaN')) -- !query 177 schema struct<> -- !query 177 output @@ -1427,7 +1427,7 @@ struct<> -- !query 178 -INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047') +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) -- !query 178 schema struct<> -- !query 178 output @@ -1435,7 +1435,7 @@ struct<> -- !query 179 -INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047') +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) -- !query 179 schema struct<> -- !query 179 output @@ -1443,7 +1443,7 @@ struct<> -- !query 180 -INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593') +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) -- !query 180 schema struct<> -- !query 180 output @@ -1451,7 +1451,7 @@ struct<> -- !query 181 -INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385') +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) -- !query 181 schema struct<> -- !query 181 output @@ -1459,7 +1459,7 @@ struct<> -- !query 182 -INSERT INTO num_exp_add VALUES (4,3,'7799465.7219') +INSERT INTO num_exp_add VALUES (4,3,7799465.7219) -- !query 182 schema struct<> -- !query 182 output @@ -1467,7 +1467,7 @@ struct<> -- !query 183 -INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019') +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) -- !query 183 schema struct<> -- !query 183 output @@ -1475,7 +1475,7 @@ struct<> -- !query 184 -INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289') +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) -- !query 184 schema struct<> -- !query 184 output @@ -1483,7 +1483,7 @@ struct<> -- !query 185 -INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883') +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) -- !query 185 schema struct<> -- !query 185 output @@ -1491,7 +1491,7 @@ struct<> -- !query 186 -INSERT INTO num_exp_add VALUES (4,4,'15598922.8238') +INSERT INTO num_exp_add VALUES (4,4,15598922.8238) -- !query 186 schema struct<> -- !query 186 output @@ -1499,7 +1499,7 @@ struct<> -- !query 187 -INSERT INTO num_exp_sub VALUES (4,4,'0') +INSERT INTO num_exp_sub VALUES (4,4,0) -- !query 187 schema struct<> -- !query 187 output @@ -1507,7 +1507,7 @@ struct<> -- !query 188 -INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161') +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) -- !query 188 schema struct<> -- !query 188 output @@ -1515,7 +1515,7 @@ struct<> -- !query 189 -INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) -- !query 189 schema struct<> -- !query 189 output @@ -1523,7 +1523,7 @@ struct<> -- !query 190 -INSERT INTO num_exp_add VALUES (4,5,'7815858.450391') +INSERT INTO num_exp_add VALUES (4,5,7815858.450391) -- !query 190 schema struct<> -- !query 190 output @@ -1531,7 +1531,7 @@ struct<> -- !query 191 -INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409') +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) -- !query 191 schema struct<> -- !query 191 output @@ -1539,7 +1539,7 @@ struct<> -- !query 192 -INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429') +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) -- !query 192 schema struct<> -- !query 192 output @@ -1547,7 +1547,7 @@ struct<> -- !query 193 -INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061') +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) -- !query 193 schema struct<> -- !query 193 output @@ -1555,7 +1555,7 @@ struct<> -- !query 194 -INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026') +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) -- !query 194 schema struct<> -- !query 194 output @@ -1563,7 +1563,7 @@ struct<> -- !query 195 -INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974') +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) -- !query 195 schema struct<> -- !query 195 output @@ -1571,7 +1571,7 @@ struct<> -- !query 196 -INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094') +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) -- !query 196 schema struct<> -- !query 196 output @@ -1579,7 +1579,7 @@ struct<> -- !query 197 -INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606') +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) -- !query 197 schema struct<> -- !query 197 output @@ -1587,7 +1587,7 @@ struct<> -- !query 198 -INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881') +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) -- !query 198 schema struct<> -- !query 198 output @@ -1595,7 +1595,7 @@ struct<> -- !query 199 -INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119') +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) -- !query 199 schema struct<> -- !query 199 output @@ -1603,7 +1603,7 @@ struct<> -- !query 200 -INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715') +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) -- !query 200 schema struct<> -- !query 200 output @@ -1611,7 +1611,7 @@ struct<> -- !query 201 -INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637') +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) -- !query 201 schema struct<> -- !query 201 output @@ -1619,7 +1619,7 @@ struct<> -- !query 202 -INSERT INTO num_exp_add VALUES (4,8,'7874342.4119') +INSERT INTO num_exp_add VALUES (4,8,7874342.4119) -- !query 202 schema struct<> -- !query 202 output @@ -1627,7 +1627,7 @@ struct<> -- !query 203 -INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119') +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) -- !query 203 schema struct<> -- !query 203 output @@ -1635,7 +1635,7 @@ struct<> -- !query 204 -INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839') +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) -- !query 204 schema struct<> -- !query 204 output @@ -1643,7 +1643,7 @@ struct<> -- !query 205 -INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143') +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) -- !query 205 schema struct<> -- !query 205 output @@ -1651,7 +1651,7 @@ struct<> -- !query 206 -INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420') +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) -- !query 206 schema struct<> -- !query 206 output @@ -1659,7 +1659,7 @@ struct<> -- !query 207 -INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420') +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) -- !query 207 schema struct<> -- !query 207 output @@ -1667,7 +1667,7 @@ struct<> -- !query 208 -INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980') +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) -- !query 208 schema struct<> -- !query 208 output @@ -1675,7 +1675,7 @@ struct<> -- !query 209 -INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409') +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) -- !query 209 schema struct<> -- !query 209 output @@ -1683,7 +1683,7 @@ struct<> -- !query 210 -INSERT INTO num_exp_add VALUES (5,0,'16397.038491') +INSERT INTO num_exp_add VALUES (5,0,16397.038491) -- !query 210 schema struct<> -- !query 210 output @@ -1691,7 +1691,7 @@ struct<> -- !query 211 -INSERT INTO num_exp_sub VALUES (5,0,'16397.038491') +INSERT INTO num_exp_sub VALUES (5,0,16397.038491) -- !query 211 schema struct<> -- !query 211 output @@ -1699,7 +1699,7 @@ struct<> -- !query 212 -INSERT INTO num_exp_mul VALUES (5,0,'0') +INSERT INTO num_exp_mul VALUES (5,0,0) -- !query 212 schema struct<> -- !query 212 output @@ -1707,7 +1707,7 @@ struct<> -- !query 213 -INSERT INTO num_exp_div VALUES (5,0,'NaN') +INSERT INTO num_exp_div VALUES (5,0,double('NaN')) -- !query 213 schema struct<> -- !query 213 output @@ -1715,7 +1715,7 @@ struct<> -- !query 214 -INSERT INTO num_exp_add VALUES (5,1,'16397.038491') +INSERT INTO num_exp_add VALUES (5,1,16397.038491) -- !query 214 schema struct<> -- !query 214 output @@ -1723,7 +1723,7 @@ struct<> -- !query 215 -INSERT INTO num_exp_sub VALUES (5,1,'16397.038491') +INSERT INTO num_exp_sub VALUES (5,1,16397.038491) -- !query 215 schema struct<> -- !query 215 output @@ -1731,7 +1731,7 @@ struct<> -- !query 216 -INSERT INTO num_exp_mul VALUES (5,1,'0') +INSERT INTO num_exp_mul VALUES (5,1,0) -- !query 216 schema struct<> -- !query 216 output @@ -1739,7 +1739,7 @@ struct<> -- !query 217 -INSERT INTO num_exp_div VALUES (5,1,'NaN') +INSERT INTO num_exp_div VALUES (5,1,double('NaN')) -- !query 217 schema struct<> -- !query 217 output @@ -1747,7 +1747,7 @@ struct<> -- !query 218 -INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047') +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) -- !query 218 schema struct<> -- !query 218 output @@ -1755,7 +1755,7 @@ struct<> -- !query 219 -INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047') +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) -- !query 219 schema struct<> -- !query 219 output @@ -1763,7 +1763,7 @@ struct<> -- !query 220 -INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077') +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) -- !query 220 schema struct<> -- !query 220 output @@ -1771,7 +1771,7 @@ struct<> -- !query 221 -INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446') +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) -- !query 221 schema struct<> -- !query 221 output @@ -1779,7 +1779,7 @@ struct<> -- !query 222 -INSERT INTO num_exp_add VALUES (5,3,'16401.348491') +INSERT INTO num_exp_add VALUES (5,3,16401.348491) -- !query 222 schema struct<> -- !query 222 output @@ -1787,7 +1787,7 @@ struct<> -- !query 223 -INSERT INTO num_exp_sub VALUES (5,3,'16392.728491') +INSERT INTO num_exp_sub VALUES (5,3,16392.728491) -- !query 223 schema struct<> -- !query 223 output @@ -1795,7 +1795,7 @@ struct<> -- !query 224 -INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621') +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) -- !query 224 schema struct<> -- !query 224 output @@ -1803,7 +1803,7 @@ struct<> -- !query 225 -INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584') +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) -- !query 225 schema struct<> -- !query 225 output @@ -1811,7 +1811,7 @@ struct<> -- !query 226 -INSERT INTO num_exp_add VALUES (5,4,'7815858.450391') +INSERT INTO num_exp_add VALUES (5,4,7815858.450391) -- !query 226 schema struct<> -- !query 226 output @@ -1819,7 +1819,7 @@ struct<> -- !query 227 -INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409') +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) -- !query 227 schema struct<> -- !query 227 output @@ -1827,7 +1827,7 @@ struct<> -- !query 228 -INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429') +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) -- !query 228 schema struct<> -- !query 228 output @@ -1835,7 +1835,7 @@ struct<> -- !query 229 -INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192') +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) -- !query 229 schema struct<> -- !query 229 output @@ -1843,7 +1843,7 @@ struct<> -- !query 230 -INSERT INTO num_exp_add VALUES (5,5,'32794.076982') +INSERT INTO num_exp_add VALUES (5,5,32794.076982) -- !query 230 schema struct<> -- !query 230 output @@ -1851,7 +1851,7 @@ struct<> -- !query 231 -INSERT INTO num_exp_sub VALUES (5,5,'0') +INSERT INTO num_exp_sub VALUES (5,5,0) -- !query 231 schema struct<> -- !query 231 output @@ -1859,7 +1859,7 @@ struct<> -- !query 232 -INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081') +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) -- !query 232 schema struct<> -- !query 232 output @@ -1867,7 +1867,7 @@ struct<> -- !query 233 -INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) -- !query 233 schema struct<> -- !query 233 output @@ -1875,7 +1875,7 @@ struct<> -- !query 234 -INSERT INTO num_exp_add VALUES (5,6,'110298.61612126') +INSERT INTO num_exp_add VALUES (5,6,110298.61612126) -- !query 234 schema struct<> -- !query 234 output @@ -1883,7 +1883,7 @@ struct<> -- !query 235 -INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926') +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) -- !query 235 schema struct<> -- !query 235 output @@ -1891,7 +1891,7 @@ struct<> -- !query 236 -INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766') +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) -- !query 236 schema struct<> -- !query 236 output @@ -1899,7 +1899,7 @@ struct<> -- !query 237 -INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689') +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) -- !query 237 schema struct<> -- !query 237 output @@ -1907,7 +1907,7 @@ struct<> -- !query 238 -INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509') +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) -- !query 238 schema struct<> -- !query 238 output @@ -1915,7 +1915,7 @@ struct<> -- !query 239 -INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491') +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) -- !query 239 schema struct<> -- !query 239 output @@ -1923,7 +1923,7 @@ struct<> -- !query 240 -INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135') +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) -- !query 240 schema struct<> -- !query 240 output @@ -1931,7 +1931,7 @@ struct<> -- !query 241 -INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710') +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) -- !query 241 schema struct<> -- !query 241 output @@ -1939,7 +1939,7 @@ struct<> -- !query 242 -INSERT INTO num_exp_add VALUES (5,8,'91278.038491') +INSERT INTO num_exp_add VALUES (5,8,91278.038491) -- !query 242 schema struct<> -- !query 242 output @@ -1947,7 +1947,7 @@ struct<> -- !query 243 -INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509') +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) -- !query 243 schema struct<> -- !query 243 output @@ -1955,7 +1955,7 @@ struct<> -- !query 244 -INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571') +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) -- !query 244 schema struct<> -- !query 244 output @@ -1963,7 +1963,7 @@ struct<> -- !query 245 -INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228') +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) -- !query 245 schema struct<> -- !query 245 output @@ -1971,7 +1971,7 @@ struct<> -- !query 246 -INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420') +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) -- !query 246 schema struct<> -- !query 246 output @@ -1979,7 +1979,7 @@ struct<> -- !query 247 -INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420') +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) -- !query 247 schema struct<> -- !query 247 output @@ -1987,7 +1987,7 @@ struct<> -- !query 248 -INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220') +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) -- !query 248 schema struct<> -- !query 248 output @@ -1995,7 +1995,7 @@ struct<> -- !query 249 -INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427') +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) -- !query 249 schema struct<> -- !query 249 output @@ -2003,7 +2003,7 @@ struct<> -- !query 250 -INSERT INTO num_exp_add VALUES (6,0,'93901.57763026') +INSERT INTO num_exp_add VALUES (6,0,93901.57763026) -- !query 250 schema struct<> -- !query 250 output @@ -2011,7 +2011,7 @@ struct<> -- !query 251 -INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026') +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) -- !query 251 schema struct<> -- !query 251 output @@ -2019,7 +2019,7 @@ struct<> -- !query 252 -INSERT INTO num_exp_mul VALUES (6,0,'0') +INSERT INTO num_exp_mul VALUES (6,0,0) -- !query 252 schema struct<> -- !query 252 output @@ -2027,7 +2027,7 @@ struct<> -- !query 253 -INSERT INTO num_exp_div VALUES (6,0,'NaN') +INSERT INTO num_exp_div VALUES (6,0,double('NaN')) -- !query 253 schema struct<> -- !query 253 output @@ -2035,7 +2035,7 @@ struct<> -- !query 254 -INSERT INTO num_exp_add VALUES (6,1,'93901.57763026') +INSERT INTO num_exp_add VALUES (6,1,93901.57763026) -- !query 254 schema struct<> -- !query 254 output @@ -2043,7 +2043,7 @@ struct<> -- !query 255 -INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026') +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) -- !query 255 schema struct<> -- !query 255 output @@ -2051,7 +2051,7 @@ struct<> -- !query 256 -INSERT INTO num_exp_mul VALUES (6,1,'0') +INSERT INTO num_exp_mul VALUES (6,1,0) -- !query 256 schema struct<> -- !query 256 output @@ -2059,7 +2059,7 @@ struct<> -- !query 257 -INSERT INTO num_exp_div VALUES (6,1,'NaN') +INSERT INTO num_exp_div VALUES (6,1,double('NaN')) -- !query 257 schema struct<> -- !query 257 output @@ -2067,7 +2067,7 @@ struct<> -- !query 258 -INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787') +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) -- !query 258 schema struct<> -- !query 258 output @@ -2075,7 +2075,7 @@ struct<> -- !query 259 -INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307') +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) -- !query 259 schema struct<> -- !query 259 output @@ -2083,7 +2083,7 @@ struct<> -- !query 260 -INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222') +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) -- !query 260 schema struct<> -- !query 260 output @@ -2091,7 +2091,7 @@ struct<> -- !query 261 -INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823') +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) -- !query 261 schema struct<> -- !query 261 output @@ -2099,7 +2099,7 @@ struct<> -- !query 262 -INSERT INTO num_exp_add VALUES (6,3,'93905.88763026') +INSERT INTO num_exp_add VALUES (6,3,93905.88763026) -- !query 262 schema struct<> -- !query 262 output @@ -2107,7 +2107,7 @@ struct<> -- !query 263 -INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026') +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) -- !query 263 schema struct<> -- !query 263 output @@ -2115,7 +2115,7 @@ struct<> -- !query 264 -INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206') +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) -- !query 264 schema struct<> -- !query 264 output @@ -2123,7 +2123,7 @@ struct<> -- !query 265 -INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907') +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) -- !query 265 schema struct<> -- !query 265 output @@ -2131,7 +2131,7 @@ struct<> -- !query 266 -INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026') +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) -- !query 266 schema struct<> -- !query 266 output @@ -2139,7 +2139,7 @@ struct<> -- !query 267 -INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974') +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) -- !query 267 schema struct<> -- !query 267 output @@ -2147,7 +2147,7 @@ struct<> -- !query 268 -INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094') +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) -- !query 268 schema struct<> -- !query 268 output @@ -2155,7 +2155,7 @@ struct<> -- !query 269 -INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469') +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) -- !query 269 schema struct<> -- !query 269 output @@ -2163,7 +2163,7 @@ struct<> -- !query 270 -INSERT INTO num_exp_add VALUES (6,5,'110298.61612126') +INSERT INTO num_exp_add VALUES (6,5,110298.61612126) -- !query 270 schema struct<> -- !query 270 output @@ -2171,7 +2171,7 @@ struct<> -- !query 271 -INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926') +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) -- !query 271 schema struct<> -- !query 271 output @@ -2179,7 +2179,7 @@ struct<> -- !query 272 -INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766') +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) -- !query 272 schema struct<> -- !query 272 output @@ -2187,7 +2187,7 @@ struct<> -- !query 273 -INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679') +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) -- !query 273 schema struct<> -- !query 273 output @@ -2195,7 +2195,7 @@ struct<> -- !query 274 -INSERT INTO num_exp_add VALUES (6,6,'187803.15526052') +INSERT INTO num_exp_add VALUES (6,6,187803.15526052) -- !query 274 schema struct<> -- !query 274 output @@ -2203,7 +2203,7 @@ struct<> -- !query 275 -INSERT INTO num_exp_sub VALUES (6,6,'0') +INSERT INTO num_exp_sub VALUES (6,6,0) -- !query 275 schema struct<> -- !query 275 output @@ -2211,7 +2211,7 @@ struct<> -- !query 276 -INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676') +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) -- !query 276 schema struct<> -- !query 276 output @@ -2219,7 +2219,7 @@ struct<> -- !query 277 -INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) -- !query 277 schema struct<> -- !query 277 output @@ -2227,7 +2227,7 @@ struct<> -- !query 278 -INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974') +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) -- !query 278 schema struct<> -- !query 278 output @@ -2235,7 +2235,7 @@ struct<> -- !query 279 -INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026') +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) -- !query 279 schema struct<> -- !query 279 output @@ -2243,7 +2243,7 @@ struct<> -- !query 280 -INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610') +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) -- !query 280 schema struct<> -- !query 280 output @@ -2251,7 +2251,7 @@ struct<> -- !query 281 -INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980') +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) -- !query 281 schema struct<> -- !query 281 output @@ -2259,7 +2259,7 @@ struct<> -- !query 282 -INSERT INTO num_exp_add VALUES (6,8,'168782.57763026') +INSERT INTO num_exp_add VALUES (6,8,168782.57763026) -- !query 282 schema struct<> -- !query 282 output @@ -2267,7 +2267,7 @@ struct<> -- !query 283 -INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026') +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) -- !query 283 schema struct<> -- !query 283 output @@ -2275,7 +2275,7 @@ struct<> -- !query 284 -INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906') +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) -- !query 284 schema struct<> -- !query 284 output @@ -2283,7 +2283,7 @@ struct<> -- !query 285 -INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184') +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) -- !query 285 schema struct<> -- !query 285 output @@ -2291,7 +2291,7 @@ struct<> -- !query 286 -INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160') +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) -- !query 286 schema struct<> -- !query 286 output @@ -2299,7 +2299,7 @@ struct<> -- !query 287 -INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680') +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) -- !query 287 schema struct<> -- !query 287 output @@ -2307,7 +2307,7 @@ struct<> -- !query 288 -INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920') +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) -- !query 288 schema struct<> -- !query 288 output @@ -2315,7 +2315,7 @@ struct<> -- !query 289 -INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789') +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) -- !query 289 schema struct<> -- !query 289 output @@ -2323,7 +2323,7 @@ struct<> -- !query 290 -INSERT INTO num_exp_add VALUES (7,0,'-83028485') +INSERT INTO num_exp_add VALUES (7,0,-83028485) -- !query 290 schema struct<> -- !query 290 output @@ -2331,7 +2331,7 @@ struct<> -- !query 291 -INSERT INTO num_exp_sub VALUES (7,0,'-83028485') +INSERT INTO num_exp_sub VALUES (7,0,-83028485) -- !query 291 schema struct<> -- !query 291 output @@ -2339,7 +2339,7 @@ struct<> -- !query 292 -INSERT INTO num_exp_mul VALUES (7,0,'0') +INSERT INTO num_exp_mul VALUES (7,0,0) -- !query 292 schema struct<> -- !query 292 output @@ -2347,7 +2347,7 @@ struct<> -- !query 293 -INSERT INTO num_exp_div VALUES (7,0,'NaN') +INSERT INTO num_exp_div VALUES (7,0,double('NaN')) -- !query 293 schema struct<> -- !query 293 output @@ -2355,7 +2355,7 @@ struct<> -- !query 294 -INSERT INTO num_exp_add VALUES (7,1,'-83028485') +INSERT INTO num_exp_add VALUES (7,1,-83028485) -- !query 294 schema struct<> -- !query 294 output @@ -2363,7 +2363,7 @@ struct<> -- !query 295 -INSERT INTO num_exp_sub VALUES (7,1,'-83028485') +INSERT INTO num_exp_sub VALUES (7,1,-83028485) -- !query 295 schema struct<> -- !query 295 output @@ -2371,7 +2371,7 @@ struct<> -- !query 296 -INSERT INTO num_exp_mul VALUES (7,1,'0') +INSERT INTO num_exp_mul VALUES (7,1,0) -- !query 296 schema struct<> -- !query 296 output @@ -2379,7 +2379,7 @@ struct<> -- !query 297 -INSERT INTO num_exp_div VALUES (7,1,'NaN') +INSERT INTO num_exp_div VALUES (7,1,double('NaN')) -- !query 297 schema struct<> -- !query 297 output @@ -2387,7 +2387,7 @@ struct<> -- !query 298 -INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047') +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) -- !query 298 schema struct<> -- !query 298 output @@ -2395,7 +2395,7 @@ struct<> -- !query 299 -INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953') +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) -- !query 299 schema struct<> -- !query 299 output @@ -2403,7 +2403,7 @@ struct<> -- !query 300 -INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795') +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) -- !query 300 schema struct<> -- !query 300 output @@ -2411,7 +2411,7 @@ struct<> -- !query 301 -INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700') +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) -- !query 301 schema struct<> -- !query 301 output @@ -2419,7 +2419,7 @@ struct<> -- !query 302 -INSERT INTO num_exp_add VALUES (7,3,'-83028480.69') +INSERT INTO num_exp_add VALUES (7,3,-83028480.69) -- !query 302 schema struct<> -- !query 302 output @@ -2427,7 +2427,7 @@ struct<> -- !query 303 -INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31') +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) -- !query 303 schema struct<> -- !query 303 output @@ -2435,7 +2435,7 @@ struct<> -- !query 304 -INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35') +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) -- !query 304 schema struct<> -- !query 304 output @@ -2443,7 +2443,7 @@ struct<> -- !query 305 -INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974') +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) -- !query 305 schema struct<> -- !query 305 output @@ -2451,7 +2451,7 @@ struct<> -- !query 306 -INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881') +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) -- !query 306 schema struct<> -- !query 306 output @@ -2459,7 +2459,7 @@ struct<> -- !query 307 -INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119') +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) -- !query 307 schema struct<> -- !query 307 output @@ -2467,7 +2467,7 @@ struct<> -- !query 308 -INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715') +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) -- !query 308 schema struct<> -- !query 308 output @@ -2475,7 +2475,7 @@ struct<> -- !query 309 -INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686') +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) -- !query 309 schema struct<> -- !query 309 output @@ -2483,7 +2483,7 @@ struct<> -- !query 310 -INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509') +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) -- !query 310 schema struct<> -- !query 310 output @@ -2491,7 +2491,7 @@ struct<> -- !query 311 -INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491') +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) -- !query 311 schema struct<> -- !query 311 output @@ -2499,7 +2499,7 @@ struct<> -- !query 312 -INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135') +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) -- !query 312 schema struct<> -- !query 312 output @@ -2507,7 +2507,7 @@ struct<> -- !query 313 -INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574') +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) -- !query 313 schema struct<> -- !query 313 output @@ -2515,7 +2515,7 @@ struct<> -- !query 314 -INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974') +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) -- !query 314 schema struct<> -- !query 314 output @@ -2523,7 +2523,7 @@ struct<> -- !query 315 -INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026') +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) -- !query 315 schema struct<> -- !query 315 output @@ -2531,7 +2531,7 @@ struct<> -- !query 316 -INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610') +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) -- !query 316 schema struct<> -- !query 316 output @@ -2539,7 +2539,7 @@ struct<> -- !query 317 -INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294') +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) -- !query 317 schema struct<> -- !query 317 output @@ -2547,7 +2547,7 @@ struct<> -- !query 318 -INSERT INTO num_exp_add VALUES (7,7,'-166056970') +INSERT INTO num_exp_add VALUES (7,7,-166056970) -- !query 318 schema struct<> -- !query 318 output @@ -2555,7 +2555,7 @@ struct<> -- !query 319 -INSERT INTO num_exp_sub VALUES (7,7,'0') +INSERT INTO num_exp_sub VALUES (7,7,0) -- !query 319 schema struct<> -- !query 319 output @@ -2563,7 +2563,7 @@ struct<> -- !query 320 -INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225') +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) -- !query 320 schema struct<> -- !query 320 output @@ -2571,7 +2571,7 @@ struct<> -- !query 321 -INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) -- !query 321 schema struct<> -- !query 321 output @@ -2579,7 +2579,7 @@ struct<> -- !query 322 -INSERT INTO num_exp_add VALUES (7,8,'-82953604') +INSERT INTO num_exp_add VALUES (7,8,-82953604) -- !query 322 schema struct<> -- !query 322 output @@ -2587,7 +2587,7 @@ struct<> -- !query 323 -INSERT INTO num_exp_sub VALUES (7,8,'-83103366') +INSERT INTO num_exp_sub VALUES (7,8,-83103366) -- !query 323 schema struct<> -- !query 323 output @@ -2595,7 +2595,7 @@ struct<> -- !query 324 -INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285') +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) -- !query 324 schema struct<> -- !query 324 output @@ -2603,7 +2603,7 @@ struct<> -- !query 325 -INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118') +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) -- !query 325 schema struct<> -- !query 325 output @@ -2611,7 +2611,7 @@ struct<> -- !query 326 -INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420') +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) -- !query 326 schema struct<> -- !query 326 output @@ -2619,7 +2619,7 @@ struct<> -- !query 327 -INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580') +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) -- !query 327 schema struct<> -- !query 327 output @@ -2627,7 +2627,7 @@ struct<> -- !query 328 -INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700') +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) -- !query 328 schema struct<> -- !query 328 output @@ -2635,7 +2635,7 @@ struct<> -- !query 329 -INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382') +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) -- !query 329 schema struct<> -- !query 329 output @@ -2643,7 +2643,7 @@ struct<> -- !query 330 -INSERT INTO num_exp_add VALUES (8,0,'74881') +INSERT INTO num_exp_add VALUES (8,0,74881) -- !query 330 schema struct<> -- !query 330 output @@ -2651,7 +2651,7 @@ struct<> -- !query 331 -INSERT INTO num_exp_sub VALUES (8,0,'74881') +INSERT INTO num_exp_sub VALUES (8,0,74881) -- !query 331 schema struct<> -- !query 331 output @@ -2659,7 +2659,7 @@ struct<> -- !query 332 -INSERT INTO num_exp_mul VALUES (8,0,'0') +INSERT INTO num_exp_mul VALUES (8,0,0) -- !query 332 schema struct<> -- !query 332 output @@ -2667,7 +2667,7 @@ struct<> -- !query 333 -INSERT INTO num_exp_div VALUES (8,0,'NaN') +INSERT INTO num_exp_div VALUES (8,0,double('NaN')) -- !query 333 schema struct<> -- !query 333 output @@ -2675,7 +2675,7 @@ struct<> -- !query 334 -INSERT INTO num_exp_add VALUES (8,1,'74881') +INSERT INTO num_exp_add VALUES (8,1,74881) -- !query 334 schema struct<> -- !query 334 output @@ -2683,7 +2683,7 @@ struct<> -- !query 335 -INSERT INTO num_exp_sub VALUES (8,1,'74881') +INSERT INTO num_exp_sub VALUES (8,1,74881) -- !query 335 schema struct<> -- !query 335 output @@ -2691,7 +2691,7 @@ struct<> -- !query 336 -INSERT INTO num_exp_mul VALUES (8,1,'0') +INSERT INTO num_exp_mul VALUES (8,1,0) -- !query 336 schema struct<> -- !query 336 output @@ -2699,7 +2699,7 @@ struct<> -- !query 337 -INSERT INTO num_exp_div VALUES (8,1,'NaN') +INSERT INTO num_exp_div VALUES (8,1,double('NaN')) -- !query 337 schema struct<> -- !query 337 output @@ -2707,7 +2707,7 @@ struct<> -- !query 338 -INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047') +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) -- !query 338 schema struct<> -- !query 338 output @@ -2715,7 +2715,7 @@ struct<> -- !query 339 -INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047') +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) -- !query 339 schema struct<> -- !query 339 output @@ -2723,7 +2723,7 @@ struct<> -- !query 340 -INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407') +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) -- !query 340 schema struct<> -- !query 340 output @@ -2731,7 +2731,7 @@ struct<> -- !query 341 -INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615') +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) -- !query 341 schema struct<> -- !query 341 output @@ -2739,7 +2739,7 @@ struct<> -- !query 342 -INSERT INTO num_exp_add VALUES (8,3,'74885.31') +INSERT INTO num_exp_add VALUES (8,3,74885.31) -- !query 342 schema struct<> -- !query 342 output @@ -2747,7 +2747,7 @@ struct<> -- !query 343 -INSERT INTO num_exp_sub VALUES (8,3,'74876.69') +INSERT INTO num_exp_sub VALUES (8,3,74876.69) -- !query 343 schema struct<> -- !query 343 output @@ -2755,7 +2755,7 @@ struct<> -- !query 344 -INSERT INTO num_exp_mul VALUES (8,3,'322737.11') +INSERT INTO num_exp_mul VALUES (8,3,322737.11) -- !query 344 schema struct<> -- !query 344 output @@ -2763,7 +2763,7 @@ struct<> -- !query 345 -INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410') +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) -- !query 345 schema struct<> -- !query 345 output @@ -2771,7 +2771,7 @@ struct<> -- !query 346 -INSERT INTO num_exp_add VALUES (8,4,'7874342.4119') +INSERT INTO num_exp_add VALUES (8,4,7874342.4119) -- !query 346 schema struct<> -- !query 346 output @@ -2779,7 +2779,7 @@ struct<> -- !query 347 -INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119') +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) -- !query 347 schema struct<> -- !query 347 output @@ -2787,7 +2787,7 @@ struct<> -- !query 348 -INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839') +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) -- !query 348 schema struct<> -- !query 348 output @@ -2795,7 +2795,7 @@ struct<> -- !query 349 -INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956') +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) -- !query 349 schema struct<> -- !query 349 output @@ -2803,7 +2803,7 @@ struct<> -- !query 350 -INSERT INTO num_exp_add VALUES (8,5,'91278.038491') +INSERT INTO num_exp_add VALUES (8,5,91278.038491) -- !query 350 schema struct<> -- !query 350 output @@ -2811,7 +2811,7 @@ struct<> -- !query 351 -INSERT INTO num_exp_sub VALUES (8,5,'58483.961509') +INSERT INTO num_exp_sub VALUES (8,5,58483.961509) -- !query 351 schema struct<> -- !query 351 output @@ -2819,7 +2819,7 @@ struct<> -- !query 352 -INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571') +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) -- !query 352 schema struct<> -- !query 352 output @@ -2827,7 +2827,7 @@ struct<> -- !query 353 -INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456') +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) -- !query 353 schema struct<> -- !query 353 output @@ -2835,7 +2835,7 @@ struct<> -- !query 354 -INSERT INTO num_exp_add VALUES (8,6,'168782.57763026') +INSERT INTO num_exp_add VALUES (8,6,168782.57763026) -- !query 354 schema struct<> -- !query 354 output @@ -2843,7 +2843,7 @@ struct<> -- !query 355 -INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026') +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) -- !query 355 schema struct<> -- !query 355 output @@ -2851,7 +2851,7 @@ struct<> -- !query 356 -INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906') +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) -- !query 356 schema struct<> -- !query 356 output @@ -2859,7 +2859,7 @@ struct<> -- !query 357 -INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424') +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) -- !query 357 schema struct<> -- !query 357 output @@ -2867,7 +2867,7 @@ struct<> -- !query 358 -INSERT INTO num_exp_add VALUES (8,7,'-82953604') +INSERT INTO num_exp_add VALUES (8,7,-82953604) -- !query 358 schema struct<> -- !query 358 output @@ -2875,7 +2875,7 @@ struct<> -- !query 359 -INSERT INTO num_exp_sub VALUES (8,7,'83103366') +INSERT INTO num_exp_sub VALUES (8,7,83103366) -- !query 359 schema struct<> -- !query 359 output @@ -2883,7 +2883,7 @@ struct<> -- !query 360 -INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285') +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) -- !query 360 schema struct<> -- !query 360 output @@ -2891,7 +2891,7 @@ struct<> -- !query 361 -INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172') +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) -- !query 361 schema struct<> -- !query 361 output @@ -2899,7 +2899,7 @@ struct<> -- !query 362 -INSERT INTO num_exp_add VALUES (8,8,'149762') +INSERT INTO num_exp_add VALUES (8,8,149762) -- !query 362 schema struct<> -- !query 362 output @@ -2907,7 +2907,7 @@ struct<> -- !query 363 -INSERT INTO num_exp_sub VALUES (8,8,'0') +INSERT INTO num_exp_sub VALUES (8,8,0) -- !query 363 schema struct<> -- !query 363 output @@ -2915,7 +2915,7 @@ struct<> -- !query 364 -INSERT INTO num_exp_mul VALUES (8,8,'5607164161') +INSERT INTO num_exp_mul VALUES (8,8,5607164161) -- !query 364 schema struct<> -- !query 364 output @@ -2923,7 +2923,7 @@ struct<> -- !query 365 -INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) -- !query 365 schema struct<> -- !query 365 output @@ -2931,7 +2931,7 @@ struct<> -- !query 366 -INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420') +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) -- !query 366 schema struct<> -- !query 366 output @@ -2939,7 +2939,7 @@ struct<> -- !query 367 -INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420') +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) -- !query 367 schema struct<> -- !query 367 output @@ -2947,7 +2947,7 @@ struct<> -- !query 368 -INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020') +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) -- !query 368 schema struct<> -- !query 368 output @@ -2955,7 +2955,7 @@ struct<> -- !query 369 -INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735') +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) -- !query 369 schema struct<> -- !query 369 output @@ -2963,7 +2963,7 @@ struct<> -- !query 370 -INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) -- !query 370 schema struct<> -- !query 370 output @@ -2971,7 +2971,7 @@ struct<> -- !query 371 -INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420') +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) -- !query 371 schema struct<> -- !query 371 output @@ -2979,7 +2979,7 @@ struct<> -- !query 372 -INSERT INTO num_exp_mul VALUES (9,0,'0') +INSERT INTO num_exp_mul VALUES (9,0,0) -- !query 372 schema struct<> -- !query 372 output @@ -2987,7 +2987,7 @@ struct<> -- !query 373 -INSERT INTO num_exp_div VALUES (9,0,'NaN') +INSERT INTO num_exp_div VALUES (9,0,double('NaN')) -- !query 373 schema struct<> -- !query 373 output @@ -2995,7 +2995,7 @@ struct<> -- !query 374 -INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) -- !query 374 schema struct<> -- !query 374 output @@ -3003,7 +3003,7 @@ struct<> -- !query 375 -INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420') +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) -- !query 375 schema struct<> -- !query 375 output @@ -3011,7 +3011,7 @@ struct<> -- !query 376 -INSERT INTO num_exp_mul VALUES (9,1,'0') +INSERT INTO num_exp_mul VALUES (9,1,0) -- !query 376 schema struct<> -- !query 376 output @@ -3019,7 +3019,7 @@ struct<> -- !query 377 -INSERT INTO num_exp_div VALUES (9,1,'NaN') +INSERT INTO num_exp_div VALUES (9,1,double('NaN')) -- !query 377 schema struct<> -- !query 377 output @@ -3027,7 +3027,7 @@ struct<> -- !query 378 -INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467') +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) -- !query 378 schema struct<> -- !query 378 output @@ -3035,7 +3035,7 @@ struct<> -- !query 379 -INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627') +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) -- !query 379 schema struct<> -- !query 379 output @@ -3043,7 +3043,7 @@ struct<> -- !query 380 -INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740') +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) -- !query 380 schema struct<> -- !query 380 output @@ -3051,7 +3051,7 @@ struct<> -- !query 381 -INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526') +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) -- !query 381 schema struct<> -- !query 381 output @@ -3059,7 +3059,7 @@ struct<> -- !query 382 -INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420') +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) -- !query 382 schema struct<> -- !query 382 output @@ -3067,7 +3067,7 @@ struct<> -- !query 383 -INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420') +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) -- !query 383 schema struct<> -- !query 383 output @@ -3075,7 +3075,7 @@ struct<> -- !query 384 -INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020') +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) -- !query 384 schema struct<> -- !query 384 output @@ -3083,7 +3083,7 @@ struct<> -- !query 385 -INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677') +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) -- !query 385 schema struct<> -- !query 385 output @@ -3091,7 +3091,7 @@ struct<> -- !query 386 -INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420') +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) -- !query 386 schema struct<> -- !query 386 output @@ -3099,7 +3099,7 @@ struct<> -- !query 387 -INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420') +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) -- !query 387 schema struct<> -- !query 387 output @@ -3107,7 +3107,7 @@ struct<> -- !query 388 -INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980') +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) -- !query 388 schema struct<> -- !query 388 output @@ -3115,7 +3115,7 @@ struct<> -- !query 389 -INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484') +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) -- !query 389 schema struct<> -- !query 389 output @@ -3123,7 +3123,7 @@ struct<> -- !query 390 -INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420') +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) -- !query 390 schema struct<> -- !query 390 output @@ -3131,7 +3131,7 @@ struct<> -- !query 391 -INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420') +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) -- !query 391 schema struct<> -- !query 391 output @@ -3139,7 +3139,7 @@ struct<> -- !query 392 -INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220') +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) -- !query 392 schema struct<> -- !query 392 output @@ -3147,7 +3147,7 @@ struct<> -- !query 393 -INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807') +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) -- !query 393 schema struct<> -- !query 393 output @@ -3155,7 +3155,7 @@ struct<> -- !query 394 -INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160') +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) -- !query 394 schema struct<> -- !query 394 output @@ -3163,7 +3163,7 @@ struct<> -- !query 395 -INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680') +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) -- !query 395 schema struct<> -- !query 395 output @@ -3171,7 +3171,7 @@ struct<> -- !query 396 -INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920') +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) -- !query 396 schema struct<> -- !query 396 output @@ -3179,7 +3179,7 @@ struct<> -- !query 397 -INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280') +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) -- !query 397 schema struct<> -- !query 397 output @@ -3187,7 +3187,7 @@ struct<> -- !query 398 -INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420') +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) -- !query 398 schema struct<> -- !query 398 output @@ -3195,7 +3195,7 @@ struct<> -- !query 399 -INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580') +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) -- !query 399 schema struct<> -- !query 399 output @@ -3203,7 +3203,7 @@ struct<> -- !query 400 -INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700') +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) -- !query 400 schema struct<> -- !query 400 output @@ -3211,7 +3211,7 @@ struct<> -- !query 401 -INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689') +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) -- !query 401 schema struct<> -- !query 401 output @@ -3219,7 +3219,7 @@ struct<> -- !query 402 -INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420') +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) -- !query 402 schema struct<> -- !query 402 output @@ -3227,7 +3227,7 @@ struct<> -- !query 403 -INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420') +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) -- !query 403 schema struct<> -- !query 403 output @@ -3235,7 +3235,7 @@ struct<> -- !query 404 -INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020') +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) -- !query 404 schema struct<> -- !query 404 output @@ -3243,7 +3243,7 @@ struct<> -- !query 405 -INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748') +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) -- !query 405 schema struct<> -- !query 405 output @@ -3251,7 +3251,7 @@ struct<> -- !query 406 -INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840') +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) -- !query 406 schema struct<> -- !query 406 output @@ -3259,7 +3259,7 @@ struct<> -- !query 407 -INSERT INTO num_exp_sub VALUES (9,9,'0') +INSERT INTO num_exp_sub VALUES (9,9,0) -- !query 407 schema struct<> -- !query 407 output @@ -3267,7 +3267,7 @@ struct<> -- !query 408 -INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400') +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) -- !query 408 schema struct<> -- !query 408 output @@ -3275,7 +3275,7 @@ struct<> -- !query 409 -INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) -- !query 409 schema struct<> -- !query 409 output @@ -3283,7 +3283,7 @@ struct<> -- !query 410 -INSERT INTO num_exp_sqrt VALUES (0,'0') +INSERT INTO num_exp_sqrt VALUES (0,0) -- !query 410 schema struct<> -- !query 410 output @@ -3291,7 +3291,7 @@ struct<> -- !query 411 -INSERT INTO num_exp_sqrt VALUES (1,'0') +INSERT INTO num_exp_sqrt VALUES (1,0) -- !query 411 schema struct<> -- !query 411 output @@ -3299,7 +3299,7 @@ struct<> -- !query 412 -INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505') +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) -- !query 412 schema struct<> -- !query 412 output @@ -3307,7 +3307,7 @@ struct<> -- !query 413 -INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396') +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) -- !query 413 schema struct<> -- !query 413 output @@ -3315,7 +3315,7 @@ struct<> -- !query 414 -INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923') +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) -- !query 414 schema struct<> -- !query 414 output @@ -3323,7 +3323,7 @@ struct<> -- !query 415 -INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473') +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) -- !query 415 schema struct<> -- !query 415 output @@ -3331,7 +3331,7 @@ struct<> -- !query 416 -INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406') +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) -- !query 416 schema struct<> -- !query 416 output @@ -3339,7 +3339,7 @@ struct<> -- !query 417 -INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230') +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) -- !query 417 schema struct<> -- !query 417 output @@ -3347,7 +3347,7 @@ struct<> -- !query 418 -INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542') +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) -- !query 418 schema struct<> -- !query 418 output @@ -3355,7 +3355,7 @@ struct<> -- !query 419 -INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766') +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) -- !query 419 schema struct<> -- !query 419 output @@ -3363,7 +3363,7 @@ struct<> -- !query 420 -INSERT INTO num_exp_ln VALUES (0,'NaN') +INSERT INTO num_exp_ln VALUES (0,double('NaN')) -- !query 420 schema struct<> -- !query 420 output @@ -3371,7 +3371,7 @@ struct<> -- !query 421 -INSERT INTO num_exp_ln VALUES (1,'NaN') +INSERT INTO num_exp_ln VALUES (1,double('NaN')) -- !query 421 schema struct<> -- !query 421 output @@ -3379,7 +3379,7 @@ struct<> -- !query 422 -INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514') +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) -- !query 422 schema struct<> -- !query 422 output @@ -3387,7 +3387,7 @@ struct<> -- !query 423 -INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971') +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) -- !query 423 schema struct<> -- !query 423 output @@ -3395,7 +3395,7 @@ struct<> -- !query 424 -INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464') +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) -- !query 424 schema struct<> -- !query 424 output @@ -3403,7 +3403,7 @@ struct<> -- !query 425 -INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038') +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) -- !query 425 schema struct<> -- !query 425 output @@ -3411,7 +3411,7 @@ struct<> -- !query 426 -INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127') +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) -- !query 426 schema struct<> -- !query 426 output @@ -3419,7 +3419,7 @@ struct<> -- !query 427 -INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991') +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) -- !query 427 schema struct<> -- !query 427 output @@ -3427,7 +3427,7 @@ struct<> -- !query 428 -INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668') +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) -- !query 428 schema struct<> -- !query 428 output @@ -3435,7 +3435,7 @@ struct<> -- !query 429 -INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962') +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) -- !query 429 schema struct<> -- !query 429 output @@ -3443,7 +3443,7 @@ struct<> -- !query 430 -INSERT INTO num_exp_log10 VALUES (0,'NaN') +INSERT INTO num_exp_log10 VALUES (0,double('NaN')) -- !query 430 schema struct<> -- !query 430 output @@ -3451,7 +3451,7 @@ struct<> -- !query 431 -INSERT INTO num_exp_log10 VALUES (1,'NaN') +INSERT INTO num_exp_log10 VALUES (1,double('NaN')) -- !query 431 schema struct<> -- !query 431 output @@ -3459,7 +3459,7 @@ struct<> -- !query 432 -INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459') +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) -- !query 432 schema struct<> -- !query 432 output @@ -3467,7 +3467,7 @@ struct<> -- !query 433 -INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075') +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) -- !query 433 schema struct<> -- !query 433 output @@ -3475,7 +3475,7 @@ struct<> -- !query 434 -INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345') +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) -- !query 434 schema struct<> -- !query 434 output @@ -3483,7 +3483,7 @@ struct<> -- !query 435 -INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626') +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) -- !query 435 schema struct<> -- !query 435 output @@ -3491,7 +3491,7 @@ struct<> -- !query 436 -INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671') +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) -- !query 436 schema struct<> -- !query 436 output @@ -3499,7 +3499,7 @@ struct<> -- !query 437 -INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914') +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) -- !query 437 schema struct<> -- !query 437 output @@ -3507,7 +3507,7 @@ struct<> -- !query 438 -INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138') +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) -- !query 438 schema struct<> -- !query 438 output @@ -3515,7 +3515,7 @@ struct<> -- !query 439 -INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059') +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) -- !query 439 schema struct<> -- !query 439 output @@ -3523,7 +3523,7 @@ struct<> -- !query 440 -INSERT INTO num_exp_power_10_ln VALUES (0,'NaN') +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) -- !query 440 schema struct<> -- !query 440 output @@ -3531,7 +3531,7 @@ struct<> -- !query 441 -INSERT INTO num_exp_power_10_ln VALUES (1,'NaN') +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) -- !query 441 schema struct<> -- !query 441 output @@ -3539,7 +3539,7 @@ struct<> -- !query 442 -INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184') +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) -- !query 442 schema struct<> -- !query 442 output @@ -3547,7 +3547,7 @@ struct<> -- !query 443 -INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393') +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) -- !query 443 schema struct<> -- !query 443 output @@ -3555,7 +3555,7 @@ struct<> -- !query 444 -INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636') +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) -- !query 444 schema struct<> -- !query 444 output @@ -3563,7 +3563,7 @@ struct<> -- !query 445 -INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098') +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) -- !query 445 schema struct<> -- !query 445 output @@ -3571,7 +3571,7 @@ struct<> -- !query 446 -INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067') +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) -- !query 446 schema struct<> -- !query 446 output @@ -3579,15 +3579,19 @@ struct<> -- !query 447 -INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627') +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) -- !query 447 schema struct<> -- !query 447 output +org.apache.spark.sql.catalyst.parser.ParseException +decimal can only support precision up to 38 +== SQL == +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) -- !query 448 -INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952') +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) -- !query 448 schema struct<> -- !query 448 output @@ -3595,7 +3599,7 @@ struct<> -- !query 449 -INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457') +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) -- !query 449 schema struct<> -- !query 449 output @@ -3603,7 +3607,7 @@ struct<> -- !query 450 -INSERT INTO num_data VALUES (0, '0') +INSERT INTO num_data VALUES (0, 0) -- !query 450 schema struct<> -- !query 450 output @@ -3611,7 +3615,7 @@ struct<> -- !query 451 -INSERT INTO num_data VALUES (1, '0') +INSERT INTO num_data VALUES (1, 0) -- !query 451 schema struct<> -- !query 451 output @@ -3619,7 +3623,7 @@ struct<> -- !query 452 -INSERT INTO num_data VALUES (2, '-34338492.215397047') +INSERT INTO num_data VALUES (2, -34338492.215397047) -- !query 452 schema struct<> -- !query 452 output @@ -3627,7 +3631,7 @@ struct<> -- !query 453 -INSERT INTO num_data VALUES (3, '4.31') +INSERT INTO num_data VALUES (3, 4.31) -- !query 453 schema struct<> -- !query 453 output @@ -3635,7 +3639,7 @@ struct<> -- !query 454 -INSERT INTO num_data VALUES (4, '7799461.4119') +INSERT INTO num_data VALUES (4, 7799461.4119) -- !query 454 schema struct<> -- !query 454 output @@ -3643,7 +3647,7 @@ struct<> -- !query 455 -INSERT INTO num_data VALUES (5, '16397.038491') +INSERT INTO num_data VALUES (5, 16397.038491) -- !query 455 schema struct<> -- !query 455 output @@ -3651,7 +3655,7 @@ struct<> -- !query 456 -INSERT INTO num_data VALUES (6, '93901.57763026') +INSERT INTO num_data VALUES (6, 93901.57763026) -- !query 456 schema struct<> -- !query 456 output @@ -3659,7 +3663,7 @@ struct<> -- !query 457 -INSERT INTO num_data VALUES (7, '-83028485') +INSERT INTO num_data VALUES (7, -83028485) -- !query 457 schema struct<> -- !query 457 output @@ -3667,7 +3671,7 @@ struct<> -- !query 458 -INSERT INTO num_data VALUES (8, '74881') +INSERT INTO num_data VALUES (8, 74881) -- !query 458 schema struct<> -- !query 458 output @@ -3675,7 +3679,7 @@ struct<> -- !query 459 -INSERT INTO num_data VALUES (9, '-24926804.045047420') +INSERT INTO num_data VALUES (9, -24926804.045047420) -- !query 459 schema struct<> -- !query 459 output @@ -4196,7 +4200,6 @@ struct 4 7405685069595001 7405685069594999.0773399947 5 5068226527.321263 5068226527.3212726541 6 281839893606.99365 281839893606.9937234336 -7 1716699575118595580 1716699575118597095.4233081991 8 167361463828.0749 167361463828.0749132007 9 107511333880051872 107511333880052007.0414112467 @@ -4218,7 +4221,7 @@ struct<> -- !query 499 -INSERT INTO fract_only VALUES (1, '0.0') +INSERT INTO fract_only VALUES (1, 0.0) -- !query 499 schema struct<> -- !query 499 output @@ -4226,7 +4229,7 @@ struct<> -- !query 500 -INSERT INTO fract_only VALUES (2, '0.1') +INSERT INTO fract_only VALUES (2, 0.1) -- !query 500 schema struct<> -- !query 500 output @@ -4234,7 +4237,7 @@ struct<> -- !query 501 -INSERT INTO fract_only VALUES (4, '-0.9999') +INSERT INTO fract_only VALUES (4, -0.9999) -- !query 501 schema struct<> -- !query 501 output @@ -4242,7 +4245,7 @@ struct<> -- !query 502 -INSERT INTO fract_only VALUES (5, '0.99994') +INSERT INTO fract_only VALUES (5, 0.99994) -- !query 502 schema struct<> -- !query 502 output @@ -4250,7 +4253,7 @@ struct<> -- !query 503 -INSERT INTO fract_only VALUES (7, '0.00001') +INSERT INTO fract_only VALUES (7, 0.00001) -- !query 503 schema struct<> -- !query 503 output @@ -4258,7 +4261,7 @@ struct<> -- !query 504 -INSERT INTO fract_only VALUES (8, '0.00017') +INSERT INTO fract_only VALUES (8, 0.00017) -- !query 504 schema struct<> -- !query 504 output @@ -4343,7 +4346,7 @@ struct<> -- !query 514 -INSERT INTO ceil_floor_round VALUES ('-5.5') +INSERT INTO ceil_floor_round VALUES (-5.5) -- !query 514 schema struct<> -- !query 514 output @@ -4351,7 +4354,7 @@ struct<> -- !query 515 -INSERT INTO ceil_floor_round VALUES ('-5.499999') +INSERT INTO ceil_floor_round VALUES (-5.499999) -- !query 515 schema struct<> -- !query 515 output @@ -4359,7 +4362,7 @@ struct<> -- !query 516 -INSERT INTO ceil_floor_round VALUES ('9.5') +INSERT INTO ceil_floor_round VALUES (9.5) -- !query 516 schema struct<> -- !query 516 output @@ -4367,7 +4370,7 @@ struct<> -- !query 517 -INSERT INTO ceil_floor_round VALUES ('9.4999999') +INSERT INTO ceil_floor_round VALUES (9.4999999) -- !query 517 schema struct<> -- !query 517 output @@ -4375,7 +4378,7 @@ struct<> -- !query 518 -INSERT INTO ceil_floor_round VALUES ('0.0') +INSERT INTO ceil_floor_round VALUES (0.0) -- !query 518 schema struct<> -- !query 518 output @@ -4383,7 +4386,7 @@ struct<> -- !query 519 -INSERT INTO ceil_floor_round VALUES ('0.0000001') +INSERT INTO ceil_floor_round VALUES (0.0000001) -- !query 519 schema struct<> -- !query 519 output @@ -4391,7 +4394,7 @@ struct<> -- !query 520 -INSERT INTO ceil_floor_round VALUES ('-0.000001') +INSERT INTO ceil_floor_round VALUES (-0.000001) -- !query 520 schema struct<> -- !query 520 output @@ -4429,7 +4432,7 @@ struct<> -- !query 524 -INSERT INTO num_input_test VALUES (trim(' 123')) +INSERT INTO num_input_test VALUES (double(trim(' 123'))) -- !query 524 schema struct<> -- !query 524 output @@ -4437,7 +4440,7 @@ struct<> -- !query 525 -INSERT INTO num_input_test VALUES (trim(' 3245874 ')) +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) -- !query 525 schema struct<> -- !query 525 output @@ -4445,7 +4448,7 @@ struct<> -- !query 526 -INSERT INTO num_input_test VALUES (trim(' -93853')) +INSERT INTO num_input_test VALUES (double(trim(' -93853'))) -- !query 526 schema struct<> -- !query 526 output @@ -4453,7 +4456,7 @@ struct<> -- !query 527 -INSERT INTO num_input_test VALUES ('555.50') +INSERT INTO num_input_test VALUES (555.50) -- !query 527 schema struct<> -- !query 527 output @@ -4461,7 +4464,7 @@ struct<> -- !query 528 -INSERT INTO num_input_test VALUES ('-555.50') +INSERT INTO num_input_test VALUES (-555.50) -- !query 528 schema struct<> -- !query 528 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index f8bce0c97c8c5..6cb3690a464e1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO TIMESTAMP_TBL VALUES ('now') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO TIMESTAMP_TBL VALUES ('now') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO TIMESTAMP_TBL VALUES ('today') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('today')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO TIMESTAMP_TBL VALUES ('yesterday') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('yesterday')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow')) -- !query 5 schema struct<> -- !query 5 output @@ -51,7 +51,7 @@ struct<> -- !query 6 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow EST')) -- !query 6 schema struct<> -- !query 6 output @@ -59,7 +59,7 @@ struct<> -- !query 7 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow Zulu')) -- !query 7 schema struct<> -- !query 7 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO TIMESTAMP_TBL VALUES ('epoch') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('epoch')) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')) -- !query 15 schema struct<> -- !query 15 output @@ -131,7 +131,7 @@ struct<> -- !query 16 -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')) -- !query 16 schema struct<> -- !query 16 output @@ -219,11 +219,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 24 schema struct<54:string,diff:interval> -- !query 24 output - interval -236720 hours - interval 0 microseconds - interval 3 hours 4 minutes 5 seconds - interval 41393 hours 19 minutes 20 seconds - interval 953 hours 32 minutes 1 seconds + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds -- !query 25 @@ -242,11 +242,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 26 schema struct<54:string,diff:interval> -- !query 26 output - interval -236720 hours - interval 0 microseconds - interval 3 hours 4 minutes 5 seconds - interval 41393 hours 19 minutes 20 seconds - interval 953 hours 32 minutes 1 seconds + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds -- !query 27 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 ae0ccf556fbc3..960e644732990 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 @@ -143,19 +143,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. - // SPARK-28885 String value is not allowed to be stored as numeric type with - // ANSI store assignment policy. - "postgreSQL/numeric.sql", - "postgreSQL/int2.sql", - "postgreSQL/int4.sql", - "postgreSQL/int8.sql", - "postgreSQL/float4.sql", - "postgreSQL/float8.sql", - // SPARK-28885 String value is not allowed to be stored as date/timestamp type with - // ANSI store assignment policy. - "postgreSQL/date.sql", - "postgreSQL/timestamp.sql" + "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. ) // Create all the test cases. From 23b3c4fafdf37a482b3f823a5701d99d96236519 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 21 Nov 2019 00:56:48 +0800 Subject: [PATCH 55/83] [SPARK-29951][SQL] Make the behavior of Postgre dialect independent of ansi mode config ### What changes were proposed in this pull request? Fix the inconsistent behavior of build-in function SQL LEFT/RIGHT. ### Why are the changes needed? As the comment in https://github.com/apache/spark/pull/26497#discussion_r345708065, Postgre dialect should not be affected by the ANSI mode config. During reran the existing tests, only the LEFT/RIGHT build-in SQL function broke the assumption. We fix this by following https://www.postgresql.org/docs/12/sql-keywords-appendix.html: `LEFT/RIGHT reserved (can be function or type)` ### Does this PR introduce any user-facing change? Yes, the Postgre dialect will not be affected by the ANSI mode config. ### How was this patch tested? Existing UT. Closes #26584 from xuanyuanking/SPARK-29951. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 7 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 +-- .../sql/catalyst/parser/AstBuilder.scala | 16 +- .../sql/catalyst/parser/ParseDriver.scala | 12 +- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 - .../sql-tests/results/postgreSQL/text.sql.out | 160 ++++++++---------- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 - 8 files changed, 119 insertions(+), 111 deletions(-) mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 79bc134596237..3117ee40a8c9b 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,12 +19,13 @@ license: | limitations under the License. --- -When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect=PostgreSQL` or keep default `spark.sql.dialect=Spark` with setting `spark.sql.dialect.spark.ansi.enabled` to true, Spark SQL will use the ANSI mode parser. +In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +When the ANSI mode is disabled, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when the ANSI mode enabled. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. By default `spark.sql.dialect.spark.ansi.enabled` is false. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4f81dc47394a5..cc273fd36011e 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -46,9 +46,9 @@ grammar SqlBase; } /** - * When true, ANSI SQL parsing mode is enabled. + * When true, the behavior of keywords follows ANSI SQL standard. */ - public boolean ansi = false; + public boolean SQL_standard_keyword_behavior = false; } singleStatement @@ -744,7 +744,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' + | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | identifier '->' expression #lambda | '(' identifier (',' identifier)+ ')' '->' expression #lambda @@ -788,7 +788,7 @@ booleanValue interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? - | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) + | {SQL_standard_keyword_behavior}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval @@ -908,6 +908,12 @@ qualifiedNameList : qualifiedName (',' qualifiedName)* ; +functionName + : qualifiedName + | LEFT + | RIGHT + ; + qualifiedName : identifier ('.' identifier)* ; @@ -927,14 +933,14 @@ errorCapturingIdentifierExtra identifier : strictIdentifier - | {!ansi}? strictNonReserved + | {!SQL_standard_keyword_behavior}? strictNonReserved ; strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative - | {ansi}? ansiNonReserved #unquotedIdentifier - | {!ansi}? nonReserved #unquotedIdentifier + | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier ; quotedIdentifier @@ -951,7 +957,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1151,9 +1157,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +// Same definition as the one when `SQL_standard_keyword_behavior=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e0476879f0e80..7bec46678f58d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1581,7 +1581,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { // Create the function call. - val name = ctx.qualifiedName.getText + val name = ctx.functionName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) @@ -1591,7 +1591,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case expressions => expressions } - val function = UnresolvedFunction(visitFunctionName(ctx.qualifiedName), arguments, isDistinct) + val function = UnresolvedFunction( + getFunctionIdentifier(ctx.functionName), arguments, isDistinct) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { @@ -1631,6 +1632,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Get a function identifier consist by database (optional) and name. + */ + protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { + if (ctx.qualifiedName != null) { + visitFunctionName(ctx.qualifiedName) + } else { + FunctionIdentifier(ctx.getText, None) + } + } + /** * Create an [[LambdaFunction]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index e291d5f9cd84c..30c36598d81d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** @@ -88,11 +89,18 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logDebug(s"Parsing command: $command") + // When we use PostgreSQL dialect or use Spark dialect with setting + // `spark.sql.dialect.spark.ansi.enabled=true`, the parser will use ANSI SQL standard keywords. + val SQLStandardKeywordBehavior = conf.dialect match { + case Dialect.POSTGRESQL => true + case Dialect.SPARK => conf.dialectSparkAnsiEnabled + } + val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.ansi = conf.dialectSparkAnsiEnabled + lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -100,7 +108,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - parser.ansi = conf.dialectSparkAnsiEnabled + parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d4fcefe99ee75..74046cd91c962 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2515,7 +2515,9 @@ class SQLConf extends Serializable with Logging { def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString + def dialect: Dialect.Value = Dialect.withName(getConf(DIALECT)) + + def usePostgreSQLDialect: Boolean = dialect == Dialect.POSTGRESQL def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index a1fe95462ecae..05953123da86f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -44,11 +44,7 @@ select concat_ws(',',10,20,null,30); select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); --- [SPARK-28036] Built-in udf left/right has inconsistent behavior --- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.dialect.spark.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.dialect.spark.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out old mode 100644 new mode 100755 index cac1e7ee5ab12..311b0eb5a5844 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 44 +-- Number of queries: 42 -- !query 0 @@ -151,18 +151,10 @@ edcba -- !query 18 -set spark.sql.dialect.spark.ansi.enabled=false --- !query 18 schema -struct --- !query 18 output -spark.sql.dialect.spark.ansi.enabled false - - --- !query 19 select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i --- !query 19 schema +-- !query 18 schema struct --- !query 19 output +-- !query 18 output -5 -4 -3 @@ -176,200 +168,192 @@ struct 5 ahoj ahoj --- !query 20 -set spark.sql.dialect.spark.ansi.enabled=true --- !query 20 schema -struct --- !query 20 output -spark.sql.dialect.spark.ansi.enabled true - - --- !query 21 +-- !query 19 /* * format */ select format_string(NULL) --- !query 21 schema +-- !query 19 schema struct --- !query 21 output +-- !query 19 output NULL --- !query 22 +-- !query 20 select format_string('Hello') --- !query 22 schema +-- !query 20 schema struct --- !query 22 output +-- !query 20 output Hello --- !query 23 +-- !query 21 select format_string('Hello %s', 'World') --- !query 23 schema +-- !query 21 schema struct --- !query 23 output +-- !query 21 output Hello World --- !query 24 +-- !query 22 select format_string('Hello %%') --- !query 24 schema +-- !query 22 schema struct --- !query 24 output +-- !query 22 output Hello % --- !query 25 +-- !query 23 select format_string('Hello %%%%') --- !query 25 schema +-- !query 23 schema struct --- !query 25 output +-- !query 23 output Hello %% --- !query 26 +-- !query 24 select format_string('Hello %s %s', 'World') --- !query 26 schema +-- !query 24 schema struct<> --- !query 26 output +-- !query 24 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 27 +-- !query 25 select format_string('Hello %s') --- !query 27 schema +-- !query 25 schema struct<> --- !query 27 output +-- !query 25 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 28 +-- !query 26 select format_string('Hello %x', 20) --- !query 28 schema +-- !query 26 schema struct --- !query 28 output +-- !query 26 output Hello 14 --- !query 29 +-- !query 27 select format_string('%1$s %3$s', 1, 2, 3) --- !query 29 schema +-- !query 27 schema struct --- !query 29 output +-- !query 27 output 1 3 --- !query 30 +-- !query 28 select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 30 schema +-- !query 28 schema struct --- !query 30 output +-- !query 28 output 1 12 --- !query 31 +-- !query 29 select format_string('%1$s %4$s', 1, 2, 3) --- !query 31 schema +-- !query 29 schema struct<> --- !query 31 output +-- !query 29 output java.util.MissingFormatArgumentException Format specifier '%4$s' --- !query 32 +-- !query 30 select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 32 schema +-- !query 30 schema struct<> --- !query 32 output +-- !query 30 output java.util.MissingFormatArgumentException Format specifier '%13$s' --- !query 33 +-- !query 31 select format_string('%0$s', 'Hello') --- !query 33 schema +-- !query 31 schema struct --- !query 33 output +-- !query 31 output Hello --- !query 34 +-- !query 32 select format_string('Hello %s %1$s %s', 'World', 'Hello again') --- !query 34 schema +-- !query 32 schema struct --- !query 34 output +-- !query 32 output Hello World World Hello again --- !query 35 +-- !query 33 select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') --- !query 35 schema +-- !query 33 schema struct --- !query 35 output +-- !query 33 output Hello World Hello again, Hello again Hello again --- !query 36 +-- !query 34 select format_string('>>%10s<<', 'Hello') --- !query 36 schema +-- !query 34 schema struct>%10s<<, Hello):string> --- !query 36 output +-- !query 34 output >> Hello<< --- !query 37 +-- !query 35 select format_string('>>%10s<<', NULL) --- !query 37 schema +-- !query 35 schema struct>%10s<<, NULL):string> --- !query 37 output +-- !query 35 output >> null<< --- !query 38 +-- !query 36 select format_string('>>%10s<<', '') --- !query 38 schema +-- !query 36 schema struct>%10s<<, ):string> --- !query 38 output +-- !query 36 output >> << --- !query 39 +-- !query 37 select format_string('>>%-10s<<', '') --- !query 39 schema +-- !query 37 schema struct>%-10s<<, ):string> --- !query 39 output +-- !query 37 output >> << --- !query 40 +-- !query 38 select format_string('>>%-10s<<', 'Hello') --- !query 40 schema +-- !query 38 schema struct>%-10s<<, Hello):string> --- !query 40 output +-- !query 38 output >>Hello << --- !query 41 +-- !query 39 select format_string('>>%-10s<<', NULL) --- !query 41 schema +-- !query 39 schema struct>%-10s<<, NULL):string> --- !query 41 output +-- !query 39 output >>null << --- !query 42 +-- !query 40 select format_string('>>%1$10s<<', 'Hello') --- !query 42 schema +-- !query 40 schema struct>%1$10s<<, Hello):string> --- !query 42 output +-- !query 40 output >> Hello<< --- !query 43 +-- !query 41 DROP TABLE TEXT_TBL --- !query 43 schema +-- !query 41 schema struct<> --- !query 43 output +-- !query 41 output 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 960e644732990..6e86d37a310d6 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 @@ -335,7 +335,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) From 1febd373ea806326d269a60048ee52543a76c918 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 20 Nov 2019 14:04:15 -0600 Subject: [PATCH 56/83] [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests ### What changes were proposed in this pull request? Use JUnit assertions in tests uniformly, not JVM assert() statements. ### Why are the changes needed? assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything. ### Does this PR introduce any user-facing change? No. The assertion logic should be identical. ### How was this patch tested? Existing tests. Closes #26581 from srowen/assertToJUnit. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../shuffle/TestShuffleDataContext.java | 3 +- .../org/apache/spark/DistributedSuite.scala | 1 + .../spark/MapStatusesSerDeserBenchmark.scala | 2 ++ .../apache/spark/benchmark/Benchmark.scala | 1 + .../apache/spark/executor/ExecutorSuite.scala | 1 + .../apache/spark/rpc/TestRpcEndpoint.scala | 1 + .../scheduler/SchedulerIntegrationSuite.scala | 3 +- .../scheduler/TaskResultGetterSuite.scala | 1 + .../spark/scheduler/TaskSetManagerSuite.scala | 3 +- .../ShuffleDriverComponentsSuite.scala | 1 + .../util/PeriodicRDDCheckpointerSuite.scala | 1 + .../spark/sql/kafka010/KafkaTestUtils.scala | 6 ++-- .../JavaKinesisInputDStreamBuilderSuite.java | 34 +++++++++++-------- .../util/PeriodicGraphCheckpointerSuite.scala | 3 +- .../stat/JavaKolmogorovSmirnovTestSuite.java | 5 +-- .../ml/classification/LinearSVCSuite.scala | 1 + .../LogisticRegressionSuite.scala | 2 ++ .../ml/classification/OneVsRestSuite.scala | 2 ++ .../ProbabilisticClassifierSuite.scala | 2 ++ .../org/apache/spark/ml/feature/LSHTest.scala | 2 ++ .../LogisticRegressionSuite.scala | 1 + .../spark/mllib/clustering/LDASuite.scala | 1 + .../spark/mllib/tree/EnsembleTestHelper.scala | 2 ++ .../spark/scheduler/cluster/mesos/Utils.scala | 7 ++-- .../sql/streaming/JavaOutputModeSuite.java | 5 +-- .../analysis/AnalysisErrorSuite.scala | 2 ++ .../expressions/LiteralGenerator.scala | 1 + .../spark/sql/connector/InMemoryTable.scala | 2 ++ .../sort/RecordBinaryComparatorSuite.java | 30 ++++++++-------- .../org/apache/spark/sql/DatasetSuite.scala | 1 + .../spark/sql/IntegratedUDFTestUtils.scala | 2 ++ .../HashedRelationMetricsBenchmark.scala | 4 ++- .../execution/benchmark/JoinBenchmark.scala | 2 ++ .../benchmark/WideSchemaBenchmark.scala | 2 ++ .../CompressionSchemeBenchmark.scala | 1 + .../internal/ExecutorSideSQLConfSuite.scala | 2 ++ .../spark/sql/hive/HiveSparkSubmitSuite.scala | 5 +-- .../execution/ScriptTransformationSuite.scala | 1 + ...iveHadoopDelegationTokenManagerSuite.scala | 1 + .../spark/streaming/MasterFailureTest.scala | 1 + .../streaming/util/WriteAheadLogSuite.scala | 1 + 41 files changed, 102 insertions(+), 47 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 457805feeac45..fb67d7220a0b4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -28,6 +28,7 @@ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,7 +123,7 @@ private void insertFile(String filename) throws IOException { private void insertFile(String filename, byte[] block) throws IOException { OutputStream dataStream = null; File file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename); - assert(!file.exists()) : "this test file has been already generated"; + Assert.assertFalse("this test file has been already generated", file.exists()); try { dataStream = new FileOutputStream( ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename)); diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 8173a8e545ebb..3f309819065be 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import org.scalatest.Assertions._ import org.scalatest.Matchers import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.{Millis, Span} diff --git a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala index 5dbef88e73a9e..78f1246295bf8 100644 --- a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.scalatest.Assertions._ + import org.apache.spark.benchmark.Benchmark import org.apache.spark.benchmark.BenchmarkBase import org.apache.spark.scheduler.CompressedMapStatus diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 022fcbb25b0af..9629f5ab1a3dd 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -26,6 +26,7 @@ import scala.util.Try import org.apache.commons.io.output.TeeOutputStream import org.apache.commons.lang3.SystemUtils +import org.scalatest.Assertions._ import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 2115ee8b1b723..7272a98c9770b 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -33,6 +33,7 @@ import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer +import org.scalatest.Assertions._ import org.scalatest.PrivateMethodTester import org.scalatest.concurrent.Eventually import org.scalatestplus.mockito.MockitoSugar diff --git a/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala b/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala index 5e8da3e205ab0..7c65f3b126e3d 100644 --- a/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala +++ b/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala @@ -20,6 +20,7 @@ package org.apache.spark.rpc import scala.collection.mutable.ArrayBuffer import org.scalactic.TripleEquals +import org.scalatest.Assertions._ class TestRpcEndpoint extends ThreadSafeRpcEndpoint with TripleEquals { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 4f737c9499ad6..dff8975a4fe49 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -26,6 +26,7 @@ import scala.concurrent.duration.{Duration, SECONDS} import scala.reflect.ClassTag import org.scalactic.TripleEquals +import org.scalatest.Assertions import org.scalatest.Assertions.AssertionsHelper import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -463,7 +464,7 @@ class MockRDD( override def toString: String = "MockRDD " + id } -object MockRDD extends AssertionsHelper with TripleEquals { +object MockRDD extends AssertionsHelper with TripleEquals with Assertions { /** * make sure all the shuffle dependencies have a consistent number of output partitions * (mostly to make sure the test setup makes sense, not that Spark itself would get this wrong) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 8439be955c738..406bd9244870e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.MoreExecutors import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{spy, times, verify} +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 89df5de97c444..34bcae8abd512 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileAlreadyExistsException import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock +import org.scalatest.Assertions._ import org.apache.spark._ import org.apache.spark.internal.Logging @@ -128,7 +129,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex def removeExecutor(execId: String): Unit = { executors -= execId val host = executorIdToHost.get(execId) - assert(host != None) + assert(host.isDefined) val hostId = host.get val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala index d8657ecdff676..3d70ff1fed29f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala @@ -21,6 +21,7 @@ import java.util.{Map => JMap} import java.util.concurrent.atomic.AtomicBoolean import com.google.common.collect.ImmutableMap +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfterEach import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala index 06c2ceb68bd79..f14ec175232be 100644 --- a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import org.apache.hadoop.fs.Path +import org.scalatest.Assertions._ import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite} import org.apache.spark.rdd.RDD diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 6c745987b4c23..6e97c37af7df0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -18,10 +18,9 @@ package org.apache.spark.sql.kafka010 import java.io.{File, IOException} -import java.lang.{Integer => JInt} import java.net.{InetAddress, InetSocketAddress} import java.nio.charset.StandardCharsets -import java.util.{Collections, Map => JMap, Properties, UUID} +import java.util.{Collections, Properties, UUID} import java.util.concurrent.TimeUnit import javax.security.auth.login.Configuration @@ -41,13 +40,12 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs -import org.apache.kafka.common.header.Header -import org.apache.kafka.common.header.internals.RecordHeader import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, SASL_PLAINTEXT} import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.zookeeper.server.auth.SASLAuthenticationProvider +import org.scalatest.Assertions._ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java index 03becd73d1a06..7af0abe0e8d90 100644 --- a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java +++ b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java @@ -18,12 +18,14 @@ package org.apache.spark.streaming.kinesis; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.junit.Assert; +import org.junit.Test; + import org.apache.spark.streaming.kinesis.KinesisInitialPositions.TrimHorizon; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.Seconds; -import org.junit.Test; public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext { /** @@ -49,13 +51,14 @@ public void testJavaKinesisDStreamBuilder() { .checkpointInterval(checkpointInterval) .storageLevel(storageLevel) .build(); - assert(kinesisDStream.streamName() == streamName); - assert(kinesisDStream.endpointUrl() == endpointUrl); - assert(kinesisDStream.regionName() == region); - assert(kinesisDStream.initialPosition().getPosition() == initialPosition.getPosition()); - assert(kinesisDStream.checkpointAppName() == appName); - assert(kinesisDStream.checkpointInterval() == checkpointInterval); - assert(kinesisDStream._storageLevel() == storageLevel); + Assert.assertEquals(streamName, kinesisDStream.streamName()); + Assert.assertEquals(endpointUrl, kinesisDStream.endpointUrl()); + Assert.assertEquals(region, kinesisDStream.regionName()); + Assert.assertEquals(initialPosition.getPosition(), + kinesisDStream.initialPosition().getPosition()); + Assert.assertEquals(appName, kinesisDStream.checkpointAppName()); + Assert.assertEquals(checkpointInterval, kinesisDStream.checkpointInterval()); + Assert.assertEquals(storageLevel, kinesisDStream._storageLevel()); ssc.stop(); } @@ -83,13 +86,14 @@ public void testJavaKinesisDStreamBuilderOldApi() { .checkpointInterval(checkpointInterval) .storageLevel(storageLevel) .build(); - assert(kinesisDStream.streamName() == streamName); - assert(kinesisDStream.endpointUrl() == endpointUrl); - assert(kinesisDStream.regionName() == region); - assert(kinesisDStream.initialPosition().getPosition() == InitialPositionInStream.LATEST); - assert(kinesisDStream.checkpointAppName() == appName); - assert(kinesisDStream.checkpointInterval() == checkpointInterval); - assert(kinesisDStream._storageLevel() == storageLevel); + Assert.assertEquals(streamName, kinesisDStream.streamName()); + Assert.assertEquals(endpointUrl, kinesisDStream.endpointUrl()); + Assert.assertEquals(region, kinesisDStream.regionName()); + Assert.assertEquals(InitialPositionInStream.LATEST, + kinesisDStream.initialPosition().getPosition()); + Assert.assertEquals(appName, kinesisDStream.checkpointAppName()); + Assert.assertEquals(checkpointInterval, kinesisDStream.checkpointInterval()); + Assert.assertEquals(storageLevel, kinesisDStream._storageLevel()); ssc.stop(); } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala index e0c65e6940f66..e3471759b3a70 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx.util import org.apache.hadoop.fs.Path +import org.scalatest.Assertions import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.graphx.{Edge, Graph, LocalSparkContext} @@ -88,7 +89,7 @@ class PeriodicGraphCheckpointerSuite extends SparkFunSuite with LocalSparkContex } } -private object PeriodicGraphCheckpointerSuite { +private object PeriodicGraphCheckpointerSuite extends Assertions { private val defaultStorageLevel = StorageLevel.MEMORY_ONLY_SER case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) diff --git a/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java b/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java index 830f668fe07b8..9037f6b854724 100644 --- a/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java @@ -23,6 +23,7 @@ import org.apache.commons.math3.distribution.NormalDistribution; import org.apache.spark.sql.Encoders; +import org.junit.Assert; import org.junit.Test; import org.apache.spark.SharedSparkSession; @@ -60,7 +61,7 @@ public void testKSTestCDF() { .test(dataset, "sample", stdNormalCDF).head(); double pValue1 = results.getDouble(0); // Cannot reject null hypothesis - assert(pValue1 > pThreshold); + Assert.assertTrue(pValue1 > pThreshold); } @Test @@ -72,6 +73,6 @@ public void testKSTestNamedDistribution() { .test(dataset, "sample", "norm", 0.0, 1.0).head(); double pValue1 = results.getDouble(0); // Cannot reject null hypothesis - assert(pValue1 > pThreshold); + Assert.assertTrue(pValue1 > pThreshold); } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index cb9b8f9b6b472..dc38f17d296f2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.classification import scala.util.Random import breeze.linalg.{DenseVector => BDV} +import org.scalatest.Assertions._ import org.apache.spark.ml.classification.LinearSVCSuite._ import org.apache.spark.ml.feature.{Instance, LabeledPoint} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 07116606dfb52..60c9cce6a4879 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -21,6 +21,8 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ +import org.scalatest.Assertions._ + import org.apache.spark.SparkException import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index b6e8c927403ad..adffd83ab1bd1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.classification +import org.scalatest.Assertions._ + import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.LabeledPoint diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index 1c8c9829f18d1..87a8b345a65a3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.classification +import org.scalatest.Assertions._ + import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala index db4f56ed60d32..76a4acd798e34 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.feature +import org.scalatest.Assertions._ + import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.util.{MLTestingUtils, SchemaUtils} import org.apache.spark.sql.Dataset diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index d4e9da3c6263e..d96a4da46a630 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ +import org.scalatest.Assertions._ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 8906e52faebe5..321df05e272db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import java.util.{ArrayList => JArrayList} import breeze.linalg.{argmax, argtopk, max, DenseMatrix => BDM} +import org.scalatest.Assertions import org.apache.spark.SparkFunSuite import org.apache.spark.graphx.Edge diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala index e04d7b7c327a8..5458a43b4f2c6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree import scala.collection.mutable +import org.scalatest.Assertions._ + import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala index 65e595e3cf2bf..5a4bf1dd2d409 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala @@ -28,6 +28,7 @@ import org.apache.mesos.protobuf.ByteString import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{times, verify} +import org.scalatest.Assertions._ import org.apache.spark.deploy.mesos.config.MesosSecretConfig @@ -161,12 +162,14 @@ object Utils { val variableOne = envVars.filter(_.getName == "USER").head assert(variableOne.getSecret.isInitialized) assert(variableOne.getSecret.getType == Secret.Type.VALUE) - assert(variableOne.getSecret.getValue.getData == ByteString.copyFrom("user".getBytes)) + assert(variableOne.getSecret.getValue.getData == + ByteString.copyFrom("user".getBytes)) assert(variableOne.getType == Environment.Variable.Type.SECRET) val variableTwo = envVars.filter(_.getName == "PASSWORD").head assert(variableTwo.getSecret.isInitialized) assert(variableTwo.getSecret.getType == Secret.Type.VALUE) - assert(variableTwo.getSecret.getValue.getData == ByteString.copyFrom("password".getBytes)) + assert(variableTwo.getSecret.getValue.getData == + ByteString.copyFrom("password".getBytes)) assert(variableTwo.getType == Environment.Variable.Type.SECRET) } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java index d8845e0c838ff..ca2b18b8eed49 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java @@ -19,6 +19,7 @@ import java.util.Locale; +import org.junit.Assert; import org.junit.Test; public class JavaOutputModeSuite { @@ -26,8 +27,8 @@ public class JavaOutputModeSuite { @Test public void testOutputModes() { OutputMode o1 = OutputMode.Append(); - assert(o1.toString().toLowerCase(Locale.ROOT).contains("append")); + Assert.assertTrue(o1.toString().toLowerCase(Locale.ROOT).contains("append")); OutputMode o2 = OutputMode.Complete(); - assert (o2.toString().toLowerCase(Locale.ROOT).contains("complete")); + Assert.assertTrue(o2.toString().toLowerCase(Locale.ROOT).contains("complete")); } } 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 3dabbca9deeee..e0fa1f2ecb88e 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.scalatest.Assertions._ + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index a89937068a87d..d92eb01b69bf0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -22,6 +22,7 @@ import java.time.{Duration, Instant, LocalDate} import java.util.concurrent.TimeUnit import org.scalacheck.{Arbitrary, Gen} +import org.scalatest.Assertions._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 414f9d5834868..e7b2dd2ebc9da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -22,6 +22,8 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable +import org.scalatest.Assertions._ + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.{IdentityTransform, Transform} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 68f984ae0c1e3..4b23615275871 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -82,14 +82,14 @@ private void insertRow(UnsafeRow row) { int recordLength = row.getSizeInBytes(); Object baseObject = dataPage.getBaseObject(); - assert(pageCursor + recordLength <= dataPage.getBaseOffset() + dataPage.size()); + Assert.assertTrue(pageCursor + recordLength <= dataPage.getBaseOffset() + dataPage.size()); long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, pageCursor); UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength); pageCursor += uaoSize; Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, recordLength); pageCursor += recordLength; - assert(pos < 2); + Assert.assertTrue(pos < 2); array.set(pos, recordAddress); pos++; } @@ -142,8 +142,8 @@ public void testBinaryComparatorForSingleColumnRow() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) < 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -167,8 +167,8 @@ public void testBinaryComparatorForMultipleColumnRow() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) < 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -194,8 +194,8 @@ public void testBinaryComparatorForArrayColumn() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -227,8 +227,8 @@ public void testBinaryComparatorForMixedColumns() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -253,8 +253,8 @@ public void testBinaryComparatorForNullColumns() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -274,7 +274,7 @@ public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws insertRow(row1); insertRow(row2); - assert(compare(0, 1) > 0); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -294,7 +294,7 @@ public void testBinaryComparatorWhenSubtractionCanOverflowLongValue() throws Exc insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -320,7 +320,7 @@ public void testBinaryComparatorWhenOnlyTheLastColumnDiffers() throws Exception insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test 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 9c50e374f74de..fc6f087efaefd 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.sql.{Date, Timestamp} +import org.scalatest.Assertions._ import org.scalatest.exceptions.TestFailedException import org.scalatest.prop.TableDrivenPropertyChecks._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 3f6fe6eb62020..d39019bcda9a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -22,6 +22,8 @@ import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ import scala.util.Try +import org.scalatest.Assertions._ + import org.apache.spark.TestUtils import org.apache.spark.api.python.{PythonBroadcast, PythonEvalType, PythonFunction, PythonUtils} import org.apache.spark.broadcast.Broadcast diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala index ebe278bff7d86..f3647b3bb2631 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.benchmark +import org.scalatest.Assertions._ + import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED @@ -71,7 +73,7 @@ object HashedRelationMetricsBenchmark extends SqlBasedBenchmark { thread.start() thread } - threads.map(_.join()) + threads.foreach(_.join()) map.free() } benchmark.run() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala index ad81711a13947..f4786368bd9e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.benchmark +import org.scalatest.Assertions._ + import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index f4642e7d353e6..683d398faeeab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import org.scalatest.Assertions._ + import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index 8ea20f28a37b2..fcb18392235c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.math3.distribution.LogNormalDistribution +import org.scalatest.Assertions._ import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.catalyst.expressions.GenericInternalRow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index c0238069afcc4..42213b9a81882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import org.scalatest.Assertions._ + import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 3399a2d925925..8056deefa07c9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -24,6 +24,7 @@ import scala.util.Properties import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.Assertions._ import org.apache.spark._ import org.apache.spark.internal.Logging @@ -806,14 +807,14 @@ object SPARK_18360 { // Hive will use the value of `hive.metastore.warehouse.dir` to generate default table // location for tables in default database. assert(rawTable.storage.locationUri.map( - CatalogUtils.URIToString(_)).get.contains(newWarehousePath)) + CatalogUtils.URIToString).get.contains(newWarehousePath)) hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = false, purge = false) spark.sharedState.externalCatalog.createTable(tableMeta, ignoreIfExists = false) val readBack = spark.sharedState.externalCatalog.getTable("default", "test_tbl") // Spark SQL will use the location of default database to generate default table // location for tables in default database. - assert(readBack.storage.locationUri.map(CatalogUtils.URIToString(_)) + assert(readBack.storage.locationUri.map(CatalogUtils.URIToString) .get.contains(defaultDbLocation)) } finally { hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = true, purge = false) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index ed3b376f6eda1..80a50c18bcb93 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfterEach import org.scalatest.exceptions.TestFailedException diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala index ce40cf51746b2..97eab4f3f4f77 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.security import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration +import org.scalatest.Assertions._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.security.HadoopDelegationTokenManager diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 2615e7c3bc509..d0a5ababc7cac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -30,6 +30,7 @@ import scala.util.Random import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.scalatest.Assertions._ import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8e9481b5a99d3..bb60d6fa7bf78 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -34,6 +34,7 @@ import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito.{times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, PrivateMethodTester} +import org.scalatest.Assertions._ import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatestplus.mockito.MockitoSugar From 06e203b85682b63ee250b96520558fc79aae0a17 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 20 Nov 2019 18:19:30 -0600 Subject: [PATCH 57/83] [SPARK-29911][SQL] Uncache cached tables when session closed ### What changes were proposed in this pull request? The local temporary view is session-scoped. Its lifetime is the lifetime of the session that created it. But now cache data is cross-session. Its lifetime is the lifetime of the Spark application. That's will cause the memory leak if cache a local temporary view in memory when the session closed. In this PR, we uncache the cached data of local temporary view when session closed. This PR doesn't impact the cached data of global temp view and persisted view. How to reproduce: 1. create a local temporary view v1 2. cache it in memory 3. close session without drop table v1. The application will hold the memory forever. In a long running thrift server scenario. It's worse. ```shell 0: jdbc:hive2://localhost:10000> CACHE TABLE testCacheTable AS SELECT 1; CACHE TABLE testCacheTable AS SELECT 1; +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (1.498 seconds) 0: jdbc:hive2://localhost:10000> !close !close Closing: 0: jdbc:hive2://localhost:10000 0: jdbc:hive2://localhost:10000 (closed)> !connect 'jdbc:hive2://localhost:10000' !connect 'jdbc:hive2://localhost:10000' Connecting to jdbc:hive2://localhost:10000 Enter username for jdbc:hive2://localhost:10000: lajin Enter password for jdbc:hive2://localhost:10000: *** Connected to: Spark SQL (version 3.0.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ 1: jdbc:hive2://localhost:10000> select * from testCacheTable; select * from testCacheTable; Error: Error running query: org.apache.spark.sql.AnalysisException: Table or view not found: testCacheTable; line 1 pos 14; 'Project [*] +- 'UnresolvedRelation [testCacheTable] (state=,code=0) ``` Screen Shot 2019-11-15 at 2 03 49 PM ### Why are the changes needed? Resolve memory leak for thrift server ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test in UI storage tab And add an UT Closes #26543 from LantaoJin/SPARK-29911. Authored-by: LantaoJin Signed-off-by: Sean Owen --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 ++++ .../thriftserver/SparkSQLSessionManager.scala | 2 ++ .../ThriftServerQueryTestSuite.scala | 23 +++++++++++++++++++ 3 files changed, 29 insertions(+) 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 e72352714a697..e77f3c70f0bdf 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 @@ -575,6 +575,10 @@ class SessionCatalog( tempViews.get(formatTableName(name)) } + def getTempViewNames(): Seq[String] = synchronized { + tempViews.keySet.toSeq + } + /** * Return a global temporary view exactly as it was stored. */ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index c4248bfde38cc..41b324d70c315 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -75,6 +75,8 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: override def closeSession(sessionHandle: SessionHandle): Unit = { HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) + val ctx = sparkSqlOperationManager.sessionToContexts.getOrDefault(sessionHandle, sqlContext) + ctx.sparkSession.sessionState.catalog.getTempViewNames().foreach(ctx.uncacheTable) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) 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 index 0c56cf9ef5d50..dc1609d3326cd 100644 --- 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 @@ -269,6 +269,29 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } + test("SPARK-29911: Uncache cached tables when session closed") { + val cacheManager = spark.sharedState.cacheManager + val globalTempDB = spark.sharedState.globalTempViewManager.database + withJdbcStatement { statement => + statement.execute("CACHE TABLE tempTbl AS SELECT 1") + } + // the cached data of local temporary view should be uncached + assert(cacheManager.isEmpty) + try { + withJdbcStatement { statement => + statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") + statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") + } + // the cached data of global temporary view shouldn't be uncached + assert(!cacheManager.isEmpty) + } finally { + withJdbcStatement { statement => + statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") + } + assert(cacheManager.isEmpty) + } + } + /** ThriftServer wraps the root exception, so it needs to be extracted. */ override def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { super.handleExceptions { From e6b157cf704544a6217a9f9d191f542518581040 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 21 Nov 2019 09:59:31 +0900 Subject: [PATCH 58/83] [SPARK-29978][SQL][TESTS] Check `json_tuple` does not truncate results ### What changes were proposed in this pull request? I propose to add a test from the commit https://github.com/apache/spark/commit/a9365221133caadffbbbbce1aae1ace799a588a3 for 2.4. I extended the test by a few more lengths of requested field to cover more code branches in Jackson Core. In particular, [the optimization](https://github.com/apache/spark/blob/5eb8973f871fef557fb4ca3f494406ed676a431a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala#L473-L476) calls Jackson's method https://github.com/FasterXML/jackson-core/blob/42b8b566845e8f8d77537f51187a439029ed9bff/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java#L742-L746 where the internal buffer size is **8000**. In this way: - 2000 to check 2000+2000+2000 < 8000 - 2800 from the 2.4 commit. It covers the specific case: https://github.com/FasterXML/jackson-core/blob/42b8b566845e8f8d77537f51187a439029ed9bff/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java#L746 - 8000-1, 8000, 8000+1 are sizes around the size of the internal buffer - 65535 to test an outstanding large field. ### Why are the changes needed? To be sure that the current implementation and future versions of Spark don't have the bug fixed in 2.4. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `JsonFunctionsSuite`. Closes #26613 from MaxGekk/json_tuple-test. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) 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 f3483151d26da..4059079d3f288 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 @@ -644,4 +644,15 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { to_json(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS"))) checkAnswer(df, Row(s"""{"t":"$s"}""")) } + + test("json_tuple - do not truncate results") { + Seq(2000, 2800, 8000 - 1, 8000, 8000 + 1, 65535).foreach { len => + val str = Array.tabulate(len)(_ => "a").mkString + val json_tuple_result = Seq(s"""{"test":"$str"}""").toDF("json") + .withColumn("result", json_tuple('json, "test")) + .select('result) + .as[String].head.length + assert(json_tuple_result === len) + } + } } From 7a706703455d218a874cafe114ac8d720700a802 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 21 Nov 2019 10:28:32 +0900 Subject: [PATCH 59/83] [SPARK-29961][SQL] Implement builtin function - typeof ### What changes were proposed in this pull request? Add typeof function for Spark to get the underlying type of value. ```sql -- !query 0 select typeof(1) -- !query 0 schema struct -- !query 0 output int -- !query 1 select typeof(1.2) -- !query 1 schema struct -- !query 1 output decimal(2,1) -- !query 2 select typeof(array(1, 2)) -- !query 2 schema struct -- !query 2 output array -- !query 3 select typeof(a) from (values (1), (2), (3.1)) t(a) -- !query 3 schema struct -- !query 3 output decimal(11,1) decimal(11,1) decimal(11,1) ``` ##### presto ```sql presto> select typeof(array[1]); _col0 ---------------- array(integer) (1 row) ``` ##### PostgreSQL ```sql postgres=# select pg_typeof(a) from (values (1), (2), (3.0)) t(a); pg_typeof ----------- numeric numeric numeric (3 rows) ``` ##### impala https://issues.apache.org/jira/browse/IMPALA-1597 ### Why are the changes needed? a function which is better we have to help us debug, test, develop ... ### Does this PR introduce any user-facing change? add a new function ### How was this patch tested? add ut and example Closes #26599 from yaooqinn/SPARK-29961. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../spark/ml/linalg/VectorUDTSuite.scala | 14 ++++- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/misc.scala | 21 +++++++ .../sql-tests/inputs/misc-functions.sql | 10 ++++ .../sql-tests/results/misc-functions.sql.out | 58 +++++++++++++++++++ .../spark/sql/UserDefinedTypeSuite.scala | 10 ++++ 6 files changed, 112 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 67c64f762b25e..3bb47755b2703 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.ml.linalg -import org.apache.spark.SparkFunSuite +import java.util.Arrays + import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.JavaTypeInference +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class VectorUDTSuite extends SparkFunSuite { +class VectorUDTSuite extends QueryTest with SharedSparkSession { test("preloaded VectorUDT") { val dv1 = Vectors.dense(Array.empty[Double]) @@ -44,4 +47,11 @@ class VectorUDTSuite extends SparkFunSuite { assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) === Seq(new VectorUDT, DoubleType)) } + + test("typeof Vector") { + val data = Arrays.asList(Row(Vectors.dense(1.0, 2.0))) + val schema = new StructType().add("v", new VectorUDT) + checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(v)"), + Seq(Row("struct,values:array>"))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index cb18aa1a9479b..c0e0330ff1e14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -491,6 +491,7 @@ object FunctionRegistry { expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method"), expression[Version]("version"), + expression[TypeOf]("typeof"), // grouping sets expression[Cube]("cube"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index b8c23a1f08912..def81f8dfb72b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -177,3 +177,24 @@ case class Version() extends LeafExpression with CodegenFallback { UTF8String.fromString(SPARK_VERSION_SHORT + " " + SPARK_REVISION) } } + +@ExpressionDescription( + usage = """_FUNC_(expr) - Return DDL-formatted type string for the data type of the input.""", + examples = """ + Examples: + > SELECT _FUNC_(1); + int + > SELECT _FUNC_(array(1)); + array + """, + since = "3.0.0") +case class TypeOf(child: Expression) extends UnaryExpression { + override def nullable: Boolean = false + override def foldable: Boolean = true + override def dataType: DataType = StringType + override def eval(input: InternalRow): Any = UTF8String.fromString(child.dataType.catalogString) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, _ => s"""UTF8String.fromString(${child.dataType.catalogString})""") + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql new file mode 100644 index 0000000000000..95f71925e9294 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,10 @@ +-- test for misc functions + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out new file mode 100644 index 0000000000000..cd0818a5189b5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,58 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +select typeof(null) +-- !query 0 schema +struct +-- !query 0 output +null + + +-- !query 1 +select typeof(true) +-- !query 1 schema +struct +-- !query 1 output +boolean + + +-- !query 2 +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query 2 schema +struct +-- !query 2 output +tinyint smallint int bigint + + +-- !query 3 +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query 3 schema +struct +-- !query 3 output +float double decimal(2,1) + + +-- !query 4 +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query 4 schema +struct +-- !query 4 output +date timestamp interval + + +-- !query 5 +select typeof(x'ABCD'), typeof('SPARK') +-- !query 5 schema +struct +-- !query 5 output +binary string + + +-- !query 6 +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query 6 schema +struct +-- !query 6 output +array map struct 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 2b2fedd3ca218..ffc2018d2132d 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.util.Arrays + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} @@ -277,4 +279,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with Parque val udt = new TestUDT.MyDenseVectorUDT() assert(!Cast.canUpCast(udt, StringType)) } + + test("typeof user defined type") { + val schema = new StructType().add("a", new TestUDT.MyDenseVectorUDT()) + val data = Arrays.asList( + RowFactory.create(new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)))) + checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(a)"), + Seq(Row("array"))) + } } From 74cb1ffd686d67188a4540c74c6111affd6cce90 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 21 Nov 2019 10:54:01 +0900 Subject: [PATCH 60/83] [SPARK-22340][PYTHON][FOLLOW-UP] Add a better message and improve documentation for pinned thread mode ### What changes were proposed in this pull request? This PR proposes to show different warning message when the pinned thread mode is enabled: When enabled: > PYSPARK_PIN_THREAD feature is enabled. However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. > To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. When disabled: > Currently, 'setLocalProperty' (set to local properties) with multiple threads does not properly work. > Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. > To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. > To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. ### Why are the changes needed? Currently, it shows the same warning message regardless of PYSPARK_PIN_THREAD being set. In the warning message it says "you can set PYSPARK_PIN_THREAD to true ..." which is confusing. ### Does this PR introduce any user-facing change? Documentation and warning message as shown above. ### How was this patch tested? Manually tested. ```bash $ PYSPARK_PIN_THREAD=true ./bin/pyspark ``` ```python sc.setJobGroup("a", "b") ``` ``` .../pyspark/util.py:141: UserWarning: PYSPARK_PIN_THREAD feature is enabled. However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. warnings.warn(msg, UserWarning) ``` ```bash $ ./bin/pyspark ``` ```python sc.setJobGroup("a", "b") ``` ``` .../pyspark/util.py:141: UserWarning: Currently, 'setJobGroup' (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. warnings.warn(msg, UserWarning) ``` Closes #26588 from HyukjinKwon/SPARK-22340. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/context.py | 88 +++++++++++++-------------------------- python/pyspark/util.py | 29 +++++++++++++ 2 files changed, 59 insertions(+), 58 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a6aa3a65568e9..6cc343e3e495c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -40,6 +40,7 @@ from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +from pyspark.util import _warn_pin_thread if sys.version > '3': xrange = range @@ -1008,30 +1009,20 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead. - .. note:: Currently, setting a group ID (set to local properties) with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread - can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + .. note:: Currently, setting a group ID (set to local properties) with multiple threads + does not properly work. Internally threads on PVM and JVM are not synced, and JVM + thread can be reused for multiple threads on PVM, which fails to isolate local + properties for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a group ID (set to local properties) with a thread does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setJobGroup") self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): @@ -1039,29 +1030,20 @@ def setLocalProperty(self, key, value): Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool. - .. note:: Currently, setting a local property with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread + .. note:: Currently, setting a local property with multiple threads does not properly work. + Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a local property with a thread does not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setLocalProperty") self._jsc.setLocalProperty(key, value) def getLocalProperty(self, key): @@ -1075,30 +1057,20 @@ def setJobDescription(self, value): """ Set a human readable description of the current job. - .. note:: Currently, setting a job description (set to local properties) with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread - can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + .. note:: Currently, setting a job description (set to local properties) with multiple + threads does not properly work. Internally threads on PVM and JVM are not synced, + and JVM thread can be reused for multiple threads on PVM, which fails to isolate + local properties for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a job description (set to local properties) with a thread does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setJobDescription") self._jsc.setJobDescription(value) def sparkUser(self): diff --git a/python/pyspark/util.py b/python/pyspark/util.py index ad4cf7c9b1c8b..93137560de25e 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -19,6 +19,8 @@ import re import sys import traceback +import os +import warnings import inspect from py4j.protocol import Py4JJavaError @@ -112,6 +114,33 @@ def wrapper(*args, **kwargs): return wrapper +def _warn_pin_thread(name): + if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": + msg = ( + "PYSPARK_PIN_THREAD feature is enabled. " + "However, note that it cannot inherit the local properties from the parent thread " + "although it isolates each thread on PVM and JVM with its own local properties. " + "\n" + "To work around this, you should manually copy and set the local properties from " + "the parent thread to the child thread when you create another thread.") + else: + msg = ( + "Currently, '%s' (set to local properties) with multiple threads does " + "not properly work. " + "\n" + "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " + "for multiple threads on PVM, which fails to isolate local properties for each " + "thread on PVM. " + "\n" + "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " + "However, note that it cannot inherit the local properties from the parent thread " + "although it isolates each thread on PVM and JVM with its own local properties. " + "\n" + "To work around this, you should manually copy and set the local properties from " + "the parent thread to the child thread when you create another thread." % name) + warnings.warn(msg, UserWarning) + + def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version): print(""" ________________________________________________________________________________________________ From d555f8fcc964e6e81f2d849de62d06877dca70c5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 20 Nov 2019 21:02:22 -0800 Subject: [PATCH 61/83] [SPARK-29961][SQL][FOLLOWUP] Remove useless test for VectorUDT ### What changes were proposed in this pull request? A follow-up to rm useless test in VectorUDTSuite ### Why are the changes needed? rm useless test, which is already covered. ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #26620 from yaooqinn/SPARK-29961-f. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../apache/spark/ml/linalg/VectorUDTSuite.scala | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 3bb47755b2703..67c64f762b25e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.ml.linalg -import java.util.Arrays - +import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.JavaTypeInference -import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class VectorUDTSuite extends QueryTest with SharedSparkSession { +class VectorUDTSuite extends SparkFunSuite { test("preloaded VectorUDT") { val dv1 = Vectors.dense(Array.empty[Double]) @@ -47,11 +44,4 @@ class VectorUDTSuite extends QueryTest with SharedSparkSession { assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) === Seq(new VectorUDT, DoubleType)) } - - test("typeof Vector") { - val data = Arrays.asList(Row(Vectors.dense(1.0, 2.0))) - val schema = new StructType().add("v", new VectorUDT) - checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(v)"), - Seq(Row("struct,values:array>"))) - } } From affaefe1f3e623e6ae2042648ab78ee7b89d1ed6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 21 Nov 2019 15:43:57 +0900 Subject: [PATCH 62/83] [MINOR][INFRA] Add `io` and `net` to GitHub Action Cache ### What changes were proposed in this pull request? This PR aims to cache `~/.m2/repository/net` and `~/.m2/repository/io` to reduce the flakiness. ### Why are the changes needed? This will stabilize GitHub Action more before adding `hive-1.2` and `hive-2.3` combination. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? After the GitHub Action on this PR passes, check the log. Closes #26621 from dongjoon-hyun/SPARK-GHA-CACHE. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2cd48153b190c..5298492d219f8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -36,6 +36,18 @@ jobs: key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org-${{ hashFiles('**/pom.xml') }} restore-keys: | ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org- + - uses: actions/cache@v1 + with: + path: ~/.m2/repository/net + key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net- + - uses: actions/cache@v1 + with: + path: ~/.m2/repository/io + key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io- - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: From 85c004d5b0303435dc207e139cdc51f0f2d3e160 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Thu, 21 Nov 2019 16:13:42 +0900 Subject: [PATCH 63/83] [SPARK-29885][PYTHON][CORE] Improve the exception message when reading the daemon port ### What changes were proposed in this pull request? In production environment, my PySpark application occurs an exception and it's message as below: ``` 19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) org.apache.spark.SparkException: No port number in pyspark.daemon's stdout at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:204) at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122) at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95) at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117) at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:121) at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` At first, I think a physical node has many ports are occupied by a large number of processes. But I found the total number of ports in use is only 671. ``` [yarnr1115 ~]$ netstat -a | wc -l 671 671 ``` I checked the code of PythonWorkerFactory in line 204 and found: ``` daemon = pb.start() val in = new DataInputStream(daemon.getInputStream) try { daemonPort = in.readInt() } catch { case _: EOFException => throw new SparkException(s"No port number in $daemonModule's stdout") } ``` I added some code here: ``` logError("Meet EOFException, daemon is alive: ${daemon.isAlive()}") logError("Exit value: ${daemon.exitValue()}") ``` Then I recurrent the exception and it's message as below: ``` 19/10/28 16:15:03 ERROR PythonWorkerFactory: Meet EOFException, daemon is alive: false 19/10/28 16:15:03 ERROR PythonWorkerFactory: Exit value: 139 19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) org.apache.spark.SparkException: No port number in pyspark.daemon's stdout at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:206) at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122) at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95) at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117) at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:121) at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` I think the exception message has caused me a lot of confusion. This PR will add meaningful log for exception information. ### Why are the changes needed? In order to clarify the exception and try three times default. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Exists UT. Closes #26510 from beliefer/improve-except-message. Authored-by: gengjiaan Signed-off-by: HyukjinKwon --- .../org/apache/spark/api/python/PythonWorkerFactory.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 1926a5268227c..df236ba8926c1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -212,8 +212,13 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { daemonPort = in.readInt() } catch { + case _: EOFException if daemon.isAlive => + throw new SparkException("EOFException occurred while reading the port number " + + s"from $daemonModule's stdout") case _: EOFException => - throw new SparkException(s"No port number in $daemonModule's stdout") + throw new SparkException( + s"EOFException occurred while reading the port number from $daemonModule's" + + s" stdout and terminated with code: ${daemon.exitValue}.") } // test that the returned port number is within a valid range. From 297cbab98e68fed08cd790c24020b3df416e1e03 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 21 Nov 2019 18:22:05 +0800 Subject: [PATCH 64/83] [SPARK-29942][ML] Impl Complement Naive Bayes Classifier ### What changes were proposed in this pull request? Impl Complement Naive Bayes Classifier as a `modelType` option in `NaiveBayes` ### Why are the changes needed? 1, it is a better choice for text classification: it is said in [scikit-learn](https://scikit-learn.org/stable/modules/naive_bayes.html#complement-naive-bayes) that 'CNB regularly outperforms MNB (often by a considerable margin) on text classification tasks.' 2, CNB is highly similar to existing MNB, only a small part of existing MNB need to be changed, so it is a easy win to support CNB. ### Does this PR introduce any user-facing change? yes, a new `modelType` is supported ### How was this patch tested? added testsuites Closes #26575 from zhengruifeng/cnb. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- docs/ml-classification-regression.md | 7 +- .../spark/ml/classification/NaiveBayes.scala | 96 +++++++++++++++---- .../ml/classification/NaiveBayesSuite.scala | 77 +++++++++++++-- python/pyspark/ml/classification.py | 13 +++ 4 files changed, 162 insertions(+), 31 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index d8c7d8a729624..05c688960f04c 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -479,16 +479,17 @@ For prediction, it applies Bayes' theorem to compute the conditional probability of each label given an observation. MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +[Complement naive Bayes](https://people.csail.mit.edu/jrennie/papers/icml03-nb.pdf), [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). *Input data*: -These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These Multinomial, Complement and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. -A feature's value is the frequency of the term (in multinomial Naive Bayes) or +A feature's value is the frequency of the term (in Multinomial or Complement Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter -"multinomial" or "bernoulli" with "multinomial" as the default. +"multinomial", "complement", "bernoulli" or "gaussian", with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 8062870794418..52a0f4d9b9828 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -53,13 +53,13 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial", "bernoulli", "gaussian". + * Supported options: "multinomial", "complement", "bernoulli", "gaussian". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + - " and gaussian.", + "which is a string (case-sensitive). Supported options: multinomial (default), complement, " + + "bernoulli and gaussian.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -78,6 +78,11 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * (see * here). * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it supports Complement NB which is an adaptation of the Multinomial NB. Specifically, + * Complement NB uses statistics from the complement of each class to compute the model's coefficients + * The inventors of Complement NB show empirically that the parameter estimates for CNB are more stable + * than those for Multinomial NB. Like Multinomial NB, the input feature values for Complement NB must + * be nonnegative. * Since 3.0.0, it also supports Gaussian NB * (see * here) @@ -106,7 +111,7 @@ class NaiveBayes @Since("1.5.0") ( /** * Set the model type using a string (case-sensitive). - * Supported options: "multinomial" and "bernoulli". + * Supported options: "multinomial", "complement", "bernoulli", and "gaussian". * Default is "multinomial" * @group setParam */ @@ -151,7 +156,7 @@ class NaiveBayes @Since("1.5.0") ( } $(modelType) match { - case Bernoulli | Multinomial => + case Bernoulli | Multinomial | Complement => trainDiscreteImpl(dataset, instr) case Gaussian => trainGaussianImpl(dataset, instr) @@ -168,7 +173,7 @@ class NaiveBayes @Since("1.5.0") ( import spark.implicits._ val validateUDF = $(modelType) match { - case Multinomial => + case Multinomial | Complement => udf { vector: Vector => requireNonnegativeValues(vector); vector } case Bernoulli => udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } @@ -204,14 +209,29 @@ class NaiveBayes @Since("1.5.0") ( val piArray = new Array[Double](numLabels) val thetaArray = new Array[Double](numLabels * numFeatures) + val aggIter = $(modelType) match { + case Multinomial | Bernoulli => aggregated.iterator + case Complement => + val featureSum = Vectors.zeros(numFeatures) + aggregated.foreach { case (_, _, sumTermFreqs, _) => + BLAS.axpy(1.0, sumTermFreqs, featureSum) + } + aggregated.iterator.map { case (label, n, sumTermFreqs, count) => + val comp = featureSum.copy + BLAS.axpy(-1.0, sumTermFreqs, comp) + (label, n, comp, count) + } + } + val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, n, sumTermFreqs, _) => + aggIter.foreach { case (label, n, sumTermFreqs, _) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) + case Multinomial | Complement => + math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) } var j = 0 @@ -224,9 +244,16 @@ class NaiveBayes @Since("1.5.0") ( } val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) - .setOldLabels(labelArray) + $(modelType) match { + case Multinomial | Bernoulli => + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + .setOldLabels(labelArray) + case Complement => + // Since the CNB compute the coefficient in a complement way. + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray.map(v => -v), true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + } } private def trainGaussianImpl( @@ -322,10 +349,14 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Gaussian model type. */ private[classification] val Gaussian: String = "gaussian" + /** String name for Complement model type. */ + private[classification] val Complement: String = "complement" + /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) + private[classification] val supportedModelTypes = + Set(Multinomial, Bernoulli, Gaussian, Complement) - private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { + private[ml] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { case sv: SparseVector => sv.values case dv: DenseVector => dv.values @@ -335,7 +366,7 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { s"Naive Bayes requires nonnegative feature values but found $v.") } - private[NaiveBayes] def requireZeroOneBernoulliValues(v: Vector): Unit = { + private[ml] def requireZeroOneBernoulliValues(v: Vector): Unit = { val values = v match { case sv: SparseVector => sv.values case dv: DenseVector => dv.values @@ -368,7 +399,7 @@ class NaiveBayesModel private[ml] ( extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial, Gaussian} + import NaiveBayes._ /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -427,16 +458,39 @@ class NaiveBayesModel private[ml] ( override val numClasses: Int = pi.size private def multinomialCalculation(features: Vector) = { + requireNonnegativeValues(features) val prob = theta.multiply(features) BLAS.axpy(1.0, pi, prob) prob } + private def complementCalculation(features: Vector) = { + requireNonnegativeValues(features) + val probArray = theta.multiply(features).toArray + // the following lines equal to: + // val logSumExp = math.log(probArray.map(math.exp).sum) + // However, it easily returns Infinity/NaN values. + // Here follows 'scipy.special.logsumexp' (which is used in Scikit-Learn's ComplementNB) + // to compute the log of the sum of exponentials of elements in a numeric-stable way. + val max = probArray.max + var sumExp = 0.0 + var j = 0 + while (j < probArray.length) { + sumExp += math.exp(probArray(j) - max) + j += 1 + } + val logSumExp = math.log(sumExp) + max + + j = 0 + while (j < probArray.length) { + probArray(j) = probArray(j) - logSumExp + j += 1 + } + Vectors.dense(probArray) + } + private def bernoulliCalculation(features: Vector) = { - features.foreachActive((_, value) => - require(value == 0.0 || value == 1.0, - s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") - ) + requireZeroOneBernoulliValues(features) val prob = thetaMinusNegTheta.multiply(features) BLAS.axpy(1.0, pi, prob) BLAS.axpy(1.0, negThetaSum, prob) @@ -464,6 +518,8 @@ class NaiveBayesModel private[ml] ( $(modelType) match { case Multinomial => features: Vector => multinomialCalculation(features) + case Complement => + features: Vector => complementCalculation(features) case Bernoulli => features: Vector => bernoulliCalculation(features) case Gaussian => @@ -533,7 +589,7 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { val dataPath = new Path(path, "data").toString instance.getModelType match { - case Multinomial | Bernoulli => + case Multinomial | Bernoulli | Complement => // Save model data: pi, theta require(instance.sigma == null) val data = Data(instance.pi, instance.theta) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9e4844ff89079..4a555ad3ed071 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -40,6 +40,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var bernoulliDataset: Dataset[_] = _ @transient var gaussianDataset: Dataset[_] = _ @transient var gaussianDataset2: Dataset[_] = _ + @transient var complementDataset: Dataset[_] = _ private val seed = 42 @@ -70,8 +71,12 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Array(0.10, 0.10, 0.10, 0.50) // label 2: variance ) gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() + gaussianDataset2 = spark.read.format("libsvm") .load("../data/mllib/sample_multiclass_classification_data.txt") + + complementDataset = spark.read.format("libsvm") + .load("../data/mllib/sample_libsvm_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -154,6 +159,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") assert(Gaussian === "gaussian") + assert(Complement === "complement") } test("params") { @@ -218,8 +224,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Array(0.10, 0.70, 0.10, 0.10), // label 1 Array(0.10, 0.10, 0.70, 0.10) // label 2 ).map(_.map(math.log)) - val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) val trainDataset = generateNaiveBayesInput(piArray, thetaArray, nPoints, seed, "multinomial").toDF() @@ -245,6 +249,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), ("multinomial", dataset), + ("complement", dataset), ("gaussian", gaussianDataset) ) testParams.foreach { case (family, dataset) => @@ -428,19 +433,75 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val thetaRows = model.theta.rowIter.toArray assert(thetaRows(0) ~= - Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) + Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005) relTol 1E-5) assert(thetaRows(1) ~= - Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) + Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014) relTol 1E-5) assert(thetaRows(2) ~= - Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) + Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667) relTol 1E-5) val sigmaRows = model.sigma.rowIter.toArray assert(sigmaRows(0) ~= - Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) + Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607) relTol 1E-5) assert(sigmaRows(1) ~= - Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) + Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224) relTol 1E-5) assert(sigmaRows(2) ~= - Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) + Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392) relTol 1E-5) + } + + test("Naive Bayes Complement") { + /* + Using the following Python code to verify the correctness. + + import numpy as np + from sklearn.naive_bayes import ComplementNB + from sklearn.datasets import load_svmlight_file + + path = "./data/mllib/sample_libsvm_data.txt" + X, y = load_svmlight_file(path) + X = X.toarray() + clf = ComplementNB() + clf.fit(X, y) + + >>> clf.feature_log_prob_[:, -5:] + array([[ 7.2937608 , 10.26577655, 13.73151245, 13.73151245, 13.73151245], + [ 6.99678043, 7.51387415, 7.74399483, 8.32904552, 9.53119848]]) + >>> clf.predict_log_proba(X[:5]) + array([[ 0. , -74732.70765355], + [-36018.30169185, 0. ], + [-37126.4015229 , 0. ], + [-27649.81038619, 0. ], + [-28767.84075587, 0. ]]) + >>> clf.predict_proba(X[:5]) + array([[1., 0.], + [0., 1.], + [0., 1.], + [0., 1.], + [0., 1.]]) + */ + + val cnb = new NaiveBayes().setModelType(Complement) + val model = cnb.fit(complementDataset) + + val thetaRows = model.theta.rowIter.map(vec => Vectors.dense(vec.toArray.takeRight(5))).toArray + assert(thetaRows(0) ~= + Vectors.dense(7.2937608, 10.26577655, 13.73151245, 13.73151245, 13.73151245) relTol 1E-5) + assert(thetaRows(1) ~= + Vectors.dense(6.99678043, 7.51387415, 7.74399483, 8.32904552, 9.53119848) relTol 1E-5) + + val preds = model.transform(complementDataset) + .select("rawPrediction", "probability") + .as[(Vector, Vector)] + .take(5) + assert(preds(0)._1 ~= Vectors.dense(0.0, -74732.70765355) relTol 1E-5) + assert(preds(0)._2 ~= Vectors.dense(1.0, 0.0) relTol 1E-5) + assert(preds(1)._1 ~= Vectors.dense(-36018.30169185, 0.0) relTol 1E-5) + assert(preds(1)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(2)._1 ~= Vectors.dense(-37126.4015229, 0.0) relTol 1E-5) + assert(preds(2)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(3)._1 ~= Vectors.dense(-27649.81038619, 0.0) relTol 1E-5) + assert(preds(3)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(4)._1 ~= Vectors.dense(-28767.84075587, 0.0) relTol 1E-5) + assert(preds(4)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) } test("read/write") { diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 03087c34b9d0b..d6fe26dc69da8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1909,6 +1909,11 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, binary (0/1) data, it can also be used as `Bernoulli NB `_. The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + Since 3.0.0, it supports Complement NB which is an adaptation of the Multinomial NB. + Specifically, Complement NB uses statistics from the complement of each class to compute + the model's coefficients. The inventors of Complement NB show empirically that the parameter + estimates for CNB are more stable than those for Multinomial NB. Like Multinomial NB, the + input feature values for Complement NB must be nonnegative. Since 3.0.0, it also supports Gaussian NB `_. which can handle continuous data. @@ -1967,6 +1972,14 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, 'gaussian' >>> model4.sigma DenseMatrix(2, 2, [0.0, 0.25, 0.0, 0.0], 1) + >>> nb5 = NaiveBayes(smoothing=1.0, modelType="complement", weightCol="weight") + >>> model5 = nb5.fit(df) + >>> model5.getModelType() + 'complement' + >>> model5.theta + DenseMatrix(2, 2, [...], 1) + >>> model5.sigma == None + True .. versionadded:: 1.5.0 """ From 0f40d2a6ee0bba3f95785b8c36171e104c12615a Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 21 Nov 2019 18:32:28 +0800 Subject: [PATCH 65/83] [SPARK-29960][ML][PYSPARK] MulticlassClassificationEvaluator support hammingLoss ### What changes were proposed in this pull request? MulticlassClassificationEvaluator support hammingLoss ### Why are the changes needed? 1, it is an easy to compute hammingLoss based on confusion matrix 2, scikit-learn supports it ### Does this PR introduce any user-facing change? yes ### How was this patch tested? added testsuites Closes #26597 from zhengruifeng/multi_class_hamming_loss. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../MulticlassClassificationEvaluator.scala | 8 +++-- .../mllib/evaluation/MulticlassMetrics.scala | 17 ++++++++++ .../evaluation/MulticlassMetricsSuite.scala | 31 +++++++++++++++++++ python/pyspark/ml/evaluation.py | 6 +++- 4 files changed, 58 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index ab14227f06be1..435708186242f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -46,7 +46,7 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid * `"weightedPrecision"`, `"weightedRecall"`, `"weightedTruePositiveRate"`, * `"weightedFalsePositiveRate"`, `"weightedFMeasure"`, `"truePositiveRateByLabel"`, * `"falsePositiveRateByLabel"`, `"precisionByLabel"`, `"recallByLabel"`, - * `"fMeasureByLabel"`, `"logLoss"`) + * `"fMeasureByLabel"`, `"logLoss"`, `"hammingLoss"`) * * @group param */ @@ -172,13 +172,15 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "precisionByLabel" => metrics.precision($(metricLabel)) case "recallByLabel" => metrics.recall($(metricLabel)) case "fMeasureByLabel" => metrics.fMeasure($(metricLabel), $(beta)) + case "hammingLoss" => metrics.hammingLoss case "logLoss" => metrics.logLoss($(eps)) } } @Since("1.5.0") override def isLargerBetter: Boolean = $(metricName) match { - case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" => false + case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" | "hammingLoss" => + false case _ => true } @@ -199,7 +201,7 @@ object MulticlassClassificationEvaluator private val supportedMetricNames = Array("f1", "accuracy", "weightedPrecision", "weightedRecall", "weightedTruePositiveRate", "weightedFalsePositiveRate", "weightedFMeasure", "truePositiveRateByLabel", "falsePositiveRateByLabel", "precisionByLabel", "recallByLabel", - "fMeasureByLabel", "logLoss") + "fMeasureByLabel", "logLoss", "hammingLoss") @Since("1.6.0") override def load(path: String): MulticlassClassificationEvaluator = super.load(path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 9518f7e6828cf..050ebb0fa4fbd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -240,6 +240,23 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) @Since("1.1.0") lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted + /** + * Returns Hamming-loss + */ + @Since("3.0.0") + lazy val hammingLoss: Double = { + var numerator = 0.0 + var denominator = 0.0 + confusions.iterator.foreach { + case ((label, prediction), weight) => + if (label != prediction) { + numerator += weight + } + denominator += weight + } + numerator / denominator + } + /** * Returns the log-loss, aka logistic loss or cross-entropy loss. * @param eps log-loss is undefined for p=0 or p=1, so probabilities are diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index e10295c905cdb..a8c6339ba6824 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -254,4 +254,35 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val metrics2 = new MulticlassMetrics(rdd2) assert(metrics2.logLoss() ~== 0.9682005730687164 relTol delta) } + + test("MulticlassMetrics supports hammingLoss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import hamming_loss + y_true = [2, 2, 3, 4] + y_pred = [1, 2, 3, 4] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> hamming_loss(y_true, y_pred) + 0.25 + >>> hamming_loss(y_true, y_pred, sample_weight=weights) + 0.3 + */ + + val preds = Seq(1.0, 2.0, 3.0, 4.0) + val labels = Seq(2.0, 2.0, 3.0, 4.0) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(preds.zip(labels)) + val metrics = new MulticlassMetrics(rdd) + assert(metrics.hammingLoss ~== 0.25 relTol delta) + + val rdd2 = sc.parallelize(preds.zip(labels).zip(weights)) + .map { case ((pred, label), weight) => + (pred, label, weight) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.hammingLoss ~== 0.3 relTol delta) + } } diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 6539e2abaed12..556a2f85c708d 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -374,6 +374,10 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> evaluator.evaluate(dataset, {evaluator.metricName: "truePositiveRateByLabel", ... evaluator.metricLabel: 1.0}) 0.75... + >>> evaluator.setMetricName("hammingLoss") + MulticlassClassificationEvaluator... + >>> evaluator.evaluate(dataset) + 0.33... >>> mce_path = temp_path + "/mce" >>> evaluator.save(mce_path) >>> evaluator2 = MulticlassClassificationEvaluator.load(mce_path) @@ -408,7 +412,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio "(f1|accuracy|weightedPrecision|weightedRecall|weightedTruePositiveRate|" "weightedFalsePositiveRate|weightedFMeasure|truePositiveRateByLabel|" "falsePositiveRateByLabel|precisionByLabel|recallByLabel|fMeasureByLabel|" - "logLoss)", + "logLoss|hammingLoss)", typeConverter=TypeConverters.toString) metricLabel = Param(Params._dummy(), "metricLabel", "The class whose metric will be computed in truePositiveRateByLabel|" From 6146dc4562739c1c947eb944897c2fe85d1016e0 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 21 Nov 2019 18:43:21 +0800 Subject: [PATCH 66/83] [SPARK-29874][SQL] Optimize Dataset.isEmpty() ### What changes were proposed in this pull request? In origin way to judge if a DataSet is empty by ``` def isEmpty: Boolean = withAction("isEmpty", limit(1).groupBy().count().queryExecution) { plan => plan.executeCollect().head.getLong(0) == 0 } ``` will add two shuffles by `limit()`, `groupby() and count()`, then collect all data to driver. In this way we can avoid `oom` when collect data to driver. But it will trigger all partitions calculated and add more shuffle process. We change it to ``` def isEmpty: Boolean = withAction("isEmpty", select().queryExecution) { plan => plan.executeTake(1).isEmpty } ``` After these pr, we will add a column pruning to origin LogicalPlan and use `executeTake()` API. then we won't add more shuffle process and just compute only one partition's data in last stage. In this way we can reduce cost when we call `DataSet.isEmpty()` and won't bring memory issue to driver side. ### Why are the changes needed? Optimize Dataset.isEmpty() ### Does this PR introduce any user-facing change? No ### How was this patch tested? Origin UT Closes #26500 from AngersZhuuuu/SPARK-29874. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 0fcd82276392c..e1bca44dfccf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -586,8 +586,8 @@ class Dataset[T] private[sql]( * @group basic * @since 2.4.0 */ - def isEmpty: Boolean = withAction("isEmpty", limit(1).groupBy().count().queryExecution) { plan => - plan.executeCollect().head.getLong(0) == 0 + def isEmpty: Boolean = withAction("isEmpty", select().queryExecution) { plan => + plan.executeTake(1).isEmpty } /** From cdcd43cbf2479b258f4c5cfa0f6306f475d25cf2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 21 Nov 2019 23:51:12 +0800 Subject: [PATCH 67/83] [SPARK-29977][SQL] Remove newMutableProjection/newOrdering/newNaturalAscendingOrdering from SparkPlan ### What changes were proposed in this pull request? This is to refactor `SparkPlan` code; it mainly removed `newMutableProjection`/`newOrdering`/`newNaturalAscendingOrdering` from `SparkPlan`. The other modifications are listed below; - Move `BaseOrdering` from `o.a.s.sqlcatalyst.expressions.codegen.GenerateOrdering.scala` to `o.a.s.sqlcatalyst.expressions.ordering.scala` - `RowOrdering` extends `CodeGeneratorWithInterpretedFallback ` for `BaseOrdering` - Remove the unused variables (`subexpressionEliminationEnabled` and `codeGenFallBack`) from `SparkPlan` ### Why are the changes needed? For better code/test coverage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing. Closes #26615 from maropu/RefactorOrdering. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../codegen/GenerateOrdering.scala | 10 +---- .../sql/catalyst/expressions/ordering.scala | 40 +++++++++++++++++-- .../sql/execution/UnsafeKVExternalSorter.java | 2 +- .../apache/spark/sql/execution/SortExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 38 ------------------ .../aggregate/HashAggregateExec.scala | 7 ++-- .../aggregate/ObjectAggregationIterator.scala | 2 +- .../aggregate/ObjectHashAggregateExec.scala | 2 +- .../aggregate/SortAggregateExec.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../sql/execution/python/EvalPythonExec.scala | 2 +- .../sql/execution/window/WindowExecBase.scala | 12 +++--- .../spark/sql/execution/ReferenceSort.scala | 4 +- 13 files changed, 55 insertions(+), 70 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index b66b80ad31dc2..63bd59e7628b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -29,19 +29,11 @@ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -/** - * Inherits some default implementation for Java from `Ordering[Row]` - */ -class BaseOrdering extends Ordering[InternalRow] { - def compare(a: InternalRow, b: InternalRow): Int = { - throw new UnsupportedOperationException - } -} /** * Generates bytecode for an [[Ordering]] of rows for a given set of expressions. */ -object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalRow]] with Logging { +object GenerateOrdering extends CodeGenerator[Seq[SortOrder], BaseOrdering] with Logging { protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index c9706c09f6949..8867a03a4633b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -19,18 +19,28 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering import org.apache.spark.sql.types._ +/** + * A base class for generated/interpreted row ordering. + */ +class BaseOrdering extends Ordering[InternalRow] { + def compare(a: InternalRow, b: InternalRow): Int = { + throw new UnsupportedOperationException + } +} + /** * An interpreted row ordering comparator. */ -class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow] { +class InterpretedOrdering(ordering: Seq[SortOrder]) extends BaseOrdering { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = this(bindReferences(ordering, inputSchema)) - def compare(a: InternalRow, b: InternalRow): Int = { + override def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 val size = ordering.size while (i < size) { @@ -67,7 +77,7 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow } i += 1 } - return 0 + 0 } } @@ -83,7 +93,7 @@ object InterpretedOrdering { } } -object RowOrdering { +object RowOrdering extends CodeGeneratorWithInterpretedFallback[Seq[SortOrder], BaseOrdering] { /** * Returns true iff the data type can be ordered (i.e. can be sorted). @@ -102,4 +112,26 @@ object RowOrdering { * Returns true iff outputs from the expressions can be ordered. */ def isOrderable(exprs: Seq[Expression]): Boolean = exprs.forall(e => isOrderable(e.dataType)) + + override protected def createCodeGeneratedObject(in: Seq[SortOrder]): BaseOrdering = { + GenerateOrdering.generate(in) + } + + override protected def createInterpretedObject(in: Seq[SortOrder]): BaseOrdering = { + new InterpretedOrdering(in) + } + + def create(order: Seq[SortOrder], inputSchema: Seq[Attribute]): BaseOrdering = { + createObject(bindReferences(order, inputSchema)) + } + + /** + * Creates a row ordering for the given schema, in natural ascending order. + */ + def createNaturalAscendingOrdering(dataTypes: Seq[DataType]): BaseOrdering = { + val order: Seq[SortOrder] = dataTypes.zipWithIndex.map { + case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) + } + create(order, Seq.empty) + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 09426117a24b9..acd54fe25d62d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -29,7 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.catalyst.expressions.codegen.BaseOrdering; +import org.apache.spark.sql.catalyst.expressions.BaseOrdering; import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering; import org.apache.spark.sql.types.StructType; import org.apache.spark.storage.BlockManager; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 24f664ca595c7..6b6ca531c6d3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -71,7 +71,7 @@ case class SortExec( * should make it public. */ def createSorter(): UnsafeExternalRowSorter = { - val ordering = newOrdering(sortOrder, output) + val ordering = RowOrdering.create(sortOrder, output) // The comparator for comparing prefix val boundSortExpression = BindReferences.bindReference(sortOrder.head, output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 738af995376e6..ef9f38b8f9927 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -22,9 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer -import org.codehaus.commons.compiler.CompileException -import org.codehaus.janino.InternalCompilerException - import org.apache.spark.{broadcast, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec @@ -32,13 +29,11 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object SparkPlan { @@ -72,16 +67,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ val id: Int = SparkPlan.newPlanId() - // sqlContext will be null when SparkPlan nodes are created without the active sessions. - val subexpressionEliminationEnabled: Boolean = if (sqlContext != null) { - sqlContext.conf.subexpressionEliminationEnabled - } else { - false - } - - // whether we should fallback when hitting compilation errors caused by codegen - private val codeGenFallBack = (sqlContext == null) || sqlContext.conf.codegenFallback - /** * Return true if this stage of the plan supports columnar execution. */ @@ -462,29 +447,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ buf.toArray } - protected def newMutableProjection( - expressions: Seq[Expression], - inputSchema: Seq[Attribute], - useSubexprElimination: Boolean = false): MutableProjection = { - log.debug(s"Creating MutableProj: $expressions, inputSchema: $inputSchema") - MutableProjection.create(expressions, inputSchema) - } - - protected def newOrdering( - order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[InternalRow] = { - GenerateOrdering.generate(order, inputSchema) - } - - /** - * Creates a row ordering for the given schema, in natural ascending order. - */ - protected def newNaturalAscendingOrdering(dataTypes: Seq[DataType]): Ordering[InternalRow] = { - val order: Seq[SortOrder] = dataTypes.zipWithIndex.map { - case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) - } - newOrdering(order, Seq.empty) - } - /** * Cleans up the resources used by the physical operator (if any). In general, all the resources * should be cleaned up when the task finishes but operators like SortMergeJoinExec and LimitExec diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 95bef308e453d..ad8976c77b16a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -126,7 +126,7 @@ case class HashAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), child.output, iter, testFallbackStartsAt, @@ -486,10 +486,9 @@ case class HashAggregateExec( // Create a MutableProjection to merge the rows of same key together val mergeExpr = declFunctions.flatMap(_.mergeExpressions) - val mergeProjection = newMutableProjection( + val mergeProjection = MutableProjection.create( mergeExpr, - aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes), - subexpressionEliminationEnabled) + aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes)) val joinedRow = new JoinedRow() var currentKey: UnsafeRow = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala index b88ddba8e48d3..1f325c11c9e44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala @@ -22,7 +22,7 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.codegen.{BaseOrdering, GenerateOrdering} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 151da241144be..953622afebf89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -122,7 +122,7 @@ case class ObjectHashAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), child.output, iter, fallbackCountThreshold, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 7ab6ecc08a7bc..0ddf95771d5b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -93,7 +93,7 @@ case class SortAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), numOutputRows) if (!hasInput && groupingExpressions.isEmpty) { // There is no input and there is no grouping expressions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index cd3c596435a21..f327e84563da9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -175,7 +175,7 @@ case class SortMergeJoinExec( } // An ordering that can be used to compare keys from both sides. - val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType)) + val keyOrdering = RowOrdering.createNaturalAscendingOrdering(leftKeys.map(_.dataType)) val resultProj: InternalRow => InternalRow = UnsafeProjection.create(output, output) joinType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala index 3554bdb5c9e0c..a0f23e925d237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala @@ -113,7 +113,7 @@ abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], } }.toArray }.toArray - val projection = newMutableProjection(allInputs, child.output) + val projection = MutableProjection.create(allInputs, child.output) val schema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => StructField(s"_$i", dt) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala index dcb86f48bdf32..e8248b7028757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala @@ -73,7 +73,7 @@ abstract class WindowExecBase( RowBoundOrdering(offset) case (RangeFrame, CurrentRow) => - val ordering = newOrdering(orderSpec, child.output) + val ordering = RowOrdering.create(orderSpec, child.output) RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) case (RangeFrame, offset: Expression) if orderSpec.size == 1 => @@ -82,7 +82,7 @@ abstract class WindowExecBase( val expr = sortExpr.child // Create the projection which returns the current 'value'. - val current = newMutableProjection(expr :: Nil, child.output) + val current = MutableProjection.create(expr :: Nil, child.output) // Flip the sign of the offset when processing the order is descending val boundOffset = sortExpr.direction match { @@ -97,13 +97,13 @@ abstract class WindowExecBase( TimeAdd(expr, boundOffset, Some(timeZone)) case (a, b) if a == b => Add(expr, boundOffset) } - val bound = newMutableProjection(boundExpr :: Nil, child.output) + val bound = MutableProjection.create(boundExpr :: Nil, child.output) // Construct the ordering. This is used to compare the result of current value projection // to the result of bound value projection. This is done manually because we want to use // Code Generation (if it is enabled). val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil - val ordering = newOrdering(boundSortExprs, Nil) + val ordering = RowOrdering.create(boundSortExprs, Nil) RangeBoundOrdering(ordering, current, bound) case (RangeFrame, _) => @@ -167,7 +167,7 @@ abstract class WindowExecBase( ordinal, child.output, (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) + MutableProjection.create(expressions, schema)) } // Create the factory @@ -182,7 +182,7 @@ abstract class WindowExecBase( functions.map(_.asInstanceOf[OffsetWindowFunction]), child.output, (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, schema), offset) // Entire Partition Frame. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index 6abcb1f067968..25b4464823e5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -21,7 +21,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering, SortOrder} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -41,7 +41,7 @@ case class ReferenceSort( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") { child.execute().mapPartitions( { iterator => - val ordering = newOrdering(sortOrder, child.output) + val ordering = RowOrdering.create(sortOrder, child.output) val sorter = new ExternalSorter[InternalRow, Null, InternalRow]( TaskContext.get(), ordering = Some(ordering)) sorter.insertAll(iterator.map(r => (r.copy(), null))) From 54c5087a3ae306ad766df81a5a6279f219b2ea47 Mon Sep 17 00:00:00 2001 From: Ximo Guanter Date: Fri, 22 Nov 2019 00:19:25 +0800 Subject: [PATCH 68/83] [SPARK-29248][SQL] provider number of partitions when creating v2 data writer factory ### What changes were proposed in this pull request? When implementing a ScanBuilder, we require the implementor to provide the schema of the data and the number of partitions. However, when someone is implementing WriteBuilder we only pass them the schema, but not the number of partitions. This is an asymetrical developer experience. This PR adds a PhysicalWriteInfo interface that is passed to createBatchWriterFactory and createStreamingWriterFactory that adds the number of partitions of the data that is going to be written. ### Why are the changes needed? Passing in the number of partitions on the WriteBuilder would enable data sources to provision their write targets before starting to write. For example: it could be used to provision a Kafka topic with a specific number of partitions it could be used to scale a microservice prior to sending the data to it it could be used to create a DsV2 that sends the data to another spark cluster (currently not possible since the reader wouldn't be able to know the number of partitions) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tests passed Closes #26591 from edrevo/temp. Authored-by: Ximo Guanter Signed-off-by: Wenchen Fan --- .../spark/sql/kafka010/KafkaBatchWrite.scala | 4 +-- .../sql/kafka010/KafkaStreamingWrite.scala | 5 +-- .../spark/sql/connector/write/BatchWrite.java | 8 +++-- .../connector/write/DataWriterFactory.java | 5 +-- .../connector/write/PhysicalWriteInfo.java | 33 +++++++++++++++++++ .../streaming/StreamingDataWriterFactory.java | 5 +-- .../write/streaming/StreamingWrite.java | 9 +++-- .../write/PhysicalWriteInfoImpl.scala | 20 +++++++++++ .../spark/sql/connector/InMemoryTable.scala | 2 +- .../datasources/noop/NoopDataSource.scala | 9 ++--- .../datasources/v2/FileBatchWrite.scala | 4 +-- .../v2/WriteToDataSourceV2Exec.scala | 29 ++++++++-------- .../WriteToContinuousDataSourceExec.scala | 7 ++-- .../streaming/sources/ConsoleWrite.scala | 5 +-- .../sources/ForeachWriterTable.scala | 5 +-- .../streaming/sources/MicroBatchWrite.scala | 6 ++-- .../execution/streaming/sources/memory.scala | 4 +-- .../connector/SimpleWritableDataSource.scala | 2 +- .../sources/StreamingDataSourceV2Suite.scala | 5 +-- 19 files changed, 119 insertions(+), 48 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala index 8e29e38b2a644..56c0fdd7c35b7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.types.StructType @@ -40,7 +40,7 @@ private[kafka010] class KafkaBatchWrite( validateQuery(schema.toAttributes, producerParams, topic) - override def createBatchWriterFactory(): KafkaBatchWriterFactory = + override def createBatchWriterFactory(info: PhysicalWriteInfo): KafkaBatchWriterFactory = KafkaBatchWriterFactory(topic, producerParams, schema) override def commit(messages: Array[WriterCommitMessage]): Unit = {} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala index 2b50b771e694e..bcf9e3416f843 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.types.StructType @@ -41,7 +41,8 @@ private[kafka010] class KafkaStreamingWrite( validateQuery(schema.toAttributes, producerParams, topic) - override def createStreamingWriterFactory(): KafkaStreamWriterFactory = + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): KafkaStreamWriterFactory = KafkaStreamWriterFactory(topic, producerParams, schema) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java index 37c5539d2518f..3e8b14172d6b2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java @@ -23,8 +23,8 @@ * An interface that defines how to write the data to data source for batch processing. * * The writing procedure is: - * 1. Create a writer factory by {@link #createBatchWriterFactory()}, serialize and send it to all - * the partitions of the input data(RDD). + * 1. Create a writer factory by {@link #createBatchWriterFactory(PhysicalWriteInfo)}, serialize + * and send it to all the partitions of the input data(RDD). * 2. For each partition, create the data writer, and write the data of the partition with this * writer. If all the data are written successfully, call {@link DataWriter#commit()}. If * exception happens during the writing, call {@link DataWriter#abort()}. @@ -45,8 +45,10 @@ public interface BatchWrite { * * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. + * + * @param info Physical information about the input data that will be written to this table. */ - DataWriterFactory createBatchWriterFactory(); + DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info); /** * Returns whether Spark should use the commit coordinator to ensure that at most one task for diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java index bcf8d8a59e5e5..310575df05d97 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.InternalRow; /** - * A factory of {@link DataWriter} returned by {@link BatchWrite#createBatchWriterFactory()}, - * which is responsible for creating and initializing the actual data writer at executor side. + * A factory of {@link DataWriter} returned by + * {@link BatchWrite#createBatchWriterFactory(PhysicalWriteInfo)}, which is responsible for + * creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java new file mode 100644 index 0000000000000..55a092e39970e --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.write; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; + +/** + * This interface contains physical write information that data sources can use when + * generating a {@link DataWriterFactory} or a {@link StreamingDataWriterFactory}. + */ +@Evolving +public interface PhysicalWriteInfo { + /** + * The number of partitions of the input data that is going to be written. + */ + int numPartitions(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java index daaa18d5bc4e7..9946867e8ea65 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java @@ -23,11 +23,12 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; /** * A factory of {@link DataWriter} returned by - * {@link StreamingWrite#createStreamingWriterFactory()}, which is responsible for creating - * and initializing the actual data writer at executor side. + * {@link StreamingWrite#createStreamingWriterFactory(PhysicalWriteInfo)}, which is responsible for + * creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java index 0821b34891654..4f930e1c158e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java @@ -19,14 +19,15 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; import org.apache.spark.sql.connector.write.WriterCommitMessage; /** * An interface that defines how to write the data to data source in streaming queries. * * The writing procedure is: - * 1. Create a writer factory by {@link #createStreamingWriterFactory()}, serialize and send it to - * all the partitions of the input data(RDD). + * 1. Create a writer factory by {@link #createStreamingWriterFactory(PhysicalWriteInfo)}, + * serialize and send it to all the partitions of the input data(RDD). * 2. For each epoch in each partition, create the data writer, and write the data of the epoch in * the partition with this writer. If all the data are written successfully, call * {@link DataWriter#commit()}. If exception happens during the writing, call @@ -48,8 +49,10 @@ public interface StreamingWrite { * * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. + * + * @param info Information about the RDD that will be written to this data writer */ - StreamingDataWriterFactory createStreamingWriterFactory(); + StreamingDataWriterFactory createStreamingWriterFactory(PhysicalWriteInfo info); /** * Commits this writing job for the specified epoch with a list of commit messages. The commit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala new file mode 100644 index 0000000000000..a663822f3eb45 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.write + +private[sql] case class PhysicalWriteInfoImpl(numPartitions: Int) extends PhysicalWriteInfo diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index e7b2dd2ebc9da..201860e5135ba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -124,7 +124,7 @@ class InMemoryTable( } private abstract class TestBatchWrite extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { BufferedRowsWriterFactory } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index 3f4f29c3e135a..03e5f43a2a0af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType @@ -58,7 +58,8 @@ private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate } private[noop] object NoopBatchWrite extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = NoopWriterFactory + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = + NoopWriterFactory override def commit(messages: Array[WriterCommitMessage]): Unit = {} override def abort(messages: Array[WriterCommitMessage]): Unit = {} } @@ -74,8 +75,8 @@ private[noop] object NoopWriter extends DataWriter[InternalRow] { } private[noop] object NoopStreamingWrite extends StreamingWrite { - override def createStreamingWriterFactory(): StreamingDataWriterFactory = - NoopStreamingDataWriterFactory + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = NoopStreamingDataWriterFactory override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala index e7d9a247533c4..266c834909363 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -20,7 +20,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats @@ -44,7 +44,7 @@ class FileBatchWrite( committer.abortJob(job) } - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { FileWriterFactory(description, committer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 9f4392da6ab4d..7d8a115c126eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, SupportsWrite, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfoImpl, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -353,17 +353,20 @@ trait V2TableWriteExec extends UnaryExecNode { override def output: Seq[Attribute] = Nil protected def writeWithV2(batchWrite: BatchWrite): RDD[InternalRow] = { - val writerFactory = batchWrite.createBatchWriterFactory() - val useCommitCoordinator = batchWrite.useCommitCoordinator - val rdd = query.execute() - // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single - // partition rdd to make sure we at least set up one write task to write the metadata. - val rddWithNonEmptyPartitions = if (rdd.partitions.length == 0) { - sparkContext.parallelize(Array.empty[InternalRow], 1) - } else { - rdd + val rdd: RDD[InternalRow] = { + val tempRdd = query.execute() + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + if (tempRdd.partitions.length == 0) { + sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + tempRdd + } } - val messages = new Array[WriterCommitMessage](rddWithNonEmptyPartitions.partitions.length) + val writerFactory = batchWrite.createBatchWriterFactory( + PhysicalWriteInfoImpl(rdd.getNumPartitions)) + val useCommitCoordinator = batchWrite.useCommitCoordinator + val messages = new Array[WriterCommitMessage](rdd.partitions.length) val totalNumRowsAccumulator = new LongAccumulator() logInfo(s"Start processing data source write support: $batchWrite. " + @@ -371,10 +374,10 @@ trait V2TableWriteExec extends UnaryExecNode { try { sparkContext.runJob( - rddWithNonEmptyPartitions, + rdd, (context: TaskContext, iter: Iterator[InternalRow]) => DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator), - rddWithNonEmptyPartitions.partitions.indices, + rdd.partitions.indices, (index, result: DataWritingSparkTaskResult) => { val commitMessage = result.writerCommitMessage messages(index) = commitMessage diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index d4e522562e914..f1898ad3f27ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StreamExecution @@ -38,8 +39,10 @@ case class WriteToContinuousDataSourceExec(write: StreamingWrite, query: SparkPl override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = write.createStreamingWriterFactory() - val rdd = new ContinuousWriteRDD(query.execute(), writerFactory) + val queryRdd = query.execute() + val writerFactory = write.createStreamingWriterFactory( + PhysicalWriteInfoImpl(queryRdd.getNumPartitions)) + val rdd = new ContinuousWriteRDD(queryRdd, writerFactory) logInfo(s"Start processing data source write support: $write. " + s"The input RDD has ${rdd.partitions.length} partitions.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index 6afb811a4d998..ad5c7cf24caf7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.{PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,7 +38,8 @@ class ConsoleWrite(schema: StructType, options: CaseInsensitiveStringMap) assert(SparkSession.getActiveSession.isDefined) protected val spark = SparkSession.getActiveSession.get - def createStreamingWriterFactory(): StreamingDataWriterFactory = PackedRowWriterFactory + def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory = + PackedRowWriterFactory override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { // We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2 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 bae7fa7d07356..53d4bca1a5f7e 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.write.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.python.PythonForeachWriter import org.apache.spark.sql.types.StructType @@ -72,7 +72,8 @@ case class ForeachWriterTable[T]( override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = { val rowConverter: InternalRow => T = converter match { case Left(enc) => val boundEnc = enc.resolveAndBind( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index 5f12832cd2550..c2adc1dd6742a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} /** @@ -36,8 +36,8 @@ class MicroBatchWrite(eppchId: Long, val writeSupport: StreamingWrite) extends B writeSupport.abort(eppchId, messages) } - override def createBatchWriterFactory(): DataWriterFactory = { - new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory()) + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { + new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory(info)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index 51ab5ce3578af..a976876b4d8e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.types.StructType @@ -140,7 +140,7 @@ class MemoryStreamingWrite( val sink: MemorySink, schema: StructType, needTruncate: Boolean) extends StreamingWrite { - override def createStreamingWriterFactory: MemoryWriterFactory = { + override def createStreamingWriterFactory(info: PhysicalWriteInfo): MemoryWriterFactory = { MemoryWriterFactory(schema) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index 22d3750022c57..306da996e2caa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala @@ -99,7 +99,7 @@ class SimpleWritableDataSource extends TableProvider with SessionConfigSupport { } class MyBatchWrite(queryId: String, path: String, conf: Configuration) extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { SimpleCounter.resetCounter new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index e9d148c38e6cb..c4f0751a955d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsRea import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, MicroBatchStream, Offset, PartitionOffset} -import org.apache.spark.sql.connector.write.{WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{PhysicalWriteInfo, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} @@ -68,7 +68,8 @@ class FakeScanBuilder extends ScanBuilder with Scan { class FakeWriteBuilder extends WriteBuilder with StreamingWrite { override def buildForStreaming(): StreamingWrite = this - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = { throw new IllegalStateException("fake sink - cannot actually write") } override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { From 6b4b6a87cde8e29da5cbc2ee00242ec74d5477b2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 21 Nov 2019 09:47:42 -0800 Subject: [PATCH 69/83] [SPARK-29558][SQL] ResolveTables and ResolveRelations should be order-insensitive ### What changes were proposed in this pull request? Make `ResolveRelations` call `ResolveTables` at the beginning, and make `ResolveTables` call `ResolveTempViews`(newly added) at the beginning, to ensure the relation resolution priority. ### Why are the changes needed? To resolve an `UnresolvedRelation`, the general process is: 1. try to resolve to (global) temp view first. If it's not a temp view, move on 2. if the table name specifies a catalog, lookup the table from the specified catalog. Otherwise, lookup table from the current catalog. 3. when looking up table from session catalog, return a v1 relation if the table provider is v1. Currently, this process is done by 2 rules: `ResolveTables` and `ResolveRelations`. To avoid rule conflicts, we add a lot of checks: 1. `ResolveTables` only resolves `UnresolvedRelation` if it's not a temp view and the resolved table is not v1. 2. `ResolveRelations` only resolves `UnresolvedRelation` if the table name has less than 2 parts. This requires to run `ResolveTables` before `ResolveRelations`, otherwise we may resolve a v2 table to a v1 relation. To clearly guarantee the resolution priority, and avoid massive changes, this PR proposes to call one rule in another rule to ensure the rule execution order. Now the process is simple: 1. first run `ResolveTempViews`, see if we can resolve relation to temp view 2. then run `ResolveTables`, see if we can resolve relation to v2 tables. 3. finally run `ResolveRelations`, see if we can resolve relation to v1 tables. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26214 from cloud-fan/resolve. Authored-by: Wenchen Fan Signed-off-by: Ryan Blue --- .../sql/catalyst/analysis/Analyzer.scala | 26 ++++++++++++------- .../sql/catalyst/catalog/SessionCatalog.scala | 19 ++++++++++++++ .../connector/catalog/CatalogManager.scala | 2 +- .../sql/connector/catalog/LookupCatalog.scala | 17 ++++++++++-- .../sql/connector/DataSourceV2SQLSuite.scala | 16 +++++++++++- 5 files changed, 67 insertions(+), 13 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 a7443e71c0ca3..625ef2153c711 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 @@ -198,7 +198,6 @@ class Analyzer( ResolveTableValuedFunctions :: new ResolveCatalogs(catalogManager) :: ResolveInsertInto :: - ResolveTables :: ResolveRelations :: ResolveReferences :: ResolveCreateNamedStruct :: @@ -666,12 +665,26 @@ class Analyzer( } /** - * Resolve table relations with concrete relations from v2 catalog. + * Resolve relations to temp views. This is not an actual rule, and is only called by + * [[ResolveTables]]. + */ + object ResolveTempViews extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case u @ UnresolvedRelation(Seq(part1)) => + v1SessionCatalog.lookupTempView(part1).getOrElse(u) + case u @ UnresolvedRelation(Seq(part1, part2)) => + v1SessionCatalog.lookupGlobalTempView(part1, part2).getOrElse(u) + } + } + + /** + * Resolve table relations with concrete relations from v2 catalog. This is not an actual rule, + * and is only called by [[ResolveRelations]]. * * [[ResolveRelations]] still resolves v1 tables. */ object ResolveTables extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).resolveOperatorsUp { case u: UnresolvedRelation => lookupV2Relation(u.multipartIdentifier) .getOrElse(u) @@ -733,10 +746,6 @@ class Analyzer( // Note this is compatible with the views defined by older versions of Spark(before 2.2), which // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { - case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) - if v1SessionCatalog.isTemporaryTable(ident) => - resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) - case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => val defaultDatabase = AnalysisContext.get.defaultDatabase val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase) @@ -767,7 +776,7 @@ class Analyzer( case _ => plan } - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + def apply(plan: LogicalPlan): LogicalPlan = ResolveTables(plan).resolveOperatorsUp { case i @ InsertIntoStatement(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) if child.resolved => EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match { @@ -2839,7 +2848,6 @@ class Analyzer( private def lookupV2RelationAndCatalog( identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if v1SessionCatalog.isTemporaryTable(ti) => None case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => CatalogV2Util.loadTable(catalog, ident) match { case Some(table) => Some((DataSourceV2Relation.create(table), catalog, ident)) 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 e77f3c70f0bdf..96ca1ac73e043 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 @@ -767,6 +767,25 @@ class SessionCatalog( } } + def lookupTempView(table: String): Option[SubqueryAlias] = { + val formattedTable = formatTableName(table) + getTempView(formattedTable).map { view => + SubqueryAlias(formattedTable, view) + } + } + + def lookupGlobalTempView(db: String, table: String): Option[SubqueryAlias] = { + val formattedDB = formatDatabaseName(db) + if (formattedDB == globalTempViewManager.database) { + val formattedTable = formatTableName(table) + getGlobalTempView(formattedTable).map { view => + SubqueryAlias(formattedTable, formattedDB, view) + } + } else { + None + } + } + /** * Return whether a table with the specified name is a temporary view. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index d62148b2bbe45..135c180ef4000 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -71,7 +71,7 @@ class CatalogManager( * This happens when the source implementation extends the v2 TableProvider API and is not listed * in the fallback configuration, spark.sql.sources.write.useV1SourceList */ - private def v2SessionCatalog: CatalogPlugin = { + private[sql] def v2SessionCatalog: CatalogPlugin = { conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 26ba93e57fc64..613c0d1797cc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** * A trait to encapsulate catalog lookup function and helpful extractors. @@ -120,10 +121,22 @@ private[sql] trait LookupCatalog extends Logging { * Extract catalog and the rest name parts from a multi-part identifier. */ object CatalogAndIdentifierParts { - def unapply(nameParts: Seq[String]): Some[(CatalogPlugin, Seq[String])] = { + private val globalTempDB = SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = { assert(nameParts.nonEmpty) try { - Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog + // API does not support view yet, and we have to use v1 commands to deal with global temp + // views. To simplify the implementation, we put global temp views in a special namespace + // in the session catalog. The special namespace has higher priority during name resolution. + // For example, if the name of a custom catalog is the same with `GLOBAL_TEMP_DATABASE`, + // this custom catalog can't be accessed. + if (nameParts.head.equalsIgnoreCase(globalTempDB)) { + Some((catalogManager.v2SessionCatalog, nameParts)) + } else { + Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + } } catch { case _: CatalogNotFoundException => Some((currentCatalog, nameParts)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 2a44251e102a0..2958b57bf06ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableExceptio import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} @@ -1786,6 +1786,20 @@ class DataSourceV2SQLSuite } } + test("global temp view should not be masked by v2 catalog") { + val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) + + try { + sql("create global temp view v as select 1") + sql(s"alter view $globalTempDB.v rename to v2") + checkAnswer(spark.table(s"$globalTempDB.v2"), Row(1)) + sql(s"drop view $globalTempDB.v2") + } finally { + spark.sharedState.globalTempViewManager.clear() + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") From e2f056f4a89b1bd9864be8c111d39af6558c839b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Nov 2019 10:56:28 +0900 Subject: [PATCH 70/83] [SPARK-29975][SQL] introduce --CONFIG_DIM directive ### What changes were proposed in this pull request? allow the sql test files to specify different dimensions of config sets during testing. For example, ``` --CONFIG_DIM1 a=1 --CONFIG_DIM1 b=2,c=3 --CONFIG_DIM2 x=1 --CONFIG_DIM2 y=1,z=2 ``` This example defines 2 config dimensions, and each dimension defines 2 config sets. We will run the queries 4 times: 1. a=1, x=1 2. a=1, y=1, z=2 3. b=2, c=3, x=1 4. b=2, c=3, y=1, z=2 ### Why are the changes needed? Currently `SQLQueryTestSuite` takes a long time. This is because we run each test at least 3 times, to check with different codegen modes. This is not necessary for most of the tests, e.g. DESC TABLE. We should only check these codegen modes for certain tests. With the --CONFIG_DIM directive, we can do things like: test different join operator(broadcast or shuffle join) X different codegen modes. After reducing testing time, we should be able to run thrifter server SQL tests with config settings. ### Does this PR introduce any user-facing change? no ### How was this patch tested? test only Closes #26612 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../resources/sql-tests/inputs/group-by.sql | 5 ++ .../inputs/higher-order-functions.sql | 5 ++ .../resources/sql-tests/inputs/inner-join.sql | 15 +++-- .../sql-tests/inputs/join-empty-relation.sql | 5 -- .../sql-tests/inputs/natural-join.sql | 5 -- .../inputs/order-by-nulls-ordering.sql | 5 ++ .../resources/sql-tests/inputs/outer-join.sql | 21 +++---- .../inputs/postgreSQL/aggregates_part1.sql | 5 ++ .../inputs/postgreSQL/aggregates_part2.sql | 5 ++ .../inputs/postgreSQL/aggregates_part3.sql | 5 ++ .../inputs/postgreSQL/aggregates_part4.sql | 5 ++ .../sql-tests/inputs/postgreSQL/join.sql | 15 +++-- .../inputs/postgreSQL/window_part1.sql | 5 ++ .../inputs/postgreSQL/window_part2.sql | 5 ++ .../inputs/postgreSQL/window_part3.sql | 5 ++ .../inputs/postgreSQL/window_part4.sql | 5 ++ .../exists-subquery/exists-aggregate.sql | 5 ++ .../exists-joins-and-set-ops.sql | 16 +++-- .../exists-subquery/exists-orderby-limit.sql | 5 ++ .../subquery/in-subquery/in-group-by.sql | 5 ++ .../inputs/subquery/in-subquery/in-joins.sql | 16 +++-- .../subquery/in-subquery/in-order-by.sql | 5 ++ .../subquery/in-subquery/not-in-group-by.sql | 5 ++ .../subquery/in-subquery/not-in-joins.sql | 4 -- .../test/resources/sql-tests/inputs/udaf.sql | 5 ++ .../inputs/udf/udf-join-empty-relation.sql | 5 -- .../sql-tests/inputs/udf/udf-natural-join.sql | 5 -- .../sql-tests/inputs/udf/udf-outer-join.sql | 4 -- .../resources/sql-tests/inputs/window.sql | 5 ++ .../sql-tests/results/outer-join.sql.out | 22 +------ .../apache/spark/sql/SQLQueryTestSuite.scala | 58 ++++++++----------- .../ThriftServerQueryTestSuite.scala | 18 +++--- 32 files changed, 183 insertions(+), 116 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index d602f63e529d1..fedf03d774e42 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -1,3 +1,8 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) diff --git a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql index 7665346f86ba8..cfa06aea82b04 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql @@ -1,3 +1,8 @@ +-- Test higher order functions with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create or replace temporary view nested as values (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), diff --git a/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql index e87c660cb1fe6..5623161839331 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql @@ -1,7 +1,14 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql index 2e6a5f362a8fa..8afa3270f4de4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql index e0abeda3eb44f..71a50157b766c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - create temporary view nt1 as select * from values ("one", 1), ("two", 2), diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql index f7637b444b9fe..ad3977465c835 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql @@ -1,3 +1,8 @@ +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Q1. testing window functions with order by create table spark_10747(col1 int, col2 int, col3 int) using parquet; diff --git a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql index ce09c21568f13..ceb438ec34b2d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql @@ -1,7 +1,14 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN -- SPARK-17099: Incorrect result when HAVING clause is added to group by query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES @@ -29,9 +36,6 @@ CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); --- Set the cross join enabled flag for the LEFT JOIN test since there's no join condition. --- Ultimately the join should be optimized away. -set spark.sql.crossJoin.enabled = true; SELECT * FROM ( SELECT @@ -39,6 +43,3 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (t.int_col) is not null; -set spark.sql.crossJoin.enabled = false; - - diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql index 5d54be9341148..63f80bd2efa73 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql @@ -8,6 +8,11 @@ -- avoid bit-exact output here because operations may not be bit-exact. -- SET extra_float_digits = 0; +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + SELECT avg(four) AS avg_1 FROM onek; SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql index ba91366014e16..a8af1db77563c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 2] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view int4_tbl as select * from values (0), (123456), diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql index 78fdbf6ae6cd2..6f5e549644bbf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 3] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- [SPARK-28865] Table inheritance -- try it on an inheritance tree -- create table minmaxtest(f1 int); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql index 6fa2306cf1475..0d255bed24e9c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 4] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- [SPARK-27980] Ordered-Set Aggregate Functions -- ordered-set aggregates diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql index 1ada723d6ae22..cc07b00cc3670 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql @@ -7,10 +7,17 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql -- --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql index ae2a015ada245..087d7a5befd19 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; -- [SPARK-29540] Thrift in some cases can't parse string to date diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql index 728e8cab0c3ba..395149e48d5c8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L320-562 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TABLE empsalary ( depname string, empno integer, diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql index 205c7d391a973..8187f8a2773ff 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; CREATE TABLE empsalary ( diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql index 456b390fca6c3..64ba8e3b7a5ad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Spark doesn't handle UDFs in SQL -- test user-defined window function with named args and default args -- CREATE FUNCTION nth_value_def(val anyelement, n integer = 1) RETURNS anyelement diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql index b5f458f2cb184..ae6a9641aae66 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql @@ -1,5 +1,10 @@ -- Tests aggregate expressions in outer query and EXISTS subquery. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql index cefc3fe6272ab..667573b30d265 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql @@ -1,9 +1,17 @@ -- Tests EXISTS subquery support. Tests Exists subquery -- used in Joins (Both when joins occurs in outer and suquery blocks) --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql index 19fc18833760c..580fc1d4162eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql @@ -1,5 +1,10 @@ -- Tests EXISTS subquery support with ORDER BY and LIMIT clauses. +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql index b1d96b32c2478..b06e1cccca5ab 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql @@ -1,6 +1,11 @@ -- A test suite for GROUP BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index cd350a98e130b..200a71ebbb622 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -1,9 +1,17 @@ -- A test suite for IN JOINS in parent side, subquery, and both predicate subquery -- It includes correlated cases. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql index 892e39ff47c1f..042966b0a4e26 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -1,6 +1,11 @@ -- A test suite for ORDER BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql index 58cf109e136c5..54b74534c1162 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql @@ -1,6 +1,11 @@ -- A test suite for NOT IN GROUP BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index bebc18a61894b..fcdb667ad4523 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -1,9 +1,5 @@ -- A test suite for not-in-joins in parent side, subquery, and both predicate subquery -- It includes correlated cases. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql index 58613a1325dfa..0374d98feb6e6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql @@ -1,3 +1,8 @@ +-- Test aggregate operator and UDAF with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1), (2), (3), (4) as t1(int_col1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql index 47fb70d02394b..b46206d4530ed 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - -- This test file was converted from join-empty-relation.sql. CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql index e5eb812d69a1c..7cf080ea1b4eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - -- This test file was converted from natural-join.sql. create temporary view nt1 as select * from values diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql index 4eb0805c9cc67..4b09bcb988d25 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql @@ -1,8 +1,4 @@ -- This test file was converted from outer-join.sql. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false -- SPARK-17099: Incorrect result when HAVING clause is added to group by query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index faab4c61c8640..e25a252418301 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -1,3 +1,8 @@ +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out index 5db3bae5d0379..b39fdb0e58720 100644 --- a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 8 +-- Number of queries: 6 -- !query 0 @@ -59,14 +59,6 @@ struct<> -- !query 5 -set spark.sql.crossJoin.enabled = true --- !query 5 schema -struct --- !query 5 output -spark.sql.crossJoin.enabled true - - --- !query 6 SELECT * FROM ( SELECT @@ -74,15 +66,7 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (t.int_col) is not null --- !query 6 schema +-- !query 5 schema struct --- !query 6 output +-- !query 5 output 97 - - --- !query 7 -set spark.sql.crossJoin.enabled = false --- !query 7 schema -struct --- !query 7 output -spark.sql.crossJoin.enabled false 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 6e86d37a310d6..ef30573017212 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 @@ -111,7 +111,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { import IntegratedUDFTestUtils._ private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - protected val isTestWithConfigSets: Boolean = true protected val baseResourcePath = { // We use a path based on Spark home for 2 reasons: @@ -236,17 +235,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } } - // For better test coverage, runs the tests on mixed config sets: WHOLESTAGE_CODEGEN_ENABLED - // and CODEGEN_FACTORY_MODE. - private lazy val codegenConfigSets = Array( - ("true", "CODEGEN_ONLY"), - ("false", "CODEGEN_ONLY"), - ("false", "NO_CODEGEN") - ).map { case (wholeStageCodegenEnabled, codegenFactoryMode) => - Array(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageCodegenEnabled, - SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode) - } - /** Run a test case. */ protected def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) @@ -271,36 +259,36 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") - // When we are regenerating the golden files for test cases without '--IMPORT' specified, or - // running test cases against [[ThriftServerQueryTestSuite], we don't need to set any config as - // they all need to return the same result. - // When we use '--SET' and '--IMPORT' together for those import queries, we want to run the - // same queries from the original file but with different settings and save the answers. So the - // `--SET` will be respected in this case. - if ((regenerateGoldenFiles && importedTestCaseName.isEmpty) || !isTestWithConfigSets) { - runQueries(queries, testCase, None) + val settingLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) + val settings = settingLines.flatMap(_.split(",").map { kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + + if (regenerateGoldenFiles) { + runQueries(queries, testCase, Some(settings)) } else { - val configSets = { - val configLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) - val configs = configLines.map(_.split(",").map { confAndValue => - val (conf, value) = confAndValue.span(_ != '=') + // A config dimension has multiple config sets, and a config set has multiple configs. + // - config dim: Seq[Seq[(String, String)]] + // - config set: Seq[(String, String)] + // - config: (String, String)) + // We need to do cartesian product for all the config dimensions, to get a list of + // config sets, and run the query once for each config set. + val configDimLines = comments.filter(_.startsWith("--CONFIG_DIM")).map(_.substring(12)) + val configDims = configDimLines.groupBy(_.takeWhile(_ != ' ')).mapValues { lines => + lines.map(_.dropWhile(_ != ' ').substring(1)).map(_.split(",").map { kv => + val (conf, value) = kv.span(_ != '=') conf.trim -> value.substring(1).trim - }) + }.toSeq).toSeq + } - if (configs.nonEmpty) { - codegenConfigSets.flatMap { codegenConfig => - configs.map { config => - config ++ codegenConfig - } - } - } else { - codegenConfigSets - } + val configSets = configDims.values.foldLeft(Seq(Seq[(String, String)]())) { (res, dim) => + dim.flatMap { configSet => res.map(_ ++ configSet) } } configSets.foreach { configSet => try { - runQueries(queries, testCase, Some(configSet)) + runQueries(queries, testCase, Some(settings ++ configSet)) } catch { case e: Throwable => val configs = configSet.map { 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 index dc1609d3326cd..30cca95dbbce7 100644 --- 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 @@ -83,9 +83,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - // We only test this test suite with the default configuration to reduce test time. - override val isTestWithConfigSets = false - /** List of test cases to ignore, in lower cases. */ override def blackList: Set[String] = super.blackList ++ Set( // Missing UDF @@ -103,10 +100,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "subquery/in-subquery/in-group-by.sql", "subquery/in-subquery/simple-in.sql", "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql", - // SPARK-29783: need to set conf - "interval-display-iso_8601.sql", - "interval-display-sql_standard.sql" + "subquery/in-subquery/in-set-operations.sql" ) override def runQueries( @@ -118,13 +112,21 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { loadTestData(statement) + configSet.foreach { configs => + for ((k, v) <- configs) { + statement.execute(s"SET $k = $v") + } + } + testCase match { case _: PgSQLTest => - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _: AnsiTest => + statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") case _ => + statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = false") } // Run the SQL queries preparing them for comparison. From 9ec2a4e58caa4128e9c690d72239cebd6b732084 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 22 Nov 2019 18:36:50 +0900 Subject: [PATCH 71/83] [SPARK-29911][SQL][FOLLOWUP] Move related unit test to ThriftServerWithSparkContextSuite ### What changes were proposed in this pull request? This is follow up of #26543 See https://github.com/apache/spark/pull/26543#discussion_r348934276 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exist UT. Closes #26628 from LantaoJin/SPARK-29911_FOLLOWUP. Authored-by: LantaoJin Signed-off-by: HyukjinKwon --- .../ThriftServerQueryTestSuite.scala | 23 ---- .../ThriftServerWithSparkContextSuite.scala | 104 ++++++++++++++++++ 2 files changed, 104 insertions(+), 23 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala 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 index 30cca95dbbce7..7c48579fd2ca8 100644 --- 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 @@ -271,29 +271,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - test("SPARK-29911: Uncache cached tables when session closed") { - val cacheManager = spark.sharedState.cacheManager - val globalTempDB = spark.sharedState.globalTempViewManager.database - withJdbcStatement { statement => - statement.execute("CACHE TABLE tempTbl AS SELECT 1") - } - // the cached data of local temporary view should be uncached - assert(cacheManager.isEmpty) - try { - withJdbcStatement { statement => - statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") - statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") - } - // the cached data of global temporary view shouldn't be uncached - assert(!cacheManager.isEmpty) - } finally { - withJdbcStatement { statement => - statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") - } - assert(cacheManager.isEmpty) - } - } - /** ThriftServer wraps the root exception, so it needs to be extracted. */ override def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { super.handleExceptions { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala new file mode 100644 index 0000000000000..b6ed5e3156a46 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.sql.{DriverManager, Statement} + +import scala.util.{Random, Try} + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession + +class ThriftServerWithSparkContextSuite extends QueryTest with SharedSparkSession { + + 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("HiveThriftServer2 started successfully") + } + + override def afterAll(): Unit = { + try { + hiveServer2.stop() + } finally { + super.afterAll() + } + } + + test("SPARK-29911: Uncache cached tables when session closed") { + val cacheManager = spark.sharedState.cacheManager + val globalTempDB = spark.sharedState.globalTempViewManager.database + withJdbcStatement { statement => + statement.execute("CACHE TABLE tempTbl AS SELECT 1") + } + // the cached data of local temporary view should be uncached + assert(cacheManager.isEmpty) + try { + withJdbcStatement { statement => + statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") + statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") + } + // the cached data of global temporary view shouldn't be uncached + assert(!cacheManager.isEmpty) + } finally { + withJdbcStatement { statement => + statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") + } + assert(cacheManager.isEmpty) + } + } + + 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)*): 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()) + } + } +} From 2dd6807e421c96d0aaafc57ceb48f50f66f9d2e7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 22 Nov 2019 19:32:27 +0800 Subject: [PATCH 72/83] [SPARK-28023][SQL] Add trim logic in UTF8String's toInt/toLong to make it consistent with other string-numeric casting ### What changes were proposed in this pull request? Modify `UTF8String.toInt/toLong` to support trim spaces for both sides before converting it to byte/short/int/long. With this kind of "cheap" trim can help improve performance for casting string to integrals. The idea is from https://github.com/apache/spark/pull/24872#issuecomment-556917834 ### Why are the changes needed? make the behavior consistent. ### Does this PR introduce any user-facing change? yes, cast string to an integral type, and binary comparison between string and integrals will trim spaces first. their behavior will be consistent with float and double. ### How was this patch tested? 1. add ut. 2. benchmark tests the benchmark is modified based on https://github.com/apache/spark/pull/24872#issuecomment-503827016 ```scala /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT 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.benchmark import org.apache.spark.benchmark.Benchmark /** * Benchmark trim the string when casting string type to Boolean/Numeric types. * To run this benchmark: * {{{ * 1. without sbt: * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/CastBenchmark-results.txt". * }}} */ object CastBenchmark extends SqlBasedBenchmark { This conversation was marked as resolved by yaooqinn override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val title = "Cast String to Integral" runBenchmark(title) { withTempPath { dir => val N = 500L << 14 val df = spark.range(N) val types = Seq("int", "long") (1 to 5).by(2).foreach { i => df.selectExpr(s"concat(id, '${" " * i}') as str") .write.mode("overwrite").parquet(dir + i.toString) } val benchmark = new Benchmark(title, N, minNumIters = 5, output = output) Seq(true, false).foreach { trim => types.foreach { t => val str = if (trim) "trim(str)" else "str" val expr = s"cast($str as $t) as c_$t" (1 to 5).by(2).foreach { i => benchmark.addCase(expr + s" - with $i spaces") { _ => spark.read.parquet(dir + i.toString).selectExpr(expr).collect() } } } } benchmark.run() } } } } ``` #### benchmark result. normal trim v.s. trim in toInt/toLong ```java ================================================================================================ Cast String to Integral ================================================================================================ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 Intel(R) Core(TM) i5-5287U CPU 2.90GHz Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast(trim(str) as int) as c_int - with 1 spaces 10220 12994 1337 0.8 1247.5 1.0X cast(trim(str) as int) as c_int - with 3 spaces 4763 8356 357 1.7 581.4 2.1X cast(trim(str) as int) as c_int - with 5 spaces 4791 8042 NaN 1.7 584.9 2.1X cast(trim(str) as long) as c_long - with 1 spaces 4014 6755 NaN 2.0 490.0 2.5X cast(trim(str) as long) as c_long - with 3 spaces 4737 6938 NaN 1.7 578.2 2.2X cast(trim(str) as long) as c_long - with 5 spaces 4478 6919 1404 1.8 546.6 2.3X cast(str as int) as c_int - with 1 spaces 4443 6222 NaN 1.8 542.3 2.3X cast(str as int) as c_int - with 3 spaces 3659 3842 170 2.2 446.7 2.8X cast(str as int) as c_int - with 5 spaces 4372 7996 NaN 1.9 533.7 2.3X cast(str as long) as c_long - with 1 spaces 3866 5838 NaN 2.1 471.9 2.6X cast(str as long) as c_long - with 3 spaces 3793 5449 NaN 2.2 463.0 2.7X cast(str as long) as c_long - with 5 spaces 4947 5961 1198 1.7 603.9 2.1X ``` Closes #26622 from yaooqinn/cheapstringtrim. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/UTF8String.java | 44 +++++++------ docs/sql-migration-guide.md | 2 + .../test/resources/sql-tests/inputs/cast.sql | 10 +++ .../resources/sql-tests/inputs/comparator.sql | 10 +++ .../resources/sql-tests/results/cast.sql.out | 66 ++++++++++++++++++- .../sql-tests/results/comparator.sql.out | 66 ++++++++++++++++++- 6 files changed, 176 insertions(+), 22 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index d7a498d1c1c2f..deecd4f015824 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1063,7 +1063,7 @@ public static class IntWrapper implements Serializable { } /** - * Parses this UTF8String to long. + * Parses this UTF8String(trimmed if needed) to long. * * Note that, in this method we accumulate the result in negative format, and convert it to * positive format at the end, if this string is not started with '-'. This is because min value @@ -1077,18 +1077,20 @@ public static class IntWrapper implements Serializable { * @return true if the parsing was successful else false */ public boolean toLong(LongWrapper toLongResult) { - if (numBytes == 0) { - return false; - } + int offset = 0; + while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + if (offset == this.numBytes) return false; - byte b = getByte(0); + int end = this.numBytes - 1; + while (end > offset && getByte(end) <= ' ') end--; + + byte b = getByte(offset); final boolean negative = b == '-'; - int offset = 0; if (negative || b == '+') { - offset++; - if (numBytes == 1) { + if (end - offset == 0) { return false; } + offset++; } final byte separator = '.'; @@ -1096,7 +1098,7 @@ public boolean toLong(LongWrapper toLongResult) { final long stopValue = Long.MIN_VALUE / radix; long result = 0; - while (offset < numBytes) { + while (offset <= end) { b = getByte(offset); offset++; if (b == separator) { @@ -1131,7 +1133,7 @@ public boolean toLong(LongWrapper toLongResult) { // This is the case when we've encountered a decimal separator. The fractional // part will not change the number, but we will verify that the fractional part // is well formed. - while (offset < numBytes) { + while (offset <= end) { byte currentByte = getByte(offset); if (currentByte < '0' || currentByte > '9') { return false; @@ -1151,7 +1153,7 @@ public boolean toLong(LongWrapper toLongResult) { } /** - * Parses this UTF8String to int. + * Parses this UTF8String(trimmed if needed) to int. * * Note that, in this method we accumulate the result in negative format, and convert it to * positive format at the end, if this string is not started with '-'. This is because min value @@ -1168,18 +1170,20 @@ public boolean toLong(LongWrapper toLongResult) { * @return true if the parsing was successful else false */ public boolean toInt(IntWrapper intWrapper) { - if (numBytes == 0) { - return false; - } + int offset = 0; + while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + if (offset == this.numBytes) return false; - byte b = getByte(0); + int end = this.numBytes - 1; + while (end > offset && getByte(end) <= ' ') end--; + + byte b = getByte(offset); final boolean negative = b == '-'; - int offset = 0; if (negative || b == '+') { - offset++; - if (numBytes == 1) { + if (end - offset == 0) { return false; } + offset++; } final byte separator = '.'; @@ -1187,7 +1191,7 @@ public boolean toInt(IntWrapper intWrapper) { final int stopValue = Integer.MIN_VALUE / radix; int result = 0; - while (offset < numBytes) { + while (offset <= end) { b = getByte(offset); offset++; if (b == separator) { @@ -1222,7 +1226,7 @@ public boolean toInt(IntWrapper intWrapper) { // This is the case when we've encountered a decimal separator. The fractional // part will not change the number, but we will verify that the fractional part // is well formed. - while (offset < numBytes) { + while (offset <= end) { byte currentByte = getByte(offset); if (currentByte < '0' || currentByte > '9') { return false; diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2d5afa919e668..6fc78893e6881 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -222,6 +222,8 @@ license: | - Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`. + - Since Spark 3.0, when casting string value to integral types, including tinyint, smallint, int and bigint type, the leading and trailing white spaces(<= ACSII 32) will be trimmed before convert to integral values, e.g. `cast(' 1 ' as int)` results `1`. In Spark version 2.4 and earlier, the result will be `null`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 8a035f594be54..3c1702e6f837e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -60,3 +60,13 @@ DESC FUNCTION EXTENDED boolean; -- cast string to interval and interval to string SELECT CAST('interval 3 month 1 hour' AS interval); SELECT CAST(interval 3 month 1 hour AS string); + +-- trim string before cast to numeric +select cast(' 1' as tinyint); +select cast(' 1\t' as tinyint); +select cast(' 1' as smallint); +select cast(' 1' as INT); +select cast(' 1' as bigint); +select cast(' 1' as float); +select cast(' 1 ' as DOUBLE); +select cast('1.0 ' as DEC); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/comparator.sql b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql index 3e2447723e576..70af4f75ac431 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/comparator.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql @@ -1,3 +1,13 @@ -- binary type select x'00' < x'0f'; select x'00' < x'ff'; + +-- trim string to numeric +select '1 ' = 1Y; +select '\t1 ' = 1Y; +select '1 ' = 1S; +select '1 ' = 1; +select ' 1' = 1L; +select ' 1' = cast(1.0 as float); +select ' 1.0 ' = 1.0D; +select ' 1.0 ' = 1.0BD; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 609d283da555c..bf1e873a48d17 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 35 +-- Number of queries: 43 -- !query 0 @@ -287,3 +287,67 @@ SELECT CAST(interval 3 month 1 hour AS string) struct -- !query 34 output 3 months 1 hours + + +-- !query 35 +select cast(' 1' as tinyint) +-- !query 35 schema +struct +-- !query 35 output +1 + + +-- !query 36 +select cast(' 1\t' as tinyint) +-- !query 36 schema +struct +-- !query 36 output +1 + + +-- !query 37 +select cast(' 1' as smallint) +-- !query 37 schema +struct +-- !query 37 output +1 + + +-- !query 38 +select cast(' 1' as INT) +-- !query 38 schema +struct +-- !query 38 output +1 + + +-- !query 39 +select cast(' 1' as bigint) +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +select cast(' 1' as float) +-- !query 40 schema +struct +-- !query 40 output +1.0 + + +-- !query 41 +select cast(' 1 ' as DOUBLE) +-- !query 41 schema +struct +-- !query 41 output +1.0 + + +-- !query 42 +select cast('1.0 ' as DEC) +-- !query 42 schema +struct +-- !query 42 output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out index afc7b5448b7b6..a182da2a79637 100644 --- a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 2 +-- Number of queries: 10 -- !query 0 @@ -16,3 +16,67 @@ select x'00' < x'ff' struct<(X'00' < X'FF'):boolean> -- !query 1 output true + + +-- !query 2 +select '1 ' = 1Y +-- !query 2 schema +struct<(CAST(1 AS TINYINT) = 1):boolean> +-- !query 2 output +true + + +-- !query 3 +select '\t1 ' = 1Y +-- !query 3 schema +struct<(CAST( 1 AS TINYINT) = 1):boolean> +-- !query 3 output +true + + +-- !query 4 +select '1 ' = 1S +-- !query 4 schema +struct<(CAST(1 AS SMALLINT) = 1):boolean> +-- !query 4 output +true + + +-- !query 5 +select '1 ' = 1 +-- !query 5 schema +struct<(CAST(1 AS INT) = 1):boolean> +-- !query 5 output +true + + +-- !query 6 +select ' 1' = 1L +-- !query 6 schema +struct<(CAST( 1 AS BIGINT) = 1):boolean> +-- !query 6 output +true + + +-- !query 7 +select ' 1' = cast(1.0 as float) +-- !query 7 schema +struct<(CAST( 1 AS FLOAT) = CAST(1.0 AS FLOAT)):boolean> +-- !query 7 output +true + + +-- !query 8 +select ' 1.0 ' = 1.0D +-- !query 8 schema +struct<(CAST( 1.0 AS DOUBLE) = 1.0):boolean> +-- !query 8 output +true + + +-- !query 9 +select ' 1.0 ' = 1.0BD +-- !query 9 schema +struct<(CAST( 1.0 AS DOUBLE) = CAST(1.0 AS DOUBLE)):boolean> +-- !query 9 output +true From 6e581cf164c3a2930966b270ac1406dc1195c942 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Nov 2019 10:26:54 -0800 Subject: [PATCH 73/83] [SPARK-29893][SQL][FOLLOWUP] code cleanup for local shuffle reader ### What changes were proposed in this pull request? A few cleanups for https://github.com/apache/spark/pull/26516: 1. move the calculating of partition start indices from the RDD to the rule. We can reuse code from "shrink number of reducers" in the future if we split partitions by size. 2. only check extra shuffles when adding local readers to the probe side. 3. add comments. 4. simplify the config name: `optimizedLocalShuffleReader` -> `localShuffleReader` ### Why are the changes needed? make code more maintainable. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26625 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../adaptive/LocalShuffledRowRDD.scala | 32 ++---- .../adaptive/OptimizeLocalShuffleReader.scala | 98 +++++++++++++------ .../exchange/ShuffleExchangeExec.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 13 +-- 5 files changed, 87 insertions(+), 64 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 74046cd91c962..33f91d045f7d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -393,8 +393,8 @@ object SQLConf { "must be a positive integer.") .createOptional - val OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED = - buildConf("spark.sql.adaptive.shuffle.optimizedLocalShuffleReader.enabled") + val LOCAL_SHUFFLE_READER_ENABLED = + buildConf("spark.sql.adaptive.shuffle.localShuffleReader.enabled") .doc("When true and adaptive execution is enabled, this enables the optimization of" + " converting the shuffle reader to local shuffle reader for the shuffle exchange" + " of the broadcast hash join in probe side.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 44254593f8895..6385ea67c49fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -44,7 +44,7 @@ private final class LocalShuffledRowRDDPartition( * data of another input table of the join that reads from shuffle. Each partition of the RDD reads * the whole data from just one mapper output locally. So actually there is no data transferred * from the network. - + * * This RDD takes a [[ShuffleDependency]] (`dependency`). * * The `dependency` has the parent RDD of this RDD, which represents the dataset before shuffle @@ -53,11 +53,15 @@ private final class LocalShuffledRowRDDPartition( * `dependency.partitioner.numPartitions` is the number of pre-shuffle partitions. (i.e. the number * of partitions of the map output). The post-shuffle partition number is the same to the parent * RDD's partition number. + * + * `partitionStartIndicesPerMapper` specifies how to split the shuffle blocks of each mapper into + * one or more partitions. For a mapper `i`, the `j`th partition includes shuffle blocks from + * `partitionStartIndicesPerMapper[i][j]` to `partitionStartIndicesPerMapper[i][j+1]` (exclusive). */ class LocalShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - advisoryParallelism : Option[Int] = None) + partitionStartIndicesPerMapper: Array[Array[Int]]) extends RDD[InternalRow](dependency.rdd.context, Nil) { private[this] val numReducers = dependency.partitioner.numPartitions @@ -65,30 +69,12 @@ class LocalShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) - /** - * To equally divide n elements into m buckets, basically each bucket should have n/m elements, - * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns - * a sequence with length numBuckets and each value represents the start index of each bucket. - */ - private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { - val elementsPerBucket = numElements / numBuckets - val remaining = numElements % numBuckets - val splitPoint = (elementsPerBucket + 1) * remaining - (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ - (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) - } - override def getPartitions: Array[Partition] = { - val partitionStartIndices: Array[Int] = { - val expectedParallelism = advisoryParallelism.getOrElse(numReducers) - // TODO split by data size in the future. - equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray - } - val partitions = ArrayBuffer[LocalShuffledRowRDDPartition]() for (mapIndex <- 0 until numMappers) { - (partitionStartIndices :+ numReducers).sliding(2, 1).foreach { case Array(start, end) => - partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) + (partitionStartIndicesPerMapper(mapIndex) :+ numReducers).sliding(2, 1).foreach { + case Array(start, end) => + partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) } } partitions.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index a57cfae6d0991..176e5ec8312e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -39,8 +39,12 @@ import org.apache.spark.sql.internal.SQLConf case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { import OptimizeLocalShuffleReader._ - def withProbeSideLocalReader(plan: SparkPlan): SparkPlan = { - plan.transformDown { + private val ensureRequirements = EnsureRequirements(conf) + + // The build side is a broadcast query stage which should have been optimized using local reader + // already. So we only need to deal with probe side here. + private def createProbeSideLocalReader(plan: SparkPlan): SparkPlan = { + val optimizedPlan = plan.transformDown { case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildRight) => val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) @@ -48,39 +52,71 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } + + val numShuffles = ensureRequirements.apply(optimizedPlan).collect { + case e: ShuffleExchangeExec => e + }.length + + // Check whether additional shuffle introduced. If introduced, revert the local reader. + if (numShuffles > 0) { + logDebug("OptimizeLocalShuffleReader rule is not applied due" + + " to additional shuffles will be introduced.") + plan + } else { + optimizedPlan + } } - def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { + private def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { plan match { - case c: CoalescedShuffleReaderExec => - LocalShuffleReaderExec(c.child, Some(c.partitionStartIndices.length)) - case q: QueryStageExec => LocalShuffleReaderExec(q) + case c @ CoalescedShuffleReaderExec(q: QueryStageExec, _) => + LocalShuffleReaderExec( + q, getPartitionStartIndices(q, Some(c.partitionStartIndices.length))) + case q: QueryStageExec => + LocalShuffleReaderExec(q, getPartitionStartIndices(q, None)) + } + } + + // TODO: this method assumes all shuffle blocks are the same data size. We should calculate the + // partition start indices based on block size to avoid data skew. + private def getPartitionStartIndices( + shuffle: QueryStageExec, + advisoryParallelism: Option[Int]): Array[Array[Int]] = { + val shuffleDep = shuffle match { + case s: ShuffleQueryStageExec => s.plan.shuffleDependency + case ReusedQueryStageExec(_, s: ShuffleQueryStageExec, _) => s.plan.shuffleDependency } + val numReducers = shuffleDep.partitioner.numPartitions + val expectedParallelism = advisoryParallelism.getOrElse(numReducers) + val numMappers = shuffleDep.rdd.getNumPartitions + Array.fill(numMappers) { + equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray + } + } + + /** + * To equally divide n elements into m buckets, basically each bucket should have n/m elements, + * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns + * a sequence with length numBuckets and each value represents the start index of each bucket. + */ + private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { + val elementsPerBucket = numElements / numBuckets + val remaining = numElements % numBuckets + val splitPoint = (elementsPerBucket + 1) * remaining + (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ + (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) } override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + if (!conf.getConf(SQLConf.LOCAL_SHUFFLE_READER_ENABLED)) { return plan } - val optimizedPlan = plan match { + plan match { case s: SparkPlan if canUseLocalShuffleReader(s) => createLocalReader(s) - case s: SparkPlan => withProbeSideLocalReader(s) - } - - def numExchanges(plan: SparkPlan): Int = { - plan.collect { - case e: ShuffleExchangeExec => e - }.length - } - // Check whether additional shuffle introduced. If introduced, revert the local reader. - if (numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) > 0) { - logDebug("OptimizeLocalShuffleReader rule is not applied due" + - " to additional shuffles will be introduced.") - plan - } else { - optimizedPlan + case s: SparkPlan => + createProbeSideLocalReader(s) } } } @@ -110,16 +146,20 @@ object OptimizeLocalShuffleReader { } /** - * A wrapper of shuffle query stage, which submits one reduce task per mapper to read the shuffle - * files written by one mapper. By doing this, it's very likely to read the shuffle files locally, - * as the shuffle files that a reduce task needs to read are in one node. + * A wrapper of shuffle query stage, which submits one or more reduce tasks per mapper to read the + * shuffle files written by one mapper. By doing this, it's very likely to read the shuffle files + * locally, as the shuffle files that a reduce task needs to read are in one node. * * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the * shuffle exchange node during canonicalization. + * @param partitionStartIndicesPerMapper A mapper usually writes many shuffle blocks, and it's + * better to launch multiple tasks to read shuffle blocks of + * one mapper. This array contains the partition start + * indices for each mapper. */ case class LocalShuffleReaderExec( child: SparkPlan, - advisoryParallelism: Option[Int] = None) extends UnaryExecNode { + partitionStartIndicesPerMapper: Array[Array[Int]]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -136,9 +176,9 @@ case class LocalShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createLocalShuffleRDD(advisoryParallelism) + stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createLocalShuffleRDD(advisoryParallelism) + stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index baca974a6450b..b876183c78ec2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -84,8 +84,8 @@ case class ShuffleExchangeExec( } def createLocalShuffleRDD( - advisoryParallelism: Option[Int] = None): LocalShuffledRowRDD = { - new LocalShuffledRowRDD(shuffleDependency, readMetrics, advisoryParallelism) + partitionStartIndicesPerMapper: Array[Array[Int]]): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndicesPerMapper) } /** 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 d5be45b0cdda0..4fa4dadf7f654 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 @@ -119,19 +119,16 @@ class AdaptiveQueryExecSuite case reader: LocalShuffleReaderExec => reader } assert(localReaders.length == 2) - // The pre-shuffle partition size is [0, 0, 0, 72, 0] - // And the partitionStartIndices is [0, 3, 4] - assert(localReaders(0).advisoryParallelism.get == 3) - // The pre-shuffle partition size is [0, 72, 0, 72, 126] - // And the partitionStartIndices is [0, 1, 2, 3, 4] - assert(localReaders(1).advisoryParallelism.get == 5) - val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // The pre-shuffle partition size is [0, 0, 0, 72, 0] + // And the partitionStartIndices is [0, 3, 4], so advisoryParallelism = 3. // the final parallelism is // math.max(1, advisoryParallelism / numMappers): math.max(1, 3/2) = 1 // and the partitions length is 1 * numMappers = 2 assert(localShuffleRDD0.getPartitions.length == 2) + // The pre-shuffle partition size is [0, 72, 0, 72, 126] + // And the partitionStartIndices is [0, 1, 2, 3, 4], so advisoryParallelism = 5. // the final parallelism is // math.max(1, advisoryParallelism / numMappers): math.max(1, 5/2) = 2 // and the partitions length is 2 * numMappers = 4 @@ -492,7 +489,7 @@ class AdaptiveQueryExecSuite test("Change merge join to broadcast join without local shuffle reader") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ From 6b0e391aa49acd5029d00fefc0c90fcdfdf88cb6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 22 Nov 2019 10:34:26 -0800 Subject: [PATCH 74/83] [SPARK-29427][SQL] Add API to convert RelationalGroupedDataset to KeyValueGroupedDataset ### What changes were proposed in this pull request? This PR proposes to add `as` API to RelationalGroupedDataset. It creates KeyValueGroupedDataset instance using given grouping expressions, instead of a typed function in groupByKey API. Because it can leverage existing columns, it can use existing data partition, if any, when doing operations like cogroup. ### Why are the changes needed? Currently if users want to do cogroup on DataFrames, there is no good way to do except for KeyValueGroupedDataset. 1. KeyValueGroupedDataset ignores existing data partition if any. That is a problem. 2. groupByKey calls typed function to create additional keys. You can not reuse existing columns, if you just need grouping by them. ```scala // df1 and df2 are certainly partitioned and sorted. val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") .repartition($"a").sortWithinPartitions("a") val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a", "b", "c") .repartition($"a").sortWithinPartitions("a") ``` ```scala // This groupBy.as.cogroup won't unnecessarily repartition the data val df3 = df1.groupBy("a").as[Int] .cogroup(df2.groupBy("a").as[Int]) { case (key, data1, data2) => data1.zip(data2).map { p => p._1.getInt(2) + p._2.getInt(2) } } ``` ``` == Physical Plan == *(5) SerializeFromObject [input[0, int, false] AS value#11247] +- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4922/12067092816eec1b6f, a#11209: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [a#11209], [a#11225], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11246: int :- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(a#11209, 5), false, [id=#10218] : +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211] : +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204] +- *(4) Sort [a#11225 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(a#11225, 5), false, [id=#10223] +- *(3) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227] +- *(3) LocalTableScan [_1#11218, _2#11219, _3#11220] ``` ```scala // Current approach creates additional AppendColumns and repartition data again val df4 = df1.groupByKey(r => r.getInt(0)).cogroup(df2.groupByKey(r => r.getInt(0))) { case (key, data1, data2) => data1.zip(data2).map { p => p._1.getInt(2) + p._2.getInt(2) } } ``` ``` == Physical Plan == *(7) SerializeFromObject [input[0, int, false] AS value#11257] +- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4933/138102700737171997, value#11252: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [value#11252], [value#11254], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11256: int :- *(3) Sort [value#11252 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(value#11252, 5), true, [id=#10302] : +- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4930/19529195347ce07f47, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11252] : +- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(a#11209, 5), false, [id=#10297] : +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211] : +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204] +- *(6) Sort [value#11254 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(value#11254, 5), true, [id=#10312] +- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4932/15265288491f0e0c1f, createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11254] +- *(5) Sort [a#11225 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(a#11225, 5), false, [id=#10307] +- *(4) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227] +- *(4) LocalTableScan [_1#11218, _2#11219, _3#11220] ``` ### Does this PR introduce any user-facing change? Yes, this adds a new `as` API to RelationalGroupedDataset. Users can use it to create KeyValueGroupedDataset and do cogroup. ### How was this patch tested? Unit tests. Closes #26509 from viirya/SPARK-29427-2. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/sql/RelationalGroupedDataset.scala | 32 ++++++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 59 +++++++++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 21 +++++++ 3 files changed, 112 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 4d4731870700c..b1ba7d4538732 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -26,6 +26,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -129,6 +130,37 @@ class RelationalGroupedDataset protected[sql]( (inputExpr: Expression) => exprToFunc(inputExpr) } + /** + * Returns a `KeyValueGroupedDataset` where the data is grouped by the grouping expressions + * of current `RelationalGroupedDataset`. + * + * @since 3.0.0 + */ + def as[K: Encoder, T: Encoder]: KeyValueGroupedDataset[K, T] = { + val keyEncoder = encoderFor[K] + val valueEncoder = encoderFor[T] + + // Resolves grouping expressions. + val dummyPlan = Project(groupingExprs.map(alias), LocalRelation(df.logicalPlan.output)) + val analyzedPlan = df.sparkSession.sessionState.analyzer.execute(dummyPlan) + .asInstanceOf[Project] + df.sparkSession.sessionState.analyzer.checkAnalysis(analyzedPlan) + val aliasedGroupings = analyzedPlan.projectList + + // Adds the grouping expressions that are not in base DataFrame into outputs. + val addedCols = aliasedGroupings.filter(g => !df.logicalPlan.outputSet.contains(g.toAttribute)) + val qe = Dataset.ofRows( + df.sparkSession, + Project(df.logicalPlan.output ++ addedCols, df.logicalPlan)).queryExecution + + new KeyValueGroupedDataset( + keyEncoder, + valueEncoder, + qe, + df.logicalPlan.output, + aliasedGroupings.map(_.toAttribute)) + } + /** * (Scala-specific) Compute aggregates by specifying the column names and * aggregate methods. The resulting `DataFrame` will also contain the grouping columns. 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 c79e2e9c76bb9..5983fe63c79ec 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 @@ -30,6 +30,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} @@ -2221,4 +2222,62 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val idTuples = sampled.collect().map(row => row.getLong(0) -> row.getLong(1)) assert(idTuples.length == idTuples.toSet.size) } + + test("groupBy.as") { + val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") + .repartition($"a", $"b").sortWithinPartitions("a", "b") + val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a", "b", "c") + .repartition($"a", $"b").sortWithinPartitions("a", "b") + + implicit val valueEncoder = RowEncoder(df1.schema) + + val df3 = df1.groupBy("a", "b").as[GroupByKey, Row] + .cogroup(df2.groupBy("a", "b").as[GroupByKey, Row]) { case (_, data1, data2) => + data1.zip(data2).map { p => + p._1.getInt(2) + p._2.getInt(2) + } + }.toDF + + checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = df3.queryExecution.executedPlan.collect { + case h: ShuffleExchangeExec => h + } + assert(exchanges.size == 2) + } + + test("groupBy.as: custom grouping expressions") { + val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a1", "b", "c") + .repartition($"a1", $"b").sortWithinPartitions("a1", "b") + val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a1", "b", "c") + .repartition($"a1", $"b").sortWithinPartitions("a1", "b") + + implicit val valueEncoder = RowEncoder(df1.schema) + + val groupedDataset1 = df1.groupBy(($"a1" + 1).as("a"), $"b").as[GroupByKey, Row] + val groupedDataset2 = df2.groupBy(($"a1" + 1).as("a"), $"b").as[GroupByKey, Row] + + val df3 = groupedDataset1 + .cogroup(groupedDataset2) { case (_, data1, data2) => + data1.zip(data2).map { p => + p._1.getInt(2) + p._2.getInt(2) + } + }.toDF + + checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil) + } + + test("groupBy.as: throw AnalysisException for unresolved grouping expr") { + val df = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") + + implicit val valueEncoder = RowEncoder(df.schema) + + val err = intercept[AnalysisException] { + df.groupBy($"d", $"b").as[GroupByKey, Row] + } + assert(err.getMessage.contains("cannot resolve '`d`'")) + } } + +case class GroupByKey(a: Int, b: Int) 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 fc6f087efaefd..9a367d48d2144 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 @@ -1861,6 +1861,27 @@ class DatasetSuite extends QueryTest with SharedSparkSession { } } } + + test("groupBy.as") { + val df1 = Seq(DoubleData(1, "one"), DoubleData(2, "two"), DoubleData( 3, "three")).toDS() + .repartition($"id").sortWithinPartitions("id") + val df2 = Seq(DoubleData(5, "one"), DoubleData(1, "two"), DoubleData( 3, "three")).toDS() + .repartition($"id").sortWithinPartitions("id") + + val df3 = df1.groupBy("id").as[Int, DoubleData] + .cogroup(df2.groupBy("id").as[Int, DoubleData]) { case (key, data1, data2) => + if (key == 1) { + Iterator(DoubleData(key, (data1 ++ data2).foldLeft("")((cur, next) => cur + next.val1))) + } else Iterator.empty + } + checkDataset(df3, DoubleData(1, "onetwo")) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = df3.queryExecution.executedPlan.collect { + case h: ShuffleExchangeExec => h + } + assert(exchanges.size == 2) + } } object AssertExecutionId { From f28eab2de72f7da883b970fc19edd4f569340bd7 Mon Sep 17 00:00:00 2001 From: Norman Maurer Date: Fri, 22 Nov 2019 15:20:54 -0800 Subject: [PATCH 75/83] [SPARK-29971][CORE] Fix buffer leaks in `TransportFrameDecoder/TransportCipher` ### What changes were proposed in this pull request? - Correctly release `ByteBuf` in `TransportCipher` in all cases - Move closing / releasing logic to `handlerRemoved(...)` so we are guaranteed that is always called. - Correctly release `frameBuf` it is not null when the handler is removed (and so also when the channel becomes inactive) ### Why are the changes needed? We need to carefully manage the ownership / lifecycle of `ByteBuf` instances so we don't leak any of these. We did not correctly do this in all cases: - when end up in invalid cipher state. - when partial data was received and the channel is closed before the full frame is decoded Fixes https://github.com/netty/netty/issues/9784. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the newly added UTs. Closes #26609 from normanmaurer/fix_leaks. Authored-by: Norman Maurer Signed-off-by: Marcelo Vanzin --- .../spark/network/crypto/TransportCipher.java | 50 ++++++---- .../util/ByteArrayReadableChannel.java | 24 ++--- .../network/util/TransportFrameDecoder.java | 28 +++++- .../network/crypto/TransportCipherSuite.java | 91 +++++++++++++++++++ 4 files changed, 157 insertions(+), 36 deletions(-) create mode 100644 common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 8995bbc940f63..36ca73f6ac0f0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -90,7 +90,8 @@ CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); } - private CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + @VisibleForTesting + CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { return new CryptoInputStream(cipher, conf, ch, key, new IvParameterSpec(inIv)); } @@ -166,34 +167,45 @@ private static class DecryptionHandler extends ChannelInboundHandlerAdapter { @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { - if (!isCipherValid) { - throw new IOException("Cipher is in invalid state."); - } - byteChannel.feedData((ByteBuf) data); - - byte[] decryptedData = new byte[byteChannel.readableBytes()]; - int offset = 0; - while (offset < decryptedData.length) { - // SPARK-25535: workaround for CRYPTO-141. - try { - offset += cis.read(decryptedData, offset, decryptedData.length - offset); - } catch (InternalError ie) { - isCipherValid = false; - throw ie; + ByteBuf buffer = (ByteBuf) data; + + try { + if (!isCipherValid) { + throw new IOException("Cipher is in invalid state."); + } + byte[] decryptedData = new byte[buffer.readableBytes()]; + byteChannel.feedData(buffer); + + int offset = 0; + while (offset < decryptedData.length) { + // SPARK-25535: workaround for CRYPTO-141. + try { + offset += cis.read(decryptedData, offset, decryptedData.length - offset); + } catch (InternalError ie) { + isCipherValid = false; + throw ie; + } } - } - ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + } finally { + buffer.release(); + } } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // We do the closing of the stream / channel in handlerRemoved(...) as + // this method will be called in all cases: + // + // - when the Channel becomes inactive + // - when the handler is removed from the ChannelPipeline try { if (isCipherValid) { cis.close(); } } finally { - super.channelInactive(ctx); + super.handlerRemoved(ctx); } } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java index 25d103d0e316f..fe461d0b39862 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java @@ -19,23 +19,27 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; import java.nio.channels.ReadableByteChannel; import io.netty.buffer.ByteBuf; public class ByteArrayReadableChannel implements ReadableByteChannel { private ByteBuf data; + private boolean closed; - public int readableBytes() { - return data.readableBytes(); - } - - public void feedData(ByteBuf buf) { + public void feedData(ByteBuf buf) throws ClosedChannelException { + if (closed) { + throw new ClosedChannelException(); + } data = buf; } @Override public int read(ByteBuffer dst) throws IOException { + if (closed) { + throw new ClosedChannelException(); + } int totalRead = 0; while (data.readableBytes() > 0 && dst.remaining() > 0) { int bytesToRead = Math.min(data.readableBytes(), dst.remaining()); @@ -43,20 +47,16 @@ public int read(ByteBuffer dst) throws IOException { totalRead += bytesToRead; } - if (data.readableBytes() == 0) { - data.release(); - } - return totalRead; } @Override - public void close() throws IOException { + public void close() { + closed = true; } @Override public boolean isOpen() { - return true; + return !closed; } - } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java index 1980361a15523..cef0e415aa40a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -184,8 +184,12 @@ private ByteBuf decodeNext() { return null; } - // Reset buf and size for next frame. + return consumeCurrentFrameBuf(); + } + + private ByteBuf consumeCurrentFrameBuf() { ByteBuf frame = frameBuf; + // Reset buf and size for next frame. frameBuf = null; consolidatedFrameBufSize = 0; consolidatedNumComponents = 0; @@ -215,13 +219,9 @@ private ByteBuf nextBufferForFrame(int bytesToRead) { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { - for (ByteBuf b : buffers) { - b.release(); - } if (interceptor != null) { interceptor.channelInactive(); } - frameLenBuf.release(); super.channelInactive(ctx); } @@ -233,6 +233,24 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E super.exceptionCaught(ctx, cause); } + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // Release all buffers that are still in our ownership. + // Doing this in handlerRemoved(...) guarantees that this will happen in all cases: + // - When the Channel becomes inactive + // - When the decoder is removed from the ChannelPipeline + for (ByteBuf b : buffers) { + b.release(); + } + buffers.clear(); + frameLenBuf.release(); + ByteBuf frame = consumeCurrentFrameBuf(); + if (frame != null) { + frame.release(); + } + super.handlerRemoved(ctx); + } + public void setInterceptor(Interceptor interceptor) { Preconditions.checkState(this.interceptor == null, "Already have an interceptor."); this.interceptor = interceptor; diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java new file mode 100644 index 0000000000000..6b2186f73cd0c --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.network.crypto; + +import javax.crypto.spec.SecretKeySpec; +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.commons.crypto.stream.CryptoInputStream; +import org.apache.commons.crypto.stream.CryptoOutputStream; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; +import org.hamcrest.CoreMatchers; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportCipherSuite { + + @Test + public void testBufferNotLeaksOnInternalError() throws IOException { + String algorithm = "TestAlgorithm"; + TransportConf conf = new TransportConf("Test", MapConfigProvider.EMPTY); + TransportCipher cipher = new TransportCipher(conf.cryptoConf(), conf.cipherTransformation(), + new SecretKeySpec(new byte[256], algorithm), new byte[0], new byte[0]) { + + @Override + CryptoOutputStream createOutputStream(WritableByteChannel ch) { + return null; + } + + @Override + CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + CryptoInputStream mockInputStream = mock(CryptoInputStream.class); + when(mockInputStream.read(any(byte[].class), anyInt(), anyInt())) + .thenThrow(new InternalError()); + return mockInputStream; + } + }; + + EmbeddedChannel channel = new EmbeddedChannel(); + cipher.addToChannel(channel); + + ByteBuf buffer = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); + ByteBuf buffer2 = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); + + try { + channel.writeInbound(buffer); + fail("Should have raised InternalError"); + } catch (InternalError expected) { + // expected + assertEquals(0, buffer.refCnt()); + } + + try { + channel.writeInbound(buffer2); + fail("Should have raised an exception"); + } catch (Throwable expected) { + assertThat(expected, CoreMatchers.instanceOf(IOException.class)); + assertEquals(0, buffer2.refCnt()); + } + + // Simulate closing the connection + assertFalse(channel.finish()); + } +} From fc7a37b147d76edcd0de97ca1645dba736317a96 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 23 Nov 2019 17:24:56 +0900 Subject: [PATCH 76/83] [SPARK-30003][SQL] Do not throw stack overflow exception in non-root unknown hint resolution ### What changes were proposed in this pull request? This is rather a followup of https://github.com/apache/spark/pull/25464 (see https://github.com/apache/spark/pull/25464/files#r349543286) It will cause an infinite recursion via mapping children - we should return the hint rather than its parent plan in unknown hint resolution. ### Why are the changes needed? Prevent Stack over flow during hint resolution. ### Does this PR introduce any user-facing change? Yes, it avoids stack overflow exception It was caused by https://github.com/apache/spark/pull/25464 and this is only in the master. No behaviour changes to end users as it happened only in the master. ### How was this patch tested? Unittest was added. Closes #26642 from HyukjinKwon/SPARK-30003. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../apache/spark/sql/catalyst/analysis/ResolveHints.scala | 2 +- .../spark/sql/catalyst/analysis/ResolveHintsSuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index d904ba3aca5d5..5b77d67bd1340 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -223,7 +223,7 @@ object ResolveHints { createRepartition(shuffle = false, hint) case "REPARTITION_BY_RANGE" => createRepartitionByRange(hint) - case _ => plan + case _ => hint } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index cddcddd51e38d..49ab34d2ea3a0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -245,4 +245,11 @@ class ResolveHintsSuite extends AnalysisTest { e => e.getLevel == Level.WARN && e.getRenderedMessage.contains("Unrecognized hint: unknown_hint"))) } + + test("SPARK-30003: Do not throw stack overflow exception in non-root unknown hint resolution") { + checkAnalysis( + Project(testRelation.output, UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE"))), + Project(testRelation.output, testRelation), + caseSensitive = false) + } } From c98e5eb3396a6db92f2420e743afa9ddff319ca2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 10:02:22 -0800 Subject: [PATCH 77/83] [SPARK-29981][BUILD] Add hive-1.2/2.3 profiles ### What changes were proposed in this pull request? This PR aims the followings. - Add two profiles, `hive-1.2` and `hive-2.3` (default) - Validate if we keep the existing combination at least. (Hadoop-2.7 + Hive 1.2 / Hadoop-3.2 + Hive 2.3). For now, we assumes that `hive-1.2` is explicitly used with `hadoop-2.7` and `hive-2.3` with `hadoop-3.2`. The followings are beyond the scope of this PR. - SPARK-29988 Adjust Jenkins jobs for `hive-1.2/2.3` combination - SPARK-29989 Update release-script for `hive-1.2/2.3` combination - SPARK-29991 Support `hive-1.2/2.3` in PR Builder ### Why are the changes needed? This will help to switch our dependencies to update the exposed dependencies. ### Does this PR introduce any user-facing change? This is a dev-only change that the build profile combinations are changed. - `-Phadoop-2.7` => `-Phadoop-2.7 -Phive-1.2` - `-Phadoop-3.2` => `-Phadoop-3.2 -Phive-2.3` ### How was this patch tested? Pass the Jenkins with the dependency check and tests to make it sure we don't change anything for now. - [Jenkins (-Phadoop-2.7 -Phive-1.2)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114192/consoleFull) - [Jenkins (-Phadoop-3.2 -Phive-2.3)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114192/consoleFull) Also, from now, GitHub Action validates the following combinations. ![gha](https://user-images.githubusercontent.com/9700541/69355365-822d5e00-0c36-11ea-93f7-e00e5459e1d0.png) Closes #26619 from dongjoon-hyun/SPARK-29981. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 9 ++++++-- appveyor.yml | 2 +- dev/run-tests.py | 4 ++-- dev/test-dependencies.sh | 11 ++++++--- pom.xml | 43 ++++++++++++++++++++++-------------- sql/hive/pom.xml | 2 +- 6 files changed, 46 insertions(+), 25 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5298492d219f8..30f3272c8b933 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -16,10 +16,15 @@ jobs: matrix: java: [ '1.8', '11' ] hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] + hive: [ 'hive-1.2', 'hive-2.3' ] exclude: - java: '11' hadoop: 'hadoop-2.7' - name: Build Spark with JDK ${{ matrix.java }} and ${{ matrix.hadoop }} + - java: '11' + hive: 'hive-1.2' + - hadoop: 'hadoop-3.2' + hive: 'hive-1.2' + name: Build Spark - JDK${{ matrix.java }}/${{ matrix.hadoop }}/${{ matrix.hive }} steps: - uses: actions/checkout@master @@ -56,7 +61,7 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install rm -rf ~/.m2/repository/org/apache/spark diff --git a/appveyor.yml b/appveyor.yml index 00c688ba18eb6..325fd67abc674 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -53,7 +53,7 @@ install: build_script: # '-Djna.nosys=true' is required to avoid kernel32.dll load failure. # See SPARK-28759. - - cmd: mvn -DskipTests -Psparkr -Phive -Djna.nosys=true package + - cmd: mvn -DskipTests -Psparkr -Phive -Phive-1.2 -Djna.nosys=true package environment: NOT_CRAN: true diff --git a/dev/run-tests.py b/dev/run-tests.py index fc8b7251a85f8..2d52ead06a041 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -283,8 +283,8 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.7": ["-Phadoop-2.7"], - "hadoop3.2": ["-Phadoop-3.2"], + "hadoop2.7": ["-Phadoop-2.7", "-Phive-1.2"], + "hadoop3.2": ["-Phadoop-3.2", "-Phive-2.3"], } if hadoop_version in sbt_maven_hadoop_profiles: diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index cc0292e9c2ea5..7d5725aaf137e 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -67,15 +67,20 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do + if [[ $HADOOP_PROFILE == **hadoop-3** ]]; then + HIVE_PROFILE=hive-2.3 + else + HIVE_PROFILE=hive-1.2 + fi echo "Performing Maven install for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install clean -q + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q echo "Performing Maven validate for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE validate -q + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_PROFILE" mkdir -p dev/pr-deps - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE dependency:build-classpath -pl assembly -am \ + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | rev | cut -d "/" -f 1 | rev | sort \ | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_PROFILE diff --git a/pom.xml b/pom.xml index 44593b78c9a04..a9b544bba158d 100644 --- a/pom.xml +++ b/pom.xml @@ -128,19 +128,19 @@ 3.4.14 2.7.1 0.4.2 - org.spark-project.hive - + org.apache.hive + core - 1.2.1.spark2 + 2.3.6 2.3.6 - 1.2.1 + 2.3.5 2.3.1 10.12.1.1 1.10.1 1.5.7 - nohive + com.twitter 1.6.0 9.4.18.v20190429 @@ -181,7 +181,7 @@ 3.8.1 2.6.2 - 3.2.10 + 4.1.17 3.0.15 2.29 2.10.5 @@ -228,7 +228,7 @@ --> compile compile - ${hive.deps.scope} + provided compile compile test @@ -2921,16 +2921,27 @@ 3.2.0 2.13.0 - org.apache.hive - core - ${hive23.version} - 2.3.5 - - provided - - 4.1.17 + + + + hive-1.2 + + org.spark-project.hive + + + 1.2.1.spark2 + + 1.2.1 + ${hive.deps.scope} + nohive + 3.2.10 + + + + + hive-2.3 + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 882cabd347073..69961acd510d8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -209,7 +209,7 @@ - hadoop-3.2 + hive-2.3 ${hive.group} From 6625b69027661d34352aa22e75dee6f31d069b41 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 12:50:50 -0800 Subject: [PATCH 78/83] [SPARK-29981][BUILD][FOLLOWUP] Change hive.version.short ### What changes were proposed in this pull request? This is a follow-up according to liancheng 's advice. - https://github.com/apache/spark/pull/26619#discussion_r349326090 ### Why are the changes needed? Previously, we chose the full version to be carefully. As of today, it seems that `Apache Hive 2.3` branch seems to become stable. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the compile combination on GitHub Action. 1. hadoop-2.7/hive-1.2/JDK8 2. hadoop-2.7/hive-2.3/JDK8 3. hadoop-3.2/hive-2.3/JDK8 4. hadoop-3.2/hive-2.3/JDK11 Also, pass the Jenkins with `hadoop-2.7` and `hadoop-3.2` for (1) and (4). (2) and (3) is not ready in Jenkins. Closes #26645 from dongjoon-hyun/SPARK-RENAME-HIVE-DIRECTORY. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 4 ++-- .../spark/sql/execution/datasources/orc/OrcColumnVector.java | 0 .../spark/sql/execution/datasources/orc/OrcFilters.scala | 0 .../spark/sql/execution/datasources/orc/OrcShimUtils.scala | 0 .../spark/sql/execution/datasources/orc/OrcFilterSuite.scala | 0 .../spark/sql/execution/datasources/orc/OrcColumnVector.java | 0 .../spark/sql/execution/datasources/orc/OrcFilters.scala | 0 .../spark/sql/execution/datasources/orc/OrcShimUtils.scala | 0 .../spark/sql/execution/datasources/orc/OrcFilterSuite.scala | 0 sql/hive-thriftserver/{v1.2.1 => v1.2}/if/TCLIService.thrift | 0 .../org/apache/hive/service/cli/thrift/TArrayTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TBinaryColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TBoolColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TBoolValue.java | 0 .../java/org/apache/hive/service/cli/thrift/TByteColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TByteValue.java | 0 .../java/org/apache/hive/service/cli/thrift/TCLIService.java | 0 .../apache/hive/service/cli/thrift/TCLIServiceConstants.java | 0 .../hive/service/cli/thrift/TCancelDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TCancelDelegationTokenResp.java | 0 .../apache/hive/service/cli/thrift/TCancelOperationReq.java | 0 .../apache/hive/service/cli/thrift/TCancelOperationResp.java | 0 .../apache/hive/service/cli/thrift/TCloseOperationReq.java | 0 .../apache/hive/service/cli/thrift/TCloseOperationResp.java | 0 .../org/apache/hive/service/cli/thrift/TCloseSessionReq.java | 0 .../org/apache/hive/service/cli/thrift/TCloseSessionResp.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TColumnDesc.java | 0 .../java/org/apache/hive/service/cli/thrift/TColumnValue.java | 0 .../org/apache/hive/service/cli/thrift/TDoubleColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TDoubleValue.java | 0 .../apache/hive/service/cli/thrift/TExecuteStatementReq.java | 0 .../apache/hive/service/cli/thrift/TExecuteStatementResp.java | 0 .../org/apache/hive/service/cli/thrift/TFetchOrientation.java | 0 .../org/apache/hive/service/cli/thrift/TFetchResultsReq.java | 0 .../org/apache/hive/service/cli/thrift/TFetchResultsResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetCatalogsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetCatalogsResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetColumnsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetColumnsResp.java | 0 .../hive/service/cli/thrift/TGetDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TGetDelegationTokenResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetFunctionsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetFunctionsResp.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoReq.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoResp.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoType.java | 0 .../org/apache/hive/service/cli/thrift/TGetInfoValue.java | 0 .../hive/service/cli/thrift/TGetOperationStatusReq.java | 0 .../hive/service/cli/thrift/TGetOperationStatusResp.java | 0 .../hive/service/cli/thrift/TGetResultSetMetadataReq.java | 0 .../hive/service/cli/thrift/TGetResultSetMetadataResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetSchemasReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetSchemasResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTableTypesReq.java | 0 .../apache/hive/service/cli/thrift/TGetTableTypesResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTablesReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetTablesResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java | 0 .../org/apache/hive/service/cli/thrift/THandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/thrift/TI16Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI16Value.java | 0 .../java/org/apache/hive/service/cli/thrift/TI32Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI32Value.java | 0 .../java/org/apache/hive/service/cli/thrift/TI64Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI64Value.java | 0 .../org/apache/hive/service/cli/thrift/TMapTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TOpenSessionReq.java | 0 .../org/apache/hive/service/cli/thrift/TOpenSessionResp.java | 0 .../org/apache/hive/service/cli/thrift/TOperationHandle.java | 0 .../org/apache/hive/service/cli/thrift/TOperationState.java | 0 .../org/apache/hive/service/cli/thrift/TOperationType.java | 0 .../apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TProtocolVersion.java | 0 .../hive/service/cli/thrift/TRenewDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TRenewDelegationTokenResp.java | 0 .../src/gen/java/org/apache/hive/service/cli/thrift/TRow.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TRowSet.java | 0 .../org/apache/hive/service/cli/thrift/TSessionHandle.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TStatus.java | 0 .../java/org/apache/hive/service/cli/thrift/TStatusCode.java | 0 .../org/apache/hive/service/cli/thrift/TStringColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TStringValue.java | 0 .../org/apache/hive/service/cli/thrift/TStructTypeEntry.java | 0 .../java/org/apache/hive/service/cli/thrift/TTableSchema.java | 0 .../java/org/apache/hive/service/cli/thrift/TTypeDesc.java | 0 .../java/org/apache/hive/service/cli/thrift/TTypeEntry.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TTypeId.java | 0 .../apache/hive/service/cli/thrift/TTypeQualifierValue.java | 0 .../org/apache/hive/service/cli/thrift/TTypeQualifiers.java | 0 .../org/apache/hive/service/cli/thrift/TUnionTypeEntry.java | 0 .../apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java | 0 .../main/java/org/apache/hive/service/AbstractService.java | 0 .../main/java/org/apache/hive/service/CompositeService.java | 0 .../src/main/java/org/apache/hive/service/CookieSigner.java | 0 .../main/java/org/apache/hive/service/ServiceOperations.java | 0 .../src/main/java/org/apache/hive/service/ServiceUtils.java | 0 .../java/org/apache/hive/service/auth/HiveAuthFactory.java | 0 .../main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 0 .../java/org/apache/hive/service/auth/KerberosSaslHelper.java | 0 .../java/org/apache/hive/service/auth/PlainSaslHelper.java | 0 .../org/apache/hive/service/auth/TSetIpAddressProcessor.java | 0 .../src/main/java/org/apache/hive/service/cli/CLIService.java | 0 .../src/main/java/org/apache/hive/service/cli/Column.java | 0 .../main/java/org/apache/hive/service/cli/ColumnBasedSet.java | 0 .../java/org/apache/hive/service/cli/ColumnDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/ColumnValue.java | 0 .../org/apache/hive/service/cli/EmbeddedCLIServiceClient.java | 0 .../java/org/apache/hive/service/cli/FetchOrientation.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoType.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoValue.java | 0 .../src/main/java/org/apache/hive/service/cli/Handle.java | 0 .../java/org/apache/hive/service/cli/HandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/HiveSQLException.java | 0 .../main/java/org/apache/hive/service/cli/ICLIService.java | 0 .../java/org/apache/hive/service/cli/OperationHandle.java | 0 .../main/java/org/apache/hive/service/cli/OperationState.java | 0 .../main/java/org/apache/hive/service/cli/OperationType.java | 0 .../java/org/apache/hive/service/cli/PatternOrIdentifier.java | 0 .../main/java/org/apache/hive/service/cli/RowBasedSet.java | 0 .../src/main/java/org/apache/hive/service/cli/RowSet.java | 0 .../main/java/org/apache/hive/service/cli/RowSetFactory.java | 0 .../main/java/org/apache/hive/service/cli/SessionHandle.java | 0 .../main/java/org/apache/hive/service/cli/TableSchema.java | 0 .../src/main/java/org/apache/hive/service/cli/Type.java | 0 .../main/java/org/apache/hive/service/cli/TypeDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/TypeQualifiers.java | 0 .../hive/service/cli/operation/ClassicTableTypeMapping.java | 0 .../hive/service/cli/operation/ExecuteStatementOperation.java | 0 .../hive/service/cli/operation/GetCatalogsOperation.java | 0 .../hive/service/cli/operation/GetColumnsOperation.java | 0 .../hive/service/cli/operation/GetFunctionsOperation.java | 0 .../hive/service/cli/operation/GetSchemasOperation.java | 0 .../hive/service/cli/operation/GetTableTypesOperation.java | 0 .../apache/hive/service/cli/operation/GetTablesOperation.java | 0 .../hive/service/cli/operation/GetTypeInfoOperation.java | 0 .../hive/service/cli/operation/HiveCommandOperation.java | 0 .../hive/service/cli/operation/HiveTableTypeMapping.java | 0 .../apache/hive/service/cli/operation/MetadataOperation.java | 0 .../java/org/apache/hive/service/cli/operation/Operation.java | 0 .../apache/hive/service/cli/operation/OperationManager.java | 0 .../org/apache/hive/service/cli/operation/SQLOperation.java | 0 .../apache/hive/service/cli/operation/TableTypeMapping.java | 0 .../java/org/apache/hive/service/cli/session/HiveSession.java | 0 .../org/apache/hive/service/cli/session/HiveSessionBase.java | 0 .../org/apache/hive/service/cli/session/HiveSessionImpl.java | 0 .../hive/service/cli/session/HiveSessionImplwithUGI.java | 0 .../org/apache/hive/service/cli/session/SessionManager.java | 0 .../hive/service/cli/thrift/ThriftBinaryCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 0 .../hive/service/cli/thrift/ThriftCLIServiceClient.java | 0 .../apache/hive/service/cli/thrift/ThriftHttpCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftHttpServlet.java | 0 .../main/java/org/apache/hive/service/server/HiveServer2.java | 0 .../apache/hive/service/server/ThreadWithGarbageCleanup.java | 0 .../spark/sql/hive/thriftserver/ThriftserverShimUtils.scala | 0 sql/hive-thriftserver/{v2.3.5 => v2.3}/if/TCLIService.thrift | 0 .../org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java | 0 .../org/apache/hive/service/rpc/thrift/TBinaryColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TBoolColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TBoolValue.java | 0 .../java/org/apache/hive/service/rpc/thrift/TByteColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TByteValue.java | 0 .../java/org/apache/hive/service/rpc/thrift/TCLIService.java | 0 .../apache/hive/service/rpc/thrift/TCLIServiceConstants.java | 0 .../hive/service/rpc/thrift/TCancelDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TCancelDelegationTokenResp.java | 0 .../apache/hive/service/rpc/thrift/TCancelOperationReq.java | 0 .../apache/hive/service/rpc/thrift/TCancelOperationResp.java | 0 .../apache/hive/service/rpc/thrift/TCloseOperationReq.java | 0 .../apache/hive/service/rpc/thrift/TCloseOperationResp.java | 0 .../org/apache/hive/service/rpc/thrift/TCloseSessionReq.java | 0 .../org/apache/hive/service/rpc/thrift/TCloseSessionResp.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TColumnDesc.java | 0 .../java/org/apache/hive/service/rpc/thrift/TColumnValue.java | 0 .../org/apache/hive/service/rpc/thrift/TDoubleColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TDoubleValue.java | 0 .../apache/hive/service/rpc/thrift/TExecuteStatementReq.java | 0 .../apache/hive/service/rpc/thrift/TExecuteStatementResp.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchOrientation.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchResultsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchResultsResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetColumnsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetColumnsResp.java | 0 .../apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java | 0 .../hive/service/rpc/thrift/TGetCrossReferenceResp.java | 0 .../hive/service/rpc/thrift/TGetDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TGetDelegationTokenResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoType.java | 0 .../org/apache/hive/service/rpc/thrift/TGetInfoValue.java | 0 .../hive/service/rpc/thrift/TGetOperationStatusReq.java | 0 .../hive/service/rpc/thrift/TGetOperationStatusResp.java | 0 .../apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java | 0 .../apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java | 0 .../hive/service/rpc/thrift/TGetResultSetMetadataReq.java | 0 .../hive/service/rpc/thrift/TGetResultSetMetadataResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetSchemasReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetSchemasResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java | 0 .../apache/hive/service/rpc/thrift/TGetTableTypesResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTablesReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTablesResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java | 0 .../org/apache/hive/service/rpc/thrift/THandleIdentifier.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI16Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI16Value.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI32Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI32Value.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI64Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI64Value.java | 0 .../apache/hive/service/rpc/thrift/TJobExecutionStatus.java | 0 .../org/apache/hive/service/rpc/thrift/TMapTypeEntry.java | 0 .../org/apache/hive/service/rpc/thrift/TOpenSessionReq.java | 0 .../org/apache/hive/service/rpc/thrift/TOpenSessionResp.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationHandle.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationState.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationType.java | 0 .../apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java | 0 .../apache/hive/service/rpc/thrift/TProgressUpdateResp.java | 0 .../org/apache/hive/service/rpc/thrift/TProtocolVersion.java | 0 .../hive/service/rpc/thrift/TRenewDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TRenewDelegationTokenResp.java | 0 .../src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java | 0 .../org/apache/hive/service/rpc/thrift/TSessionHandle.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TStatus.java | 0 .../java/org/apache/hive/service/rpc/thrift/TStatusCode.java | 0 .../org/apache/hive/service/rpc/thrift/TStringColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TStringValue.java | 0 .../org/apache/hive/service/rpc/thrift/TStructTypeEntry.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTableSchema.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTypeDesc.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTypeEntry.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java | 0 .../apache/hive/service/rpc/thrift/TTypeQualifierValue.java | 0 .../org/apache/hive/service/rpc/thrift/TTypeQualifiers.java | 0 .../org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java | 0 .../apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java | 0 .../main/java/org/apache/hive/service/AbstractService.java | 0 .../main/java/org/apache/hive/service/CompositeService.java | 0 .../src/main/java/org/apache/hive/service/CookieSigner.java | 0 .../main/java/org/apache/hive/service/ServiceOperations.java | 0 .../src/main/java/org/apache/hive/service/ServiceUtils.java | 0 .../java/org/apache/hive/service/auth/HiveAuthFactory.java | 0 .../main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 0 .../java/org/apache/hive/service/auth/KerberosSaslHelper.java | 0 .../java/org/apache/hive/service/auth/PlainSaslHelper.java | 0 .../org/apache/hive/service/auth/TSetIpAddressProcessor.java | 0 .../src/main/java/org/apache/hive/service/cli/CLIService.java | 0 .../main/java/org/apache/hive/service/cli/ColumnBasedSet.java | 0 .../java/org/apache/hive/service/cli/ColumnDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/ColumnValue.java | 0 .../java/org/apache/hive/service/cli/FetchOrientation.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoType.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoValue.java | 0 .../src/main/java/org/apache/hive/service/cli/Handle.java | 0 .../java/org/apache/hive/service/cli/HandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/HiveSQLException.java | 0 .../main/java/org/apache/hive/service/cli/ICLIService.java | 0 .../java/org/apache/hive/service/cli/OperationHandle.java | 0 .../main/java/org/apache/hive/service/cli/OperationState.java | 0 .../main/java/org/apache/hive/service/cli/OperationType.java | 0 .../main/java/org/apache/hive/service/cli/RowBasedSet.java | 0 .../src/main/java/org/apache/hive/service/cli/RowSet.java | 0 .../main/java/org/apache/hive/service/cli/RowSetFactory.java | 0 .../main/java/org/apache/hive/service/cli/SessionHandle.java | 0 .../main/java/org/apache/hive/service/cli/TableSchema.java | 0 .../main/java/org/apache/hive/service/cli/TypeDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/TypeQualifiers.java | 0 .../hive/service/cli/operation/ClassicTableTypeMapping.java | 0 .../hive/service/cli/operation/ExecuteStatementOperation.java | 0 .../hive/service/cli/operation/GetCatalogsOperation.java | 0 .../hive/service/cli/operation/GetColumnsOperation.java | 0 .../service/cli/operation/GetCrossReferenceOperation.java | 0 .../hive/service/cli/operation/GetFunctionsOperation.java | 0 .../hive/service/cli/operation/GetPrimaryKeysOperation.java | 0 .../hive/service/cli/operation/GetSchemasOperation.java | 0 .../hive/service/cli/operation/GetTableTypesOperation.java | 0 .../apache/hive/service/cli/operation/GetTablesOperation.java | 0 .../hive/service/cli/operation/GetTypeInfoOperation.java | 0 .../hive/service/cli/operation/HiveCommandOperation.java | 0 .../hive/service/cli/operation/HiveTableTypeMapping.java | 0 .../apache/hive/service/cli/operation/MetadataOperation.java | 0 .../java/org/apache/hive/service/cli/operation/Operation.java | 0 .../apache/hive/service/cli/operation/OperationManager.java | 0 .../org/apache/hive/service/cli/operation/SQLOperation.java | 0 .../apache/hive/service/cli/operation/TableTypeMapping.java | 0 .../java/org/apache/hive/service/cli/session/HiveSession.java | 0 .../org/apache/hive/service/cli/session/HiveSessionBase.java | 0 .../hive/service/cli/session/HiveSessionHookContext.java | 0 .../hive/service/cli/session/HiveSessionHookContextImpl.java | 0 .../org/apache/hive/service/cli/session/HiveSessionImpl.java | 0 .../hive/service/cli/session/HiveSessionImplwithUGI.java | 0 .../org/apache/hive/service/cli/session/SessionManager.java | 0 .../hive/service/cli/thrift/ThriftBinaryCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 0 .../hive/service/cli/thrift/ThriftCLIServiceClient.java | 0 .../apache/hive/service/cli/thrift/ThriftHttpCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftHttpServlet.java | 0 .../main/java/org/apache/hive/service/server/HiveServer2.java | 0 .../apache/hive/service/server/ThreadWithGarbageCleanup.java | 0 .../spark/sql/hive/thriftserver/ThriftserverShimUtils.scala | 0 311 files changed, 2 insertions(+), 2 deletions(-) rename sql/core/{v1.2.1 => v1.2}/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java (100%) rename sql/core/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala (100%) rename sql/core/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala (100%) rename sql/core/{v1.2.1 => v1.2}/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/if/TCLIService.thrift (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/AbstractService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/CompositeService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/CookieSigner.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/ServiceOperations.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/ServiceUtils.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/CLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/FetchOrientation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/GetInfoType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/GetInfoValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Handle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/HiveSQLException.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ICLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationState.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowBasedSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/SessionHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TableSchema.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Type.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/Operation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSession.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/SessionManager.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/server/HiveServer2.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/if/TCLIService.thrift (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/AbstractService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/CompositeService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/CookieSigner.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/ServiceOperations.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/ServiceUtils.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/CLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/FetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/GetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/GetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/Handle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/HiveSQLException.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ICLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationState.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/SessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TableSchema.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/Operation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSession.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/SessionManager.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/server/HiveServer2.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala (100%) diff --git a/pom.xml b/pom.xml index a9b544bba158d..825a227e8669b 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ 2.3.6 2.3.6 - 2.3.5 + 2.3 2.3.1 10.12.1.1 @@ -2932,7 +2932,7 @@ 1.2.1.spark2 - 1.2.1 + 1.2 ${hive.deps.scope} nohive 3.2.10 diff --git a/sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala 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/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 100% rename from sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala diff --git a/sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala 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/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 100% rename from sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala diff --git a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift rename to sql/hive-thriftserver/v1.2/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala rename to sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala diff --git a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift b/sql/hive-thriftserver/v2.3/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift rename to sql/hive-thriftserver/v2.3/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala rename to sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala From 6cd6d5f57ed53aed234b169ad5be3e133dab608f Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 23 Nov 2019 16:16:24 -0800 Subject: [PATCH 79/83] [SPARK-29970][WEBUI] Preserver open/close state of Timelineview ### What changes were proposed in this pull request? Fix a bug related to Timelineview that does not preserve open/close state properly. ### Why are the changes needed? To preserve open/close state is originally intended but it doesn't work. ### Does this PR introduce any user-facing change? Yes. open/close state for Timeineview is to be preserved so if you open Timelineview in Stage page and go to another page, and then go back to Stage page, Timelineview should keep open. ### How was this patch tested? Manual test. Closes #26607 from sarutak/fix-timeline-view-state. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ui/static/timeline-view.js | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index 705a08f0293d3..b2cd616791734 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -83,8 +83,9 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { }); } -$(function (){ - if (window.localStorage.getItem("expand-application-timeline") == "true") { +$(function () { + if ($("span.expand-application-timeline").length && + window.localStorage.getItem("expand-application-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-application-timeline", "false"); $("span.expand-application-timeline").trigger('click'); @@ -159,8 +160,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { }); } -$(function (){ - if (window.localStorage.getItem("expand-job-timeline") == "true") { +$(function () { + if ($("span.expand-job-timeline").length && + window.localStorage.getItem("expand-job-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-job-timeline", "false"); $("span.expand-job-timeline").trigger('click'); @@ -226,8 +228,9 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma }); } -$(function (){ - if (window.localStorage.getItem("expand-task-assignment-timeline") == "true") { +$(function () { + if ($("span.expand-task-assignment-timeline").length && + window.localStorage.getItem("expand-task-assignment-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-task-assignment-timeline", "false"); $("span.expand-task-assignment-timeline").trigger('click'); From 6898be9f02828fabe3c417244f63e0fc79ba58d3 Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Sat, 23 Nov 2019 18:09:02 -0800 Subject: [PATCH 80/83] [SPARK-29681][WEBUI] Support column sorting in Environment tab ### What changes were proposed in this pull request? Add extra classnames to table headers in EnvironmentPage tables in Spark UI. ### Why are the changes needed? SparkUI uses sorttable.js to provide the sort functionality in different tables. This library tries to guess the datatype of each column during initialization phase - numeric/alphanumeric etc and uses it to sort the columns whenever user clicks on a column. That way it guesses incorrect data type for environment tab. sorttable.js has way to hint the datatype of table columns explicitly. This is done by passing custom HTML class attribute. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested sorting in tables in Environment tab in Spark UI. ![Annotation 2019-11-22 154058](https://user-images.githubusercontent.com/2551496/69417432-a8d6bc00-0d3e-11ea-865b-f8017976c6f4.png) ![Annotation 2019-11-22 153600](https://user-images.githubusercontent.com/2551496/69417433-a8d6bc00-0d3e-11ea-9a75-8e1f4d66107e.png) ![Annotation 2019-11-22 153841](https://user-images.githubusercontent.com/2551496/69417435-a96f5280-0d3e-11ea-85f6-9f61b015e161.png) Closes #26638 from prakharjain09/SPARK-29681-SPARK-UI-SORT. Authored-by: Prakhar Jain Signed-off-by: Dongjoon Hyun --- .../apache/spark/ui/env/EnvironmentPage.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 76537afd81ce0..c6eb461ad601c 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -39,15 +39,20 @@ private[ui] class EnvironmentPage( "Scala Version" -> appEnv.runtime.scalaVersion) val runtimeInformationTable = UIUtils.listingTable( - propertyHeader, jvmRow, jvmInformation.toSeq.sorted, fixedWidth = true) + propertyHeader, jvmRow, jvmInformation.toSeq.sorted, fixedWidth = true, + headerClasses = headerClasses) val sparkPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.sparkProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.sparkProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val hadoopPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.hadoopProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.hadoopProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val systemPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.systemProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.systemProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val classpathEntriesTable = UIUtils.listingTable( - classPathHeaders, classPathRow, appEnv.classpathEntries.sorted, fixedWidth = true) + classPathHeader, classPathRow, appEnv.classpathEntries.sorted, fixedWidth = true, + headerClasses = headerClasses) val content = {kv._1}{kv._2} private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} From 564826d960f523036e505545dd008f78c12df7ee Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sat, 23 Nov 2019 19:34:19 -0800 Subject: [PATCH 81/83] [SPARK-28812][SQL][DOC] Document SHOW PARTITIONS in SQL Reference ### What changes were proposed in this pull request? Document SHOW PARTITIONS statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before** **After** ![image](https://user-images.githubusercontent.com/14225158/69405056-89468180-0cb3-11ea-8eb7-93046eaf551c.png) ![image](https://user-images.githubusercontent.com/14225158/69405067-93688000-0cb3-11ea-810a-11cab9e4a041.png) ![image](https://user-images.githubusercontent.com/14225158/69405120-c01c9780-0cb3-11ea-91c0-91eeaa9238a0.png) Closes #26635 from dilipbiswal/show_partitions. Authored-by: Dilip Biswal Signed-off-by: Dongjoon Hyun --- docs/sql-ref-syntax-aux-show-partitions.md | 83 +++++++++++++++++++++- 1 file changed, 82 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md index c6499de9cbb9e..216f3f0d679ec 100644 --- a/docs/sql-ref-syntax-aux-show-partitions.md +++ b/docs/sql-ref-syntax-aux-show-partitions.md @@ -18,5 +18,86 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +### Description -**This page is under construction** +The `SHOW PARTITIONS` statement is used to list partitions of a table. An optional +partition spec may be specified to return the partitions matching the supplied +partition spec. + +### Syntax +{% highlight sql %} +SHOW PARTITIONS table_name + [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) ] +{% endhighlight %} + +### Parameters +
+
table_name
+
The name of an existing table.
+
+
+
PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
+
An optional parameter that specifies a comma separated list of key and value pairs for + partitions. When specified, the partitions that match the partition spec are returned.
+
+ +### Examples +{% highlight sql %} +-- create a partitioned table and insert a few rows. +USE salesdb; +CREATE TABLE customer(id INT, name STRING) PARTITIONED BY (state STRING, city STRING); +INSERT INTO customer PARTITION (state = 'CA', city = 'Fremont') VALUES (100, 'John'); +INSERT INTO customer PARTITION (state = 'CA', city = 'San Jose') VALUES (200, 'Marry'); +INSERT INTO customer PARTITION (state = 'AZ', city = 'Peoria') VALUES (300, 'Daniel'); + +-- Lists all partitions for table `customer` +SHOW PARTITIONS customer; + +----------------------+ + |partition | + +----------------------+ + |state=AZ/city=Peoria | + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Lists all partitions for the qualified table `customer` +SHOW PARTITIONS salesdb.customer; + +----------------------+ + |partition | + +----------------------+ + |state=AZ/city=Peoria | + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Specify a full partition spec to list specific partition +SHOW PARTITIONS customer PARTITION (state = 'CA', city = 'Fremont'); + +---------------------+ + |partition | + +---------------------+ + |state=CA/city=Fremont| + +---------------------+ + +-- Specify a partial partition spec to list the specific partitions +SHOW PARTITIONS customer PARTITION (state = 'CA'); + +----------------------+ + |partition | + +----------------------+ + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Specify a partial spec to list specific partition +SHOW PARTITIONS customer PARTITION (city = 'San Jose'); + +----------------------+ + |partition | + +----------------------+ + |state=CA/city=San Jose| + +----------------------+ +{% endhighlight %} + +### Related statements +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [INSERT STATEMENT](sql-ref-syntax-dml-insert.html) +- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) +- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) From 13338eaa9509fa526438067aeb4cd41f4048931a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 19:53:52 -0800 Subject: [PATCH 82/83] [SPARK-29554][SQL][FOLLOWUP] Rename Version to SparkVersion ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/26209 . This renames class `Version` to class `SparkVersion`. ### Why are the changes needed? According to the review comment, this uses more specific class name. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26647 from dongjoon-hyun/SPARK-29554. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/expressions/misc.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c0e0330ff1e14..7cc64d43858c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -490,7 +490,7 @@ object FunctionRegistry { expression[CurrentDatabase]("current_database"), expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method"), - expression[Version]("version"), + expression[SparkVersion]("version"), expression[TypeOf]("typeof"), // grouping sets diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index def81f8dfb72b..f576873829f27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -169,7 +169,7 @@ case class Uuid(randomSeed: Option[Long] = None) extends LeafExpression with Sta usage = """_FUNC_() - Returns the Spark version. The string contains 2 fields, the first being a release version and the second being a git revision.""", since = "3.0.0") // scalastyle:on line.size.limit -case class Version() extends LeafExpression with CodegenFallback { +case class SparkVersion() extends LeafExpression with CodegenFallback { override def nullable: Boolean = false override def foldable: Boolean = true override def dataType: DataType = StringType From a60da23d648c186dd17f43611226892db2508b09 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 22:34:21 -0800 Subject: [PATCH 83/83] [SPARK-30007][INFRA] Publish snapshot/release artifacts with `-Phive-2.3` only ### What changes were proposed in this pull request? This PR aims to add `-Phive-2.3` to publish profiles. Since Apache Spark 3.0.0, Maven artifacts will be publish with Apache Hive 2.3 profile only. This PR also will recover `SNAPSHOT` publishing Jenkins job. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20Packaging/job/spark-master-maven-snapshots/ We will provide the pre-built distributions (with Hive 1.2.1 also) like Apache Spark 2.4. SPARK-29989 will update the release script to generate all combinations. ### Why are the changes needed? This will reduce the explicit dependency on the illegitimate Hive fork in Maven repository. ### Does this PR introduce any user-facing change? Yes, but this is dev only changes. ### How was this patch tested? Manual. Closes #26648 from dongjoon-hyun/SPARK-30007. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index f1069d4490b5a..99c4b20102929 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -138,7 +138,8 @@ fi # Hive-specific profiles for some builds HIVE_PROFILES="-Phive -Phive-thriftserver" # Profiles for publishing snapshots and release to Maven Central -PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" +# We use Apache Hive 2.3 for publishing +PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Phive-2.3 -Pspark-ganglia-lgpl -Pkinesis-asl" # Profiles for building binary releases BASE_RELEASE_PROFILES="$BASE_PROFILES -Psparkr"