Skip to content
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

[SPARK-44259][CONNECT][TESTS] Make connect-client-jvm pass on Java 21 except RemoteSparkSession-based tests #41805

Closed
wants to merge 4 commits into from

Conversation

LuciferYang
Copy link
Contributor

@LuciferYang LuciferYang commented Jun 30, 2023

What changes were proposed in this pull request?

This pr ignore all tests inherit RemoteSparkSession as default for Java 21 by override the test function in RemoteSparkSession, they are all arrow-based tests due to the use of arrow data format for rpc communication in connect.

23/06/30 11:45:41 ERROR SparkConnectService: Error during: execute. UserId: . SessionId: e7479b73-d02c-47e9-85c8-40b3e9315561.
java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.<init>(long, int) not available
	at org.apache.arrow.memory.util.MemoryUtil.directBuffer(MemoryUtil.java:174)
	at org.apache.arrow.memory.ArrowBuf.getDirectBuffer(ArrowBuf.java:229)
	at org.apache.arrow.memory.ArrowBuf.nioBuffer(ArrowBuf.java:224)
	at org.apache.arrow.vector.ipc.WriteChannel.write(WriteChannel.java:133)
	at org.apache.arrow.vector.ipc.message.MessageSerializer.writeBatchBuffers(MessageSerializer.java:303)
	at org.apache.arrow.vector.ipc.message.MessageSerializer.serialize(MessageSerializer.java:276)
	at org.apache.arrow.vector.ipc.message.MessageSerializer.serialize(MessageSerializer.java:237)
	at org.apache.spark.sql.execution.arrow.ArrowConverters$ArrowBatchWithSchemaIterator.$anonfun$next$3(ArrowConverters.scala:174)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1487)
	at org.apache.spark.sql.execution.arrow.ArrowConverters$ArrowBatchWithSchemaIterator.next(ArrowConverters.scala:181)
	at org.apache.spark.sql.execution.arrow.ArrowConverters$ArrowBatchWithSchemaIterator.next(ArrowConverters.scala:128)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
	at scala.collection.Iterator.foreach(Iterator.scala:943)
	at scala.collection.Iterator.foreach$(Iterator.scala:943)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
	at org.apache.spark.sql.connect.service.SparkConnectStreamHandler$.processAsArrowBatches(SparkConnectStreamHandler.scala:178)
	at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handlePlan(SparkConnectStreamHandler.scala:104)
	at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$1(SparkConnectStreamHandler.scala:86)
	at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$1$adapted(SparkConnectStreamHandler.scala:53)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$3(SessionHolder.scala:152)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:857)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$2(SessionHolder.scala:152)
	at org.apache.spark.JobArtifactSet$.withActive(JobArtifactSet.scala:109)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withContext$1(SessionHolder.scala:122)
	at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:209)
	at org.apache.spark.sql.connect.service.SessionHolder.withContext(SessionHolder.scala:121)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$1(SessionHolder.scala:151)
	at org.apache.spark.sql.connect.service.SessionHolder.withSessionBasedPythonPaths(SessionHolder.scala:137)
	at org.apache.spark.sql.connect.service.SessionHolder.withSession(SessionHolder.scala:150)
	at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handle(SparkConnectStreamHandler.scala:53)
	at org.apache.spark.sql.connect.service.SparkConnectService.executePlan(SparkConnectService.scala:166)
	at org.apache.spark.connect.proto.SparkConnectServiceGrpc$MethodHandlers.invoke(SparkConnectServiceGrpc.java:584)
	at org.sparkproject.connect.grpc.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182)
	at org.sparkproject.connect.grpc.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:346)
	at org.sparkproject.connect.grpc.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:860)
	at org.sparkproject.connect.grpc.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
	at org.sparkproject.connect.grpc.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
	at java.base/java.lang.Thread.run(Thread.java:1583)

All ignored test related to apache/arrow#35053, so we should wait for upgrading to a new arrow version and re-enable them for Java 21, the following TODO JIRA is created for that.

Why are the changes needed?

Make Java 21 daily test can monitor other non-arrow based tests.

Does this PR introduce any user-facing change?

No

How was this patch tested?

  • Pass GitHub Actions
  • manually tests with Java 21:
java -version
openjdk version "21-ea" 2023-09-19
OpenJDK Runtime Environment Zulu21+65-CA (build 21-ea+26)
OpenJDK 64-Bit Server VM Zulu21+65-CA (build 21-ea+26, mixed mode, sharing)
build/sbt "connect-client-jvm/test" -Phive
[info] Run completed in 4 seconds, 640 milliseconds.
[info] Total number of tests run: 846
[info] Suites: completed 22, aborted 0
[info] Tests: succeeded 846, failed 0, canceled 167, ignored 1, pending 0
[info] All tests passed.

@LuciferYang LuciferYang changed the title [SPARK-44259][CONNECT][TESTS] Ignore all Arrow-based connect tests for Java 21 [SPARK-44259][CONNECT][TESTS] Ignore all tests inherit RemoteSparkSession for Java 21 Jun 30, 2023
@HyukjinKwon
Copy link
Member

Are there a lot of tests that fail with 21? I start to feel like we should just fix them instead of skipping individual tests.

@LuciferYang
Copy link
Contributor Author

Are there a lot of tests that fail with 21? I start to feel like we should just fix them instead of skipping individual tests.

we can't fix them now, we need wait a new arrow version

@LuciferYang
Copy link
Contributor Author

LuciferYang commented Jun 30, 2023

@HyukjinKwon All ignored tests related to apache/arrow#35053 and @dongjoon-hyun just fixed it, but arrow has not released a new version yet

@LuciferYang
Copy link
Contributor Author

also cc @dongjoon-hyun FYI

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jun 30, 2023

Yes, I fixed the root cause in Arrow-side. SPARK-44247 will upgrade to Apache Arrow 13.0.0 which will recover all ignored tests. The remaining problem is that Apache Arrow 13.0.0 release is expected on August. We will see.

For this PR, I didn't choose this approach because this way was a little too intrusive.

BTW, @LuciferYang . connect module already passed after SPARK-44122. Could you remove the following from the PR description?

build/sbt "connect-/test" -Phive

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-44259][CONNECT][TESTS] Ignore all tests inherit RemoteSparkSession for Java 21 [SPARK-44259][CONNECT][TESTS] Make connect-client-jvm pass on Java 21 except RemoteSparkSession-based tests Jun 30, 2023
@LuciferYang
Copy link
Contributor Author

BTW, @LuciferYang . connect module already passed after SPARK-44122. Could you remove the following from the PR description?

Already removed ~

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jun 30, 2023

@LuciferYang . For this one, shall you add a new test tag, ExtendedArrowTest, and use it?

@dongjoon-hyun
Copy link
Member

Never mind~

@dongjoon-hyun
Copy link
Member

+1 from my side. However, please follow @HyukjinKwon 's opinion.

@LuciferYang
Copy link
Contributor Author

For this PR, I didn't choose this approach because this way was a little too intrusive.

Please wait a moment, let me try a new way to reduce code changes

* SPARK-44259: override test function to skip `RemoteSparkSession-based` tests as default,
* we should delete this function after SPARK-44121 is completed.
*/
override protected def test(testName: String, testTags: Tag*)(testFun: => Any)
Copy link
Contributor Author

@LuciferYang LuciferYang Jun 30, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dongjoon-hyun @HyukjinKwon the new code override test function, all RemoteSparkSession-based tests using Java 21 are ignored by default, so there is no need to add the assume condition to test case one by one

Copy link
Contributor Author

@LuciferYang LuciferYang Jun 30, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this seem less intrusive ?

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Yes, this looks much better.

@dongjoon-hyun
Copy link
Member

Merged to master for Apache Spark 3.5.0.
Thank you, @LuciferYang , @yaooqinn , @HyukjinKwon .

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM2

@Midhunpottammal
Copy link

Midhunpottammal commented Mar 5, 2024

Merged to master for Apache Spark 3.5.0. Thank you, @LuciferYang , @yaooqinn , @HyukjinKwon .

while java Java(TM) SE Runtime Environment (build 21.0.2+13-LTS-58)
Spark 3.5.0 giving error

Previous exception in task: sun.misc.Unsafe or java.nio.DirectByteBuffer.<init>(long, int) not available org.apache.arrow.memory.util.MemoryUtil.directBuffer(MemoryUtil.java:174) org.apache.arrow.memory.ArrowBuf.getDirectBuffer(ArrowBuf.java:229) org.apache.arrow.memory.ArrowBuf.nioBuffer(ArrowBuf.java:224) org.apache.arrow.vector.ipc.WriteChannel.write(WriteChannel.java:133) org.apache.arrow.vector.ipc.message.MessageSerializer.writeBatchBuffers(MessageSerializer.java:303)

what is the solution for this : Arrow can work with Aparch spark 3.5.0 in java 21

code :

spark = SparkSession.builder \ .appName("ArrowPySparkExample") \ .getOrCreate() spark.conf.set("Dio.netty.tryReflectionSetAccessible", "true") spark.conf.set("spark.sql.execution.arrow.pyspark.enabled", "true") pdf = pd.DataFrame(["midhun"]) df = spark.createDataFrame(pdf) result_pdf = df.select("*").toPandas()

@LuciferYang
Copy link
Contributor Author

@Midhunpottammal Spark 3.5 has not announced support for Java 21, this feature is likely to be released in Spark 4.0 :)

@dongjoon-hyun
Copy link
Member

Ya, @LuciferYang is right.

To @Midhunpottammal , you need SPARK-43831 for Java 21 support.

@Midhunpottammal
Copy link

@dongjoon-hyun @LuciferYang Thank you,

I experimented with different versions of Java, Spark, and Arrow.I managed to get Arrow working with a lower version of Java in Spark 3.5.0. Here's my stack:

pyarrow==15.0.0
pyspark==3.5.0
java == Java(TM) SE Runtime Environment (build 17.0.10+11-LTS-240)

When I try to move to Java version 21, I encounter the same error

@LuciferYang
Copy link
Contributor Author

Ya, @LuciferYang is right.

To @Midhunpottammal , you need SPARK-43831 for Java 21 support.

@Midhunpottammal As @dongjoon-hyun said, all the relevant patches in SPARK-43831 are needed for Java 21 support. So this is not a job that can be accomplished with minor changes on Spark 3.5.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants