NVIDIA / spark-rapids

Spark RAPIDS plugin - accelerate Apache Spark with GPUs
https://nvidia.github.io/spark-rapids
Apache License 2.0
736 stars 219 forks source link

[BUG] JDK17 nightly build after Spark UT Framework is merged #10801

Open NvTimLiu opened 2 weeks ago

NvTimLiu commented 2 weeks ago

Describe the bug

UT tests failed on : The value '**' of the type "STRING" cannot be cast to "INT" because it is malformed.

Note: Currently this issue only appeared on spark-rapids nightly build against JDK17;

Nightly build UT against JDK11 and JDK8 have not reported these failures

- Cast from string to int using hand-picked values
 24/05/12 01:41:36.590 Executor task launch worker for task 0.0 in stage 33.0 (TID 49) ERROR Executor: Exception in task 0.0 in stage 33.0 (TID 49)
 org.apache.spark.SparkNumberFormatException: The value '-.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
        at org.apache.spark.sql.errors.QueryExecutionErrors$.invalidInputInCastToNumberError(QueryExecutionErrors.scala:150) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils$.withException(UTF8StringUtils.scala:46) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils$.toIntExact(UTF8StringUtils.scala:33) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils.toIntExact(UTF8StringUtils.scala) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) ~[?:?]
        at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:329) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
        at java.lang.Thread.run(Thread.java:840) ~[?:?]
 24/05/12 01:41:36.607 task-result-getter-1 ERROR TaskSetManager: Task 0 in stage 33.0 failed 1 times; aborting job
 24/05/12 01:41:36.608 Executor task launch worker for task 1.0 in stage 33.0 (TID 50) ERROR Executor: Exception in task 1.0 in stage 33.0 (TID 50)
 org.apache.spark.SparkNumberFormatException: The value '+1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
        at org.apache.spark.sql.errors.QueryExecutionErrors$.invalidInputInCastToNumberError(QueryExecutionErrors.scala:150) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils$.withException(UTF8StringUtils.scala:46) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils$.toIntExact(UTF8StringUtils.scala:33) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.util.UTF8StringUtils.toIntExact(UTF8StringUtils.scala) ~[spark-catalyst_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) ~[?:?]
        at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364) ~[spark-sql_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:329) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.0.jar:3.3.0]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
        at java.lang.Thread.run(Thread.java:840) ~[?:?]
binmahone commented 2 weeks ago

The stacktrace pasted by Tim is expected in Cast from string to int using hand-picked values , it's not actually failing.

The real failed test case is test("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail") in RapidsDataFrameAggregateSuite

The root cause of this failed test case is that, when Spark-Rapids is built by JDK 17 and Run on JDK 17, the expression RelationalGroupedDataset.GroupByType.getClass.getSimpleName will return "" instead of correct value "GroupByType$", RelationalGroupedDataset is a object in Apache Spark.

I have tried to reproduce this behavior in a simple project (https://github.com/binmahone/test_jdk17_java, you can download it, build & run it with JDK17), the output is always "GroupByType$". Even if I have tried to move many pof Spark-Rapids's pom setttings to the simple project (such as scalatest configurations, etc.), it's still not reproducing the wrong value "".

So I have no clue what settings in Spark-Rapids leads to this wrong behaviour. Since this issue looks not very urgent, I'll first ignore the failed test case and unblock Nightly build

gerashegalov commented 2 weeks ago

It looks related to https://bugs.java.com/bugdatabase/view_bug.do?bug_id=JDK-8198818 but I cannot reproduce it outside Spark and Plugin either.

binmahone commented 2 weeks ago

hi @gerashegalov, dead link on my side, can you check it please? Also, just to clarify, the behavior is correct in vanilla spark, but wrong in spark-rapids

gerashegalov commented 2 weeks ago

@binmahone I can still click through it, maybe it was temporarily unavailable?

Can you access and search https://bugs.java.com/bugdatabase/ for "JDK-8198818 : Class.simpleName different for anonymous classes"?

It came up in https://youtrack.jetbrains.com/issue/KT-23072/Class.simpleName-of-anonymous-object-is-not-an-empty-string-in-JDK8-but-on-JDK9

I am not sure that Spark test code is actually robust. Calling getSimpleName on weird Scala object might be sub-optimal.

gerashegalov commented 2 weeks ago

It turns out, there is a history of getSimpleName being broken with various combinations of Scala version and JDK
https://issues.apache.org/jira/browse/SPARK-34596?jql=text%20~%20getSimpleName%20AND%20project%20%3D%20Spark%20ORDER%20BY%20created%20DESC

Yet it is still in use in various places.

We just got unlucky that we are testing Spark 3.3 that bundles 2.12.15 . This combo yields an empty string for GroupByType

$ echo 'Seq(1,2,3).toDF.groupBy($"value")' | JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 /home/gshegalov/dist/spark-3.3.4-bin-hadoop2/bin/spark-shell |& grep 'Scala\|Relational'
Using Scala version 2.12.15 (OpenJDK 64-Bit Server VM, Java 17.0.10)
res0: org.apache.spark.sql.RelationalGroupedDataset = RelationalGroupedDataset: [grouping expressions: [value], value: [value: int], type: ]

And it is fixed in 3.4+ simply because it upgraded to Scala 2.12.17

$ echo 'Seq(1,2,3).toDF.groupBy($"value")' | JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 /home/gshegalov/dist/spark-3.4.0-bin-hadoop3/bin/spark-shell |& grep 'Scala\|Relational'
Using Scala version 2.12.17 (OpenJDK 64-Bit Server VM, Java 17.0.10)
res0: org.apache.spark.sql.RelationalGroupedDataset = RelationalGroupedDataset: [grouping expressions: [value], value: [value: int], type: GroupBy]
binmahone commented 2 weeks ago

@gerashegalov great findings!

It's interesting that you can reproduce the problem with Spark 3.3 + scala 2.12.15 (I tried your way, it reproduces, too), because :

  1. Spark 3.3 hard coded scala 2.12.15 in its pom , but when it runs test("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail") everything is okay, i.e. it does not have the empty simple name problem in UT
  2. my toy project https://github.com/binmahone/test_jdk17_java also hard coded scala 2.12.15 in its pom, but it cannot reproduce the issue either:
/usr/lib/jvm/java-1.17.0-openjdk-amd64/bin/java -javaagent:/home/hongbin/.local/share/JetBrains/Toolbox/apps/intellij-idea-ultimate/lib/idea_rt.jar=46767:/home/hongbin/.local/share/JetBrains/Toolbox/apps/intellij-idea-ultimate/bin -Dfile.encoding=UTF-8 -classpath /home/hongbin/.local/share/JetBrains/IntelliJIdea2024.1/Scala/lib/runners.jar:/home/hongbin/code/test_jdk17_java/target/spark311/test-classes:/home/hongbin/code/test_jdk17_java/target/spark311/classes:/home/hongbin/.m2/repository/org/scala-lang/scala-library/2.12.15/scala-library-2.12.15.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/home/hongbin/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/home/hongbin/.m2/repository/org/scala-lang/scala-reflect/2.12.17/scala-reflect-2.12.17.jar org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner -s org.example.SimpleSuite -showProgressMessages true
Testing started at 11:26 am ...

Hello world!
GroupByType$

Even if I manually replace /home/hongbin/.m2/repository/org/scala-lang/scala-reflect/2.12.17/scala-reflect-2.12.17.jar with /home/hongbin/.m2/repository/org/scala-lang/scala-reflect/2.12.15/scala-reflect-2.12.15.jar (this is introduced by scalatest, as shown in below snapshot), it's still printing GroupByType$ instread of empty string

image

binmahone commented 2 weeks ago

So what would be you advice next ? I suggest holding upgrading 2.12.17 until we have more stronger reasons to do so. For the meanwhile we'll still exclude this test case as I did in https://github.com/NVIDIA/spark-rapids/pull/10820

gerashegalov commented 2 weeks ago

So what would be you advice next ?

Keep enabling the UT for all Spark 3.3+ version. Make sure we can make exclude/xfaill tests conditionally just like we do with pytests and "xfail" this particular test only for 3.3.x while keeping it running for 3.4+

Regarding not being able to repro with the toy project, it may have some mismatch with bytecode generation in Spark build:

binmahone commented 1 week ago

hi @NvTimLiu , with above discussion I think we can close this issue or move this to backlog, what you think?

NvTimLiu commented 1 week ago

Sounds good to me

jlowe commented 1 week ago

I'm confused how this was resolved. @gerashegalov proposed keeping the test and xfailing it only on Spark 3.3 but run it on Spark 3.4+, so we're at least running it somewhere. As it is now, this test was turned off for all Spark versions in #10820 which is not the state I thought we wanted to leave it in.

binmahone commented 1 week ago

Hi @jlowe , the Spark UT is only enabled in spark 3.3.0 for now(as stated in https://github.com/NVIDIA/spark-rapids/issues/10745), so we don't actually have anywhere else to run the test case. https://github.com/NVIDIA/spark-rapids/pull/10820 is a workaround and it leaves a "KNOWN_ISSUE" in there, so at least this issue will not be forgot. What would be your prefered way out here?

jlowe commented 1 week ago

If there's followup work to do, there needs to be a way to track that work. Github issues is how we prefer to track. There's stuff to do here, but no issue to track it. Therefore it is very likely it will never be done, because we'll forget that we were supposed to do it. The "KNOWN_ISSUE" points to a closed issue, so either:

binmahone commented 1 week ago

As advised by @jlowe , I reopened this issue to keep this issue still in track

GaryShen2008 commented 1 week ago

Low priority it since it's a conflict between scala 2.12.15 and JDK17. Change it target in 24.08. Two follow up things we may need here:

  1. Upgrade test framework to support excluding based on JDK version. So the case is only excluded on JDK17 + Scala 2.12.15, but still run on JDK8. (Let's target it in 24.08.) https://github.com/NVIDIA/spark-rapids/issues/10889
  2. Don't exclude the case when JDK17 + Scala 2.12.17+ when we support new Spark versions in the future.