[SPARK-54194][CONNECT][FOLLOWUP] Spark Connect Proto Plan Compression - Scala Client#53003
Closed
xi-db wants to merge 6 commits into
Closed
[SPARK-54194][CONNECT][FOLLOWUP] Spark Connect Proto Plan Compression - Scala Client#53003xi-db wants to merge 6 commits into
xi-db wants to merge 6 commits into
Conversation
LuciferYang
reviewed
Nov 12, 2025
| <include>io.perfmark:*</include> | ||
| <include>org.apache.arrow:*</include> | ||
| <include>org.codehaus.mojo:*</include> | ||
| <include>com.github.luben:zstd-jni</include> |
Contributor
There was a problem hiding this comment.
For 3rd-party jars, if we choose to shade them, it's advisable to also perform relocation on them.
Contributor
Author
There was a problem hiding this comment.
Thanks for the comment. For this zstd-jni, it is tricky to shade because it has native libraries. If we shade it, it will fail with error java.lang.UnsatisfiedLinkError: 'int org.sparkproject.connect.com.github.luben.zstd.Zstd.defaultCompressionLevel()', because the native library was compiled with the original package name. I'm keeping it not shaded for now.
Update: I've removed this line so we won't bundle a copy of it into the client jar.
asf-gitbox-commits
pushed a commit
that referenced
this pull request
Nov 14, 2025
… - Scala Client ### What changes were proposed in this pull request? In the previous PR #52894 of Spark Connect Proto Plan Compression, both Server-side and PySpark client changes were implemented. In this PR, the corresponding Scala client changes are implemented, so plan compression are now supported on the Scala client as well. To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client: ``` import scala.util.Random import org.apache.spark.sql.DataFrame import spark.implicits._ def randomLetters(n: Int): String = { Iterator.continually(Random.nextPrintableChar()) .filter(_.isLetter) .take(n) .mkString } val numUniqueSmallRelations = 5 val sizePerSmallRelation = 512 * 1024 val smallDfs: Seq[DataFrame] = (0 until numUniqueSmallRelations).map { _ => Seq(randomLetters(sizePerSmallRelation)).toDF("value") } var resultDf = smallDfs.head for (_ <- 0 until 500) { val idx = Random.nextInt(smallDfs.length) resultDf = resultDf.unionByName(smallDfs(idx)) } resultDf.collect() ``` It fails with RESOURCE_EXHAUSTED error with message `gRPC message exceeds maximum size 134217728: 269207219`, because the server is trying to send an ExecutePlanResponse of ~260MB to the client. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when handling large plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #53003 from xi-db/plan-compression-scala-client. Authored-by: Xi Lyu <xi.lyu@databricks.com> Signed-off-by: Herman van Hovell <herman@databricks.com> (cherry picked from commit 6cb88c1) Signed-off-by: Herman van Hovell <herman@databricks.com>
Contributor
|
Merging to master/4.1. Thanks! |
huangxiaopingRD
pushed a commit
to huangxiaopingRD/spark
that referenced
this pull request
Nov 25, 2025
… - Scala Client ### What changes were proposed in this pull request? In the previous PR apache#52894 of Spark Connect Proto Plan Compression, both Server-side and PySpark client changes were implemented. In this PR, the corresponding Scala client changes are implemented, so plan compression are now supported on the Scala client as well. To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client: ``` import scala.util.Random import org.apache.spark.sql.DataFrame import spark.implicits._ def randomLetters(n: Int): String = { Iterator.continually(Random.nextPrintableChar()) .filter(_.isLetter) .take(n) .mkString } val numUniqueSmallRelations = 5 val sizePerSmallRelation = 512 * 1024 val smallDfs: Seq[DataFrame] = (0 until numUniqueSmallRelations).map { _ => Seq(randomLetters(sizePerSmallRelation)).toDF("value") } var resultDf = smallDfs.head for (_ <- 0 until 500) { val idx = Random.nextInt(smallDfs.length) resultDf = resultDf.unionByName(smallDfs(idx)) } resultDf.collect() ``` It fails with RESOURCE_EXHAUSTED error with message `gRPC message exceeds maximum size 134217728: 269207219`, because the server is trying to send an ExecutePlanResponse of ~260MB to the client. With the improvement introduced by the PR, the above code runs successfully and prints the expected result. ### Why are the changes needed? It improves Spark Connect stability when handling large plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes apache#53003 from xi-db/plan-compression-scala-client. Authored-by: Xi Lyu <xi.lyu@databricks.com> Signed-off-by: Herman van Hovell <herman@databricks.com>
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
In the previous PR #52894 of Spark Connect Proto Plan Compression, both Server-side and PySpark client changes were implemented.
In this PR, the corresponding Scala client changes are implemented, so plan compression are now supported on the Scala client as well.
To reproduce the existing issue we are solving here, run this code on Spark Connect Scala client:
It fails with RESOURCE_EXHAUSTED error with message
gRPC message exceeds maximum size 134217728: 269207219, because the server is trying to send an ExecutePlanResponse of ~260MB to the client.With the improvement introduced by the PR, the above code runs successfully and prints the expected result.
Why are the changes needed?
It improves Spark Connect stability when handling large plans.
Does this PR introduce any user-facing change?
No.
How was this patch tested?
New tests.
Was this patch authored or co-authored using generative AI tooling?
No.