Skip to content

Conversation

@xi-db
Copy link
Contributor

@xi-db xi-db commented Nov 11, 2025

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.

<include>io.perfmark:*</include>
<include>org.apache.arrow:*</include>
<include>org.codehaus.mojo:*</include>
<include>com.github.luben:zstd-jni</include>
Copy link
Contributor

Choose a reason for hiding this comment

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

For 3rd-party jars, if we choose to shade them, it's advisable to also perform relocation on them.

Copy link
Contributor Author

@xi-db xi-db Nov 12, 2025

Choose a reason for hiding this comment

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

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.

@xi-db xi-db changed the title [WIP][SPARK-54194][CONNECT][FOLLOWUP] Spark Connect Proto Plan Compression - Scala Client [SPARK-54194][CONNECT][FOLLOWUP] Spark Connect Proto Plan Compression - Scala Client Nov 12, 2025
@xi-db xi-db marked this pull request as ready for review November 12, 2025 14:55
Copy link
Contributor

@hvanhovell hvanhovell left a comment

Choose a reason for hiding this comment

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

LGTM

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 <[email protected]>
Signed-off-by: Herman van Hovell <[email protected]>
(cherry picked from commit 6cb88c1)
Signed-off-by: Herman van Hovell <[email protected]>
@hvanhovell
Copy link
Contributor

Merging to master/4.1. Thanks!

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.

3 participants