-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32808][SQL] Fix some test cases of sql/core module in scala 2.13
#29660
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
…onV2Suite JsonLegacyTimeParserSuite DataStreamReaderWriterSuite
…yTestSuite DataFrameSuite
|
cc @srowen , this pr is try to pass some test case of |
| 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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The entrance is
spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala
Lines 28 to 38 in 04f7f6d
| object ArrayData { | |
| def toArrayData(input: Any): ArrayData = input match { | |
| case a: Array[Boolean] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Byte] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Short] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Int] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Long] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Float] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case a: Array[Double] => UnsafeArrayData.fromPrimitiveArray(a) | |
| case other => new GenericArrayData(other) | |
| } |
not easy to call toSeq, so I changed it here
| 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) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In Scala 2.13, + method is
def + (that: BigDecimal): BigDecimal = new BigDecimal(this.bigDecimal.add(that.bigDecimal, mc), mc)
and in Scala 2.12 + method is
def + (that: BigDecimal): BigDecimal = new BigDecimal(this.bigDecimal add that.bigDecimal, mc)
There are some differences in accuracy.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we want to set a MathContext here anyway?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you mean we need change to use methods with MathContext ? Like BigDecimal add(BigDecimal augend, MathContext mc) ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry , I think I don't fully understand this comments ....
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the change is OK here, because we actually do not want to modify the rounding, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, you are right ~
|
|
|
Test build #128352 has finished for PR 29660 at commit
|
There are other reasons. I'm working on it |
| 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) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we want to set a MathContext here anyway?
|
Do you want to add more changes here? we can merge it whenever it gets big and continue in another PR if desired. |
|
@srowen Maybe we can merge this first, other failures are related to the 'PlanStabilitySuite' and I will continue to fix these in another pr. |
|
Address 454b53c merge upstream master and resolve conflict file |
|
Test build #128381 has finished for PR 29660 at commit
|
|
Jenkins retest this please |
|
Test build #128409 has finished for PR 29660 at commit
|
|
local test |
|
Address 9185a95 re-sync master and local test core module ,all test passed. |
|
Test build #128430 has finished for PR 29660 at commit
|
|
org.apache.spark.sql.hive.thriftserver.CliSuite.* failed because |
|
local test |
|
retest this please |
|
Test build #128446 has finished for PR 29660 at commit
|
|
@srowen Jenkins and GitHub Action all passed ~ |
|
Merged to master. I have left the JIRA open though. |
|
thx @srowen @xuanyuanking |
### What changes were proposed in this pull request? After #29660 and #29689 there are 13 remaining failed cases of sql core module with Scala 2.13. The reason for the remaining failed cases is the optimization result of `CostBasedJoinReorder` maybe different with same input in Scala 2.12 and Scala 2.13 if there are more than one same cost candidate plans. In this pr give a way to make the optimization result deterministic as much as possible to pass all remaining failed cases of `sql/core` module in Scala 2.13, the main change of this pr as follow: - Change to use `LinkedHashMap` instead of `Map` to store `foundPlans` in `JoinReorderDP.search` method to ensure same iteration order with same insert order because iteration order of `Map` behave differently under Scala 2.12 and 2.13 - Fixed `StarJoinCostBasedReorderSuite` affected by the above change - Regenerate golden files affected by the above change. ### Why are the changes needed? We need to support a Scala 2.13 build. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Scala 2.12: Pass the Jenkins or GitHub Action - Scala 2.13: All tests passed. 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 8485, failed 13, canceled 1, ignored 52, pending 0 *** 13 TESTS FAILED *** ``` **After** ``` Tests: succeeded 8498, failed 0, canceled 1, ignored 52, pending 0 All tests passed. ``` Closes #29711 from LuciferYang/SPARK-32808-3. Authored-by: yangjie01 <[email protected]> Signed-off-by: Sean Owen <[email protected]>
### What changes were proposed in this pull request? After apache/spark#29660 and apache/spark#29689 there are 13 remaining failed cases of sql core module with Scala 2.13. The reason for the remaining failed cases is the optimization result of `CostBasedJoinReorder` maybe different with same input in Scala 2.12 and Scala 2.13 if there are more than one same cost candidate plans. In this pr give a way to make the optimization result deterministic as much as possible to pass all remaining failed cases of `sql/core` module in Scala 2.13, the main change of this pr as follow: - Change to use `LinkedHashMap` instead of `Map` to store `foundPlans` in `JoinReorderDP.search` method to ensure same iteration order with same insert order because iteration order of `Map` behave differently under Scala 2.12 and 2.13 - Fixed `StarJoinCostBasedReorderSuite` affected by the above change - Regenerate golden files affected by the above change. ### Why are the changes needed? We need to support a Scala 2.13 build. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Scala 2.12: Pass the Jenkins or GitHub Action - Scala 2.13: All tests passed. 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 8485, failed 13, canceled 1, ignored 52, pending 0 *** 13 TESTS FAILED *** ``` **After** ``` Tests: succeeded 8498, failed 0, canceled 1, ignored 52, pending 0 All tests passed. ``` Closes #29711 from LuciferYang/SPARK-32808-3. Authored-by: yangjie01 <[email protected]> Signed-off-by: Sean Owen <[email protected]>
What changes were proposed in this pull request?
The purpose of this pr is to partial resolve SPARK-32808, total of 26 failed test cases were fixed, the related suite as follow:
StreamingAggregationSuiterelated test cases (2 FAILED -> Pass)GeneratorFunctionSuiterelated test cases (2 FAILED -> Pass)UDFSuiterelated test cases (2 FAILED -> Pass)SQLQueryTestSuiterelated test cases (5 FAILED -> Pass)WholeStageCodegenSuiterelated test cases (1 FAILED -> Pass)DataFrameSuiterelated test cases (3 FAILED -> Pass)OrcV1QuerySuite\OrcV2QuerySuiterelated test cases (4 FAILED -> Pass)ExpressionsSchemaSuiterelated test cases (1 FAILED -> Pass)DataFrameStatSuiterelated test cases (1 FAILED -> Pass)JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuiterelated test cases (6 FAILED -> Pass)The main change of this pr as following:
Fix Scala 2.13 compilation problems in
ShuffleBlockFetcherIteratorandAnalyzerSpecified
Seqtoscala.collection.Seqinobjects.scalaandGenericArrayDatabecause internal useSeqmaybemutable.ArraySeqand not easy to call.toSeqShould specified
Seqtoscala.collection.Seqwhen we callRow.getAs[Seq]andRow.get(i).asInstanceOf[Seq]because the data maybemutable.ArraySeqbutSeqisimmutable.Seqin Scala 2.13Use a compatible way to let
+and-method ofDecimalhaving the same behavior in Scala 2.12 and Scala 2.13Call
toListinRelationalGroupedDataset.toDFmethod whengroupingExprsisStreamtype becauseStreamcan't serialize in Scala 2.13Add a manual sort to
classFunsMapinExpressionsSchemaSuitebecauseIterable.groupByin Scala 2.13 has different result withTraversableLike.groupByin Scala 2.12Why are the changes needed?
We need to support a Scala 2.13 build.
Does this PR introduce any user-facing change?
Should specified
Seqtoscala.collection.Seqwhen we callRow.getAs[Seq]andRow.get(i).asInstanceOf[Seq]because the data maybemutable.ArraySeqbut theSeqisimmutable.Seqin Scala 2.13How was this patch tested?
Scala 2.12: Pass the Jenkins or GitHub Action
Scala 2.13: Do the following:
Before
After