From 513d51a2c5dd2c7ff2c2fadc26ec122883372be1 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 9 Sep 2020 08:53:44 -0500 Subject: [PATCH] [SPARK-32808][SQL] Fix some test cases of `sql/core` module in scala 2.13 ### What changes were proposed in this pull request? The purpose of this pr is to partial resolve [SPARK-32808](https://issues.apache.org/jira/browse/SPARK-32808), total of 26 failed test cases were fixed, the related suite as follow: - `StreamingAggregationSuite` related test cases (2 FAILED -> Pass) - `GeneratorFunctionSuite` related test cases (2 FAILED -> Pass) - `UDFSuite` related test cases (2 FAILED -> Pass) - `SQLQueryTestSuite` related test cases (5 FAILED -> Pass) - `WholeStageCodegenSuite` related test cases (1 FAILED -> Pass) - `DataFrameSuite` related test cases (3 FAILED -> Pass) - `OrcV1QuerySuite\OrcV2QuerySuite` related test cases (4 FAILED -> Pass) - `ExpressionsSchemaSuite` related test cases (1 FAILED -> Pass) - `DataFrameStatSuite` related test cases (1 FAILED -> Pass) - `JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuite` related test cases (6 FAILED -> Pass) The main change of this pr as following: - Fix Scala 2.13 compilation problems in `ShuffleBlockFetcherIterator` and `Analyzer` - Specified `Seq` to `scala.collection.Seq` in `objects.scala` and `GenericArrayData` because internal use `Seq` maybe `mutable.ArraySeq` and not easy to call `.toSeq` - Should specified `Seq` to `scala.collection.Seq` when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but `Seq` is `immutable.Seq` in Scala 2.13 - Use a compatible way to let `+` and `-` method of `Decimal` having the same behavior in Scala 2.12 and Scala 2.13 - Call `toList` in `RelationalGroupedDataset.toDF` method when `groupingExprs` is `Stream` type because `Stream` can't serialize in Scala 2.13 - Add a manual sort to `classFunsMap` in `ExpressionsSchemaSuite` because `Iterable.groupBy` in Scala 2.13 has different result with `TraversableLike.groupBy` in Scala 2.12 ### Why are the changes needed? We need to support a Scala 2.13 build. ### Does this PR introduce _any_ user-facing change? Should specified `Seq` to `scala.collection.Seq` when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but the `Seq` is `immutable.Seq` in Scala 2.13 ### How was this patch tested? - Scala 2.12: Pass the Jenkins or GitHub Action - Scala 2.13: Do the following: ``` dev/change-scala-version.sh 2.13 mvn clean install -DskipTests -pl sql/core -Pscala-2.13 -am mvn test -pl sql/core -Pscala-2.13 ``` **Before** ``` Tests: succeeded 8166, failed 319, canceled 1, ignored 52, pending 0 *** 319 TESTS FAILED *** ``` **After** ``` Tests: succeeded 8204, failed 286, canceled 1, ignored 52, pending 0 *** 286 TESTS FAILED *** ``` Closes #29660 from LuciferYang/SPARK-32808. Authored-by: yangjie01 Signed-off-by: Sean Owen --- .../storage/ShuffleBlockFetcherIterator.scala | 2 +- .../expressions/objects/objects.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../sql/catalyst/util/GenericArrayData.scala | 8 +++- .../org/apache/spark/sql/types/Decimal.scala | 4 +- .../spark/sql/RelationalGroupedDataset.scala | 6 ++- .../spark/sql/execution/GenerateExec.scala | 2 +- .../sql-functions/sql-expression-schema.md | 46 +++++++++---------- .../apache/spark/sql/DataFrameStatSuite.scala | 4 +- .../spark/sql/ExpressionsSchemaSuite.scala | 4 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 6 +-- 12 files changed, 51 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 57b6a38ae6..e3b3fc5cc4 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -495,7 +495,7 @@ final class ShuffleBlockFetcherIterator( hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) { case Success(dirsByExecId) => fetchMultipleHostLocalBlocks( - hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains), + hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains).toMap, dirsByExecId, cached = false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 4f6a587c00..7933d05c8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -842,7 +842,7 @@ case class MapObjects private( val array = ctx.freshName("array") val determineCollectionType = inputData.dataType match { case ObjectType(cls) if cls == classOf[Object] => - val seqClass = classOf[Seq[_]].getName + val seqClass = classOf[scala.collection.Seq[_]].getName s""" $seqClass $seq = null; $elementJavaType[] $array = null; @@ -1755,7 +1755,7 @@ case class ValidateExternalType(child: Expression, expected: DataType) Seq(classOf[java.math.BigDecimal], classOf[scala.math.BigDecimal], classOf[Decimal]) .map(cls => s"$obj instanceof ${cls.getName}").mkString(" || ") case _: ArrayType => - s"$obj.getClass().isArray() || $obj instanceof ${classOf[Seq[_]].getName}" + s"$obj.getClass().isArray() || $obj instanceof ${classOf[scala.collection.Seq[_]].getName}" case _ => s"$obj instanceof ${CodeGenerator.boxedType(dataType)}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index fed5df6958..a89f055e2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -211,7 +211,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT .exists(_._2.map(_._2.exprId).distinct.length > 1), "Found duplicate rewrite attributes") - val attributeRewrites = AttributeMap(attrMappingForCurrentPlan) + val attributeRewrites = AttributeMap(attrMappingForCurrentPlan.toSeq) // Using attrMapping from the children plans to rewrite their parent node. // Note that we shouldn't rewrite a node using attrMapping from its sibling nodes. newPlan = newPlan.transformExpressions { @@ -225,7 +225,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT attrMapping ++= newAttrMapping.filter { case (a1, a2) => a1.exprId != a2.exprId } - newPlan -> attrMapping + newPlan -> attrMapping.toSeq } } rewrite(this)._1 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 711ef265c6..81f412c143 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -25,7 +25,9 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class GenericArrayData(val array: Array[Any]) extends ArrayData { - def this(seq: Seq[Any]) = this(seq.toArray) + // Specified this as`scala.collection.Seq` because seqOrArray can be + // `mutable.ArraySeq` in Scala 2.13 + def this(seq: scala.collection.Seq[Any]) = this(seq.toArray) def this(list: java.util.List[Any]) = this(list.asScala.toSeq) // TODO: This is boxing. We should specialize. @@ -38,7 +40,9 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq) def this(seqOrArray: Any) = this(seqOrArray match { - case seq: Seq[Any] => seq.toArray + // Specified this as`scala.collection.Seq` because seqOrArray can be + // `mutable.ArraySeq` in Scala 2.13 + case seq: scala.collection.Seq[Any] => seq.toArray case array: Array[Any] => array // array of objects, so no need to convert case array: Array[_] => array.toSeq.toArray[Any] // array of primitives, so box them }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index f32e48e1cc..48ae49740f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -478,7 +478,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal + that.longVal, Math.max(precision, that.precision), scale) } else { - Decimal(toBigDecimal + that.toBigDecimal) + Decimal(toBigDecimal.bigDecimal.add(that.toBigDecimal.bigDecimal)) } } @@ -486,7 +486,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal - that.longVal, Math.max(precision, that.precision), scale) } else { - Decimal(toBigDecimal - that.toBigDecimal) + Decimal(toBigDecimal.bigDecimal.subtract(that.toBigDecimal.bigDecimal)) } } 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 611c03e7b2..7e430b682f 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 @@ -54,7 +54,11 @@ class RelationalGroupedDataset protected[sql]( private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { val aggregates = if (df.sparkSession.sessionState.conf.dataFrameRetainGroupColumns) { - groupingExprs ++ aggExprs + groupingExprs match { + // call `toList` because `Stream` can't serialize in scala 2.13 + case s: Stream[Expression] => s.toList ++ aggExprs + case other => other ++ aggExprs + } } else { aggExprs } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 4c9efdbf2b..633a54ca36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -96,7 +96,7 @@ case class GenerateExec( if (outer && outputRows.isEmpty) { joinedRow.withRight(generatorNullRow) :: Nil } else { - outputRows.map(joinedRow.withRight) + outputRows.toIterator.map(joinedRow.withRight) } } ++ LazyIterator(() => boundGenerator.terminate()).map { row => // we leave the left side as the last element of its child output diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 53270b84e7..855ba3f00a 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -2,7 +2,7 @@ ## Summary - Number of queries: 339 - Number of expressions that missing example: 34 - - Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,struct,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch + - Expressions missing examples: and,bigint,binary,boolean,date,decimal,double,float,int,smallint,string,timestamp,tinyint,struct,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch ## Schema of Built-in Functions | Class name | Function name or alias | Query example | Output schema | | ---------- | ---------------------- | ------------- | ------------- | @@ -50,19 +50,19 @@ | org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | java_method | SELECT java_method('java.util.UUID', 'randomUUID') | struct | | org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | reflect | SELECT reflect('java.util.UUID', 'randomUUID') | struct | | org.apache.spark.sql.catalyst.expressions.CaseWhen | when | SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END | struct 0) THEN CAST(1 AS DECIMAL(11,1)) WHEN (2 > 0) THEN CAST(2.0 AS DECIMAL(11,1)) ELSE CAST(1.2 AS DECIMAL(11,1)) END:decimal(11,1)> | -| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | boolean | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | cast | SELECT cast('10' as int) | struct | -| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | double | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | date | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | decimal | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | boolean | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | double | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | float | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A | -| org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A | +| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceil | SELECT ceil(-0.1) | struct | | org.apache.spark.sql.catalyst.expressions.Ceil | ceiling | SELECT ceiling(-0.1) | struct | @@ -116,8 +116,8 @@ | org.apache.spark.sql.catalyst.expressions.Flatten | flatten | SELECT flatten(array(array(1, 2), array(3, 4))) | struct> | | org.apache.spark.sql.catalyst.expressions.Floor | floor | SELECT floor(-0.1) | struct | | org.apache.spark.sql.catalyst.expressions.FormatNumber | format_number | SELECT format_number(12332.123456, 4) | struct | -| org.apache.spark.sql.catalyst.expressions.FormatString | printf | SELECT printf("Hello World %d %s", 100, "days") | struct | | org.apache.spark.sql.catalyst.expressions.FormatString | format_string | SELECT format_string("Hello World %d %s", 100, "days") | struct | +| org.apache.spark.sql.catalyst.expressions.FormatString | printf | SELECT printf("Hello World %d %s", 100, "days") | struct | | org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp | from_utc_timestamp | SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul') | struct | | org.apache.spark.sql.catalyst.expressions.FromUnixTime | from_unixtime | SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') | struct | | org.apache.spark.sql.catalyst.expressions.GetJsonObject | get_json_object | SELECT get_json_object('{"a":"b"}', '$.a') | struct | @@ -150,8 +150,8 @@ | org.apache.spark.sql.catalyst.expressions.Lead | lead | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Least | least | SELECT least(10, 9, 2, 4, 3) | struct | | org.apache.spark.sql.catalyst.expressions.Left | left | SELECT left('Spark SQL', 3) | struct | -| org.apache.spark.sql.catalyst.expressions.Length | character_length | SELECT character_length('Spark SQL ') | struct | | org.apache.spark.sql.catalyst.expressions.Length | char_length | SELECT char_length('Spark SQL ') | struct | +| org.apache.spark.sql.catalyst.expressions.Length | character_length | SELECT character_length('Spark SQL ') | struct | | org.apache.spark.sql.catalyst.expressions.Length | length | SELECT length('Spark SQL ') | struct | | org.apache.spark.sql.catalyst.expressions.LengthOfJsonArray | json_array_length | SELECT json_array_length('[1,2,3,4]') | struct | | org.apache.spark.sql.catalyst.expressions.LessThan | < | SELECT 1 < 2 | struct<(1 < 2):boolean> | @@ -203,14 +203,14 @@ | org.apache.spark.sql.catalyst.expressions.PercentRank | percent_rank | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.Pi | pi | SELECT pi() | struct | | org.apache.spark.sql.catalyst.expressions.Pmod | pmod | SELECT pmod(10, 3) | struct | -| org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode_outer | SELECT posexplode_outer(array(10,20)) | struct | | org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode | SELECT posexplode(array(10,20)) | struct | +| org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode_outer | SELECT posexplode_outer(array(10,20)) | struct | | org.apache.spark.sql.catalyst.expressions.Pow | pow | SELECT pow(2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.Pow | power | SELECT power(2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.Quarter | quarter | SELECT quarter('2016-08-31') | struct | | org.apache.spark.sql.catalyst.expressions.RLike | rlike | SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\\Users.*' | struct<%SystemDrive%UsersJohn RLIKE %SystemDrive%\Users.*:boolean> | -| org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | struct | | org.apache.spark.sql.catalyst.expressions.Rand | rand | SELECT rand() | struct | +| org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | struct | | org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct | | org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct | @@ -230,19 +230,19 @@ | org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp | timestamp_seconds | SELECT timestamp_seconds(1230219000) | struct | | org.apache.spark.sql.catalyst.expressions.Sentences | sentences | SELECT sentences('Hi there! Good morning.') | struct>> | | org.apache.spark.sql.catalyst.expressions.Sequence | sequence | SELECT sequence(1, 5) | struct> | -| org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') | struct | | org.apache.spark.sql.catalyst.expressions.Sha1 | sha | SELECT sha('Spark') | struct | +| org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') | struct | | org.apache.spark.sql.catalyst.expressions.Sha2 | sha2 | SELECT sha2('Spark', 256) | struct | | org.apache.spark.sql.catalyst.expressions.ShiftLeft | shiftleft | SELECT shiftleft(2, 1) | struct | | org.apache.spark.sql.catalyst.expressions.ShiftRight | shiftright | SELECT shiftright(4, 1) | struct | | org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | shiftrightunsigned | SELECT shiftrightunsigned(4, 1) | struct | | org.apache.spark.sql.catalyst.expressions.Shuffle | shuffle | SELECT shuffle(array(1, 20, 3, 5)) | struct> | -| org.apache.spark.sql.catalyst.expressions.Signum | signum | SELECT signum(40) | struct | | org.apache.spark.sql.catalyst.expressions.Signum | sign | SELECT sign(40) | struct | +| org.apache.spark.sql.catalyst.expressions.Signum | signum | SELECT signum(40) | struct | | org.apache.spark.sql.catalyst.expressions.Sin | sin | SELECT sin(0) | struct | | org.apache.spark.sql.catalyst.expressions.Sinh | sinh | SELECT sinh(0) | struct | -| org.apache.spark.sql.catalyst.expressions.Size | size | SELECT size(array('b', 'd', 'c', 'a')) | struct | | org.apache.spark.sql.catalyst.expressions.Size | cardinality | SELECT cardinality(array('b', 'd', 'c', 'a')) | struct | +| org.apache.spark.sql.catalyst.expressions.Size | size | SELECT size(array('b', 'd', 'c', 'a')) | struct | | org.apache.spark.sql.catalyst.expressions.Slice | slice | SELECT slice(array(1, 2, 3, 4), 2, 2) | struct> | | org.apache.spark.sql.catalyst.expressions.SortArray | sort_array | SELECT sort_array(array('b', 'd', null, 'c', 'a'), true) | struct> | | org.apache.spark.sql.catalyst.expressions.SoundEx | soundex | SELECT soundex('Miller') | struct | @@ -252,8 +252,8 @@ | org.apache.spark.sql.catalyst.expressions.Stack | stack | SELECT stack(2, 1, 2, 3) | struct | | org.apache.spark.sql.catalyst.expressions.StringInstr | instr | SELECT instr('SparkSQL', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.StringLPad | lpad | SELECT lpad('hi', 5, '??') | struct | -| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT position('bar', 'foobarbar') | struct | | org.apache.spark.sql.catalyst.expressions.StringLocate | locate | SELECT locate('bar', 'foobarbar') | struct | +| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT position('bar', 'foobarbar') | struct | | org.apache.spark.sql.catalyst.expressions.StringRPad | rpad | SELECT rpad('hi', 5, '??') | struct | | org.apache.spark.sql.catalyst.expressions.StringRepeat | repeat | SELECT repeat('123', 2) | struct | | org.apache.spark.sql.catalyst.expressions.StringReplace | replace | SELECT replace('ABCabc', 'abc', 'DEF') | struct | @@ -303,11 +303,11 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.BitAndAgg | bit_and | SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BitOrAgg | bit_or | SELECT bit_or(col) FROM VALUES (3), (5) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BitXorAgg | bit_xor | SELECT bit_xor(col) FROM VALUES (3), (5) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | every | SELECT every(col) FROM VALUES (true), (true), (true) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | bool_and | SELECT bool_and(col) FROM VALUES (true), (true), (true) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | every | SELECT every(col) FROM VALUES (true), (true), (true) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | any | SELECT any(col) FROM VALUES (true), (false), (false) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | bool_or | SELECT bool_or(col) FROM VALUES (true), (false), (false) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | some | SELECT some(col) FROM VALUES (true), (false), (false) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | any | SELECT any(col) FROM VALUES (true), (false), (false) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CollectList | collect_list | SELECT collect_list(col) FROM VALUES (1), (2), (1) AS tab(col) | struct> | | org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet | collect_set | SELECT collect_set(col) FROM VALUES (1), (2), (1) AS tab(col) | struct> | | org.apache.spark.sql.catalyst.expressions.aggregate.Corr | corr | SELECT corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2) | struct | @@ -316,12 +316,12 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | count_min_sketch | N/A | N/A | | org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct | @@ -329,16 +329,16 @@ | org.apache.spark.sql.catalyst.expressions.aggregate.Percentile | percentile | SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Skewness | skewness | SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop | stddev_pop | SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev_samp | SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | -| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev | SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | std | SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev | SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | +| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev_samp | SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.Sum | sum | SELECT sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop | var_pop | SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | var_samp | SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | variance | SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col) | struct | | org.apache.spark.sql.catalyst.expressions.xml.XPathBoolean | xpath_boolean | SELECT xpath_boolean('1','a/b') | struct1, a/b):boolean> | -| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_number | SELECT xpath_number('12', 'sum(a/b)') | struct12, sum(a/b)):double> | | org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_double | SELECT xpath_double('12', 'sum(a/b)') | struct12, sum(a/b)):double> | +| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_number | SELECT xpath_number('12', 'sum(a/b)') | struct12, sum(a/b)):double> | | org.apache.spark.sql.catalyst.expressions.xml.XPathFloat | xpath_float | SELECT xpath_float('12', 'sum(a/b)') | struct12, sum(a/b)):float> | | org.apache.spark.sql.catalyst.expressions.xml.XPathInt | xpath_int | SELECT xpath_int('12', 'sum(a/b)') | struct12, sum(a/b)):int> | | org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT xpath('b1b2b3c1c2','a/b/text()') | structb1b2b3c1c2, a/b/text()):array> | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 8078165cf8..cdd2568771 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -412,8 +412,8 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { // Original bug was a NullPointerException exception caused by calling collect(), test for this val resultRow = result.collect()(0) - assert(resultRow.get(0).asInstanceOf[Seq[String]].toSet == Set("1", "2", "3")) - assert(resultRow.get(1).asInstanceOf[Seq[String]].toSet == Set("a", "b", null)) + assert(resultRow.get(0).asInstanceOf[scala.collection.Seq[String]].toSet == Set("1", "2", "3")) + assert(resultRow.get(1).asInstanceOf[scala.collection.Seq[String]].toSet == Set("a", "b", null)) } test("sampleBy") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala index fd964355e8..d18aa9c549 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala @@ -95,7 +95,9 @@ class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession { spark.sessionState.catalog.lookupFunctionInfo(funcId) } - val classFunsMap = funInfos.groupBy(_.getClassName).toSeq.sortBy(_._1) + val classFunsMap = funInfos.groupBy(_.getClassName).toSeq.sortBy(_._1).map { + case (className, infos) => (className, infos.sortBy(_.getName)) + } val outputBuffer = new ArrayBuffer[String] val outputs = new ArrayBuffer[QueryOutput] val missingExamples = new ArrayBuffer[String] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 04af7d1a68..e9b99ad002 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -494,7 +494,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { sparkContext.parallelize(Seq(Row(Map("a" -> new BigDecimal("2011000000000002456556"))))), StructType(Seq(StructField("col1", MapType(StringType, DecimalType(30, 0)))))) val udf2 = org.apache.spark.sql.functions.udf((map: Map[String, BigDecimal]) => { - map.mapValues(value => if (value == null) null else value.toBigInteger.toString) + map.mapValues(value => if (value == null) null else value.toBigInteger.toString).toMap }) checkAnswer(df2.select(udf2($"col1")), Seq(Row(Map("a" -> "2011000000000002456556")))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 9caf0c836f..d2970ef1bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -136,7 +136,7 @@ abstract class OrcQueryTest extends OrcTest { assertResult(10) { sql("SELECT name, contacts FROM t where age > 5") .rdd - .flatMap(_.getAs[Seq[_]]("contacts")) + .flatMap(_.getAs[scala.collection.Seq[_]]("contacts")) .count() } @@ -148,7 +148,7 @@ abstract class OrcQueryTest extends OrcTest { val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8") assert(df.count() === 2) assertResult(4) { - df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count() + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() } } @@ -160,7 +160,7 @@ abstract class OrcQueryTest extends OrcTest { val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8") assert(df.count() === 3) assertResult(6) { - df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count() + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() } } }